You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@hawq.apache.org by tzolov <gi...@git.apache.org> on 2016/01/29 00:14:24 UTC

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

GitHub user tzolov opened a pull request:

    https://github.com/apache/incubator-hawq/pull/302

    HAWQ-178: Add JSON plugin support in code base

    Add `pxf-json` and registers it along the pxf-hdfs, pxf-hive and pxf-hbase plugins.  
    
    Note: I've copied the PxfUnit so i can use the pxf-json unit testing. In log term the PxfUnit should be placed in a standalone project.
    
    Plugin has manually been tested against HAWQ-2.0.0-beta.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/tzolov/incubator-hawq HAWQ-178

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/incubator-hawq/pull/302.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #302
    
----
commit 7faa869ab45212c618e1567d43ee5ccbeb859ae3
Author: tzolov <ch...@gmail.com>
Date:   2015-12-26T18:10:57Z

    HAWQ-178. Initial commit

commit 32e9b4e4d0b2457d521ae6f6dea5bb4dda0945a3
Author: tzolov <ch...@gmail.com>
Date:   2015-12-29T21:39:43Z

    HAWQ-178. Add tests, fix packages

commit 423bacfd3864c5ea375835317993c76228506450
Author: tzolov <ch...@gmail.com>
Date:   2016-01-28T17:18:45Z

    HAWQ-178: Add git ignore rules

commit d8b1600ce7bf5a0991958a2f2225d0c4e863475e
Author: tzolov <ch...@gmail.com>
Date:   2016-01-28T22:55:53Z

    HAWQ-178: add pxf-json jar definition to private calsspaths

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request #302: HAWQ-178: Add JSON plugin support in code ...

Posted by tzolov <gi...@git.apache.org>.
Github user tzolov closed the pull request at:

    https://github.com/apache/incubator-hawq/pull/302


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by tzolov <gi...@git.apache.org>.
Github user tzolov commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51496341
  
    --- Diff: pxf/pxf-json/src/main/java/org/apache/hawq/pxf/plugins/json/JsonAccessor.java ---
    @@ -0,0 +1,66 @@
    +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 org.apache.commons.lang3.StringUtils;
    +import org.apache.hadoop.mapred.FileSplit;
    +import org.apache.hadoop.mapred.InputSplit;
    +import org.apache.hadoop.mapred.JobConf;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +import org.apache.hawq.pxf.plugins.hdfs.HdfsSplittableDataAccessor;
    +
    +/**
    + * This JSON accessor for PXF will read JSON data and pass it to a {@link JsonResolver}.
    + * 
    + * This accessor supports a single JSON record per line, or a more "pretty print" format.
    + */
    +public class JsonAccessor extends HdfsSplittableDataAccessor {
    +
    +	public static final String IDENTIFIER_PARAM = "IDENTIFIER";
    +	public static final String ONERECORDPERLINE_PARAM = "ONERECORDPERLINE";
    +
    +	private String identifier = "";
    +	private boolean oneRecordPerLine = true;
    +
    +	public JsonAccessor(InputData inputData) throws Exception {
    +		super(inputData, new JsonInputFormat());
    +
    +		if (!StringUtils.isEmpty(inputData.getUserProperty(IDENTIFIER_PARAM))) {
    +			identifier = inputData.getUserProperty(IDENTIFIER_PARAM);
    +		}
    +
    +		if (!StringUtils.isEmpty(inputData.getUserProperty(ONERECORDPERLINE_PARAM))) {
    +			oneRecordPerLine = Boolean.parseBoolean(inputData.getUserProperty(ONERECORDPERLINE_PARAM));
    +		}
    +	}
    +
    +	@Override
    +	protected Object getReader(JobConf conf, InputSplit split) throws IOException {
    +		conf.set(JsonInputFormat.RECORD_IDENTIFIER, identifier);
    --- End diff --
    
    moved the set code to the JsonRecordReader mode only


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by tzolov <gi...@git.apache.org>.
Github user tzolov commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51509643
  
    --- Diff: pxf/pxf-json/src/main/java/org/apache/hawq/pxf/plugins/json/JsonResolver.java ---
    @@ -0,0 +1,281 @@
    +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 java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +
    +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 static Pattern ARRAY_PROJECTION_PATTERN = Pattern.compile("(.*)\\[([0-9]+)\\]");
    +
    +	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 = JsonUtil.decodeLineToJsonNode(row.getData().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
    +	 */
    --- End diff --
    
    resolved


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by hornn <gi...@git.apache.org>.
Github user hornn commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51313320
  
    --- 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("]");
    --- End diff --
    
    perhaps better to check using a regex that element name is `.*\[[0-9]+\]`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by hornn <gi...@git.apache.org>.
Github user hornn commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51503825
  
    --- Diff: pxf/pxf-json/src/main/java/org/apache/hawq/pxf/plugins/json/JsonResolver.java ---
    @@ -0,0 +1,281 @@
    +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 java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +
    +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 static Pattern ARRAY_PROJECTION_PATTERN = Pattern.compile("(.*)\\[([0-9]+)\\]");
    +
    +	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 = JsonUtil.decodeLineToJsonNode(row.getData().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
    +	 */
    --- End diff --
    
    /Users/nhorn/dev/hawq/apache-hawq/pxf/pxf-json/src/main/java/org/apache/hawq/pxf/plugins/json/JsonResolver.java:122: warning - @return tag has no arguments.
    1 warning


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by tzolov <gi...@git.apache.org>.
Github user tzolov commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51858357
  
    --- Diff: pxf/pxf-json/src/main/java/org/apache/hawq/pxf/plugins/json/parser/PartitionedJsonParser.java ---
    @@ -0,0 +1,198 @@
    +package org.apache.hawq.pxf.plugins.json.parser;
    +
    +/*
    + * 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.io.InputStream;
    +import java.io.InputStreamReader;
    +import java.nio.charset.StandardCharsets;
    +import java.util.ArrayList;
    +import java.util.EnumSet;
    +import java.util.List;
    +
    +import org.apache.hawq.pxf.plugins.json.parser.JsonLexer.JsonLexerState;
    +
    +/**
    + * A simple parser that can support reading JSON objects from a random point in JSON text. It reads from the supplied
    + * stream (which is assumed to be positioned at any arbitrary position inside some JSON text) until it find the first
    + * JSON begin-object "{". From this point on it will keep reading JSON objects until it finds one containing a member
    + * string that the user supplies.
    + * <p/>
    + * It is not recommended to use this with JSON text where individual JSON objects that can be large (MB's or larger).
    + */
    +public class PartitionedJsonParser {
    +
    +	private static final int EOF = -1;
    +	private final InputStreamReader inputStreamReader;
    +	private final JsonLexer lexer;
    +	private long bytesRead = 0;
    +	private boolean endOfStream;
    +
    +	public PartitionedJsonParser(InputStream is) {
    +		this.lexer = new JsonLexer();
    +
    +		// You need to wrap the InputStream with an InputStreamReader, so that it can encode the incoming byte stream as
    +		// UTF-8 characters
    +		this.inputStreamReader = new InputStreamReader(is, StandardCharsets.UTF_8);
    +	}
    +
    +	private boolean scanToFirstBeginObject() throws IOException {
    +		// seek until we hit the first begin-object
    +		char prev = ' ';
    +		int i;
    +		while ((i = inputStreamReader.read()) != EOF) {
    +			char c = (char) i;
    +			bytesRead++;
    +			if (c == '{' && prev != '\\') {
    --- End diff --
    
    +1


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by tzolov <gi...@git.apache.org>.
Github user tzolov commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r52080930
  
    --- Diff: pxf/pxf-json/src/main/java/org/apache/hawq/pxf/plugins/json/JsonResolver.java ---
    @@ -0,0 +1,252 @@
    +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.util.ArrayList;
    +import java.util.Iterator;
    +import java.util.List;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +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.JsonFactory;
    +import org.codehaus.jackson.JsonNode;
    +import org.codehaus.jackson.map.ObjectMapper;
    +
    +/**
    + * 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 static final Log LOG = LogFactory.getLog(JsonResolver.class);
    +
    +	private ArrayList<OneField> oneFieldList;
    +	private ColumnDescriptorCache[] columnDescriptorCache;
    +	private ObjectMapper mapper;
    +
    +	/**
    +	 * Row with empty fields. Returned in case of broken or malformed json records.
    +	 */
    +	private final List<OneField> emptyRow;
    +
    +	public JsonResolver(InputData inputData) throws Exception {
    +		super(inputData);
    +		oneFieldList = new ArrayList<OneField>();
    +		mapper = new ObjectMapper(new JsonFactory());
    +
    +		// Precompute the column metadata. The metadata is used for mapping column names to json nodes.
    +		columnDescriptorCache = new ColumnDescriptorCache[inputData.getColumns()];
    +		for (int i = 0; i < inputData.getColumns(); ++i) {
    +			ColumnDescriptor cd = inputData.getColumn(i);
    +			columnDescriptorCache[i] = new ColumnDescriptorCache(cd);
    +		}
    +
    +		emptyRow = createEmptyRow();
    +	}
    +
    +	@Override
    +	public List<OneField> getFields(OneRow row) throws Exception {
    +		oneFieldList.clear();
    +
    +		String jsonRecordAsText = row.getData().toString();
    +
    +		JsonNode root = decodeLineToJsonNode(jsonRecordAsText);
    +
    +		if (root == null) {
    +			LOG.warn("Return empty-fields row due to invalid JSON: " + jsonRecordAsText);
    +			return emptyRow;
    +		}
    +
    +		// Iterate through the column definition and fetch our JSON data
    +		for (ColumnDescriptorCache columnMetadata : columnDescriptorCache) {
    +
    +			JsonNode node = getChildJsonNode(root, columnMetadata.getNormalizedProjections());
    +
    +			// If this node is null or missing, add a null value here
    +			if (node == null || node.isMissingNode()) {
    +				addNullField(columnMetadata.getColumnType());
    +			} else if (columnMetadata.isArray()) {
    +				// If this column is an array index, ex. "tweet.hashtags[0]"
    +				if (node.isArray()) {
    +					// If the JSON node is an array, then add it to our list
    +					addFieldFromJsonArray(columnMetadata.getColumnType(), node, columnMetadata.getArrayNodeIndex());
    +				} else {
    +					throw new IllegalStateException(columnMetadata.getColumnName() + " is not an array node");
    +				}
    +			} else {
    +				// This column is not an array type
    +				// Add the value to the record
    +				addFieldFromJsonNode(columnMetadata.getColumnType(), node);
    +			}
    +		}
    +
    +		return oneFieldList;
    +	}
    +
    +	/**
    +	 * @return Returns a row comprised of typed, empty fields. Used as a result of broken/malformed json records.
    +	 */
    +	private List<OneField> createEmptyRow() {
    +		ArrayList<OneField> emptyFieldList = new ArrayList<OneField>();
    +		for (ColumnDescriptorCache column : columnDescriptorCache) {
    +			emptyFieldList.add(new OneField(column.getColumnType().getOID(), null));
    +		}
    +		return emptyFieldList;
    +	}
    +
    +	/**
    +	 * Iterates down the root node to the child JSON node defined by the projs path.
    +	 * 
    +	 * @param root
    +	 *            node to to start the traversal from.
    +	 * @param projs
    +	 *            defines the path from the root to the desired child node.
    +	 * @return Returns the child node defined by the root and projs path.
    +	 */
    +	private JsonNode getChildJsonNode(JsonNode root, String[] projs) {
    +
    +		// Iterate through all the tokens to the desired JSON node
    +		JsonNode node = root;
    +		for (int j = 0; j < projs.length; ++j) {
    +			node = node.path(projs[j]);
    +		}
    +
    +		return node;
    +	}
    +
    +	/**
    +	 * Iterates through the given JSON node to the proper index and adds the field of corresponding type
    +	 * 
    +	 * @param type
    +	 *            The {@link DataType} type
    +	 * @param node
    +	 *            The JSON array node
    +	 * @param index
    +	 *            The array index to iterate to
    +	 * @throws IOException
    +	 */
    +	private void addFieldFromJsonArray(DataType type, JsonNode node, int index) throws IOException {
    +
    +		int count = 0;
    +		boolean added = false;
    +		for (Iterator<JsonNode> arrayNodes = node.getElements(); arrayNodes.hasNext();) {
    +			JsonNode arrayNode = arrayNodes.next();
    +
    +			if (count == index) {
    +				added = true;
    +				addFieldFromJsonNode(type, arrayNode);
    +				break;
    +			}
    +
    +			++count;
    +		}
    +
    +		// if we reached the end of the array without adding a field, add null
    +		if (!added) {
    +			addNullField(type);
    +		}
    +	}
    +
    +	/**
    +	 * Adds a field from a given JSON node value based on the {@link DataType} type.
    +	 * 
    +	 * @param type
    +	 *            The DataType type
    +	 * @param val
    +	 *            The JSON node to extract the value.
    +	 * @throws IOException
    +	 */
    +	private void addFieldFromJsonNode(DataType type, JsonNode val) throws IOException {
    +		OneField oneField = new OneField();
    +		oneField.type = type.getOID();
    +
    +		if (val.isNull()) {
    +			oneField.val = null;
    +		} else {
    +			switch (type) {
    +			case BIGINT:
    +				oneField.val = val.asLong();
    +				break;
    +			case BOOLEAN:
    +				oneField.val = val.asBoolean();
    +				break;
    +			case CHAR:
    +				oneField.val = val.asText().charAt(0);
    +				break;
    +			case BYTEA:
    +				oneField.val = val.asText().getBytes();
    +				break;
    +			case FLOAT8:
    +			case REAL:
    +				oneField.val = val.asDouble();
    +				break;
    +			case INTEGER:
    +			case SMALLINT:
    +				oneField.val = val.asInt();
    +				break;
    +			case BPCHAR:
    +			case TEXT:
    +			case VARCHAR:
    +				oneField.val = val.asText();
    +				break;
    +			default:
    +				throw new IOException("Unsupported type " + type);
    +			}
    +		}
    +
    +		oneFieldList.add(oneField);
    +	}
    +
    +	/**
    +	 * Adds a null field of the given type.
    +	 * 
    +	 * @param type
    +	 *            The {@link DataType} type
    +	 */
    +	private void addNullField(DataType type) {
    +		oneFieldList.add(new OneField(type.getOID(), null));
    +	}
    +
    +	/**
    +	 * Converts the input line parameter into {@link JsonNode} instance.
    +	 * 
    +	 * @param line
    +	 *            JSON text
    +	 * @return Returns a {@link JsonNode} that represents the input line or null for invalid json.
    +	 */
    +	private JsonNode decodeLineToJsonNode(String line) {
    +
    +		try {
    +			return mapper.readTree(line);
    +		} catch (Exception e) {
    +			LOG.error("Failed to pars JSON object", e);
    --- End diff --
    
    +1


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by tzolov <gi...@git.apache.org>.
Github user tzolov commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51857642
  
    --- Diff: pxf/pxf-json/src/main/java/org/apache/hawq/pxf/plugins/json/parser/PartitionedJsonParser.java ---
    @@ -0,0 +1,198 @@
    +package org.apache.hawq.pxf.plugins.json.parser;
    +
    +/*
    + * 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.io.InputStream;
    +import java.io.InputStreamReader;
    +import java.nio.charset.StandardCharsets;
    +import java.util.ArrayList;
    +import java.util.EnumSet;
    +import java.util.List;
    +
    +import org.apache.hawq.pxf.plugins.json.parser.JsonLexer.JsonLexerState;
    +
    +/**
    + * A simple parser that can support reading JSON objects from a random point in JSON text. It reads from the supplied
    + * stream (which is assumed to be positioned at any arbitrary position inside some JSON text) until it find the first
    --- End diff --
    
    done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq issue #302: HAWQ-178: Add JSON plugin support in code base

Posted by changleicn <gi...@git.apache.org>.
Github user changleicn commented on the issue:

    https://github.com/apache/incubator-hawq/pull/302
  
    @tzolov the code is in, can you please close this PR?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by hornn <gi...@git.apache.org>.
Github user hornn commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51835264
  
    --- Diff: pxf/pxf-json/src/test/java/org/apache/hawq/pxf/plugins/json/JsonExtensionTest.java ---
    @@ -0,0 +1,288 @@
    +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.File;
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import org.apache.hadoop.fs.Path;
    +import org.apache.hawq.pxf.api.Fragmenter;
    +import org.apache.hawq.pxf.api.ReadAccessor;
    +import org.apache.hawq.pxf.api.ReadResolver;
    +import org.apache.hawq.pxf.api.io.DataType;
    +import org.apache.hawq.pxf.plugins.hdfs.HdfsDataFragmenter;
    +import org.apache.hawq.pxf.plugins.json.JsonAccessor;
    +import org.apache.hawq.pxf.plugins.json.JsonResolver;
    +import org.junit.After;
    +import org.junit.Before;
    +import org.junit.Test;
    +
    +public class JsonExtensionTest extends PxfUnit {
    +
    +	private List<Pair<String, DataType>> columnDefs = null;
    +	private List<Pair<String, String>> extraParams = new ArrayList<Pair<String, String>>();
    +
    +	@Before
    +	public void before() {
    +
    +		columnDefs = new ArrayList<Pair<String, DataType>>();
    +
    +		columnDefs.add(new Pair<String, DataType>("created_at", DataType.TEXT));
    +		columnDefs.add(new Pair<String, DataType>("id", DataType.BIGINT));
    +		columnDefs.add(new Pair<String, DataType>("text", DataType.TEXT));
    +		columnDefs.add(new Pair<String, DataType>("user.screen_name", DataType.TEXT));
    +		columnDefs.add(new Pair<String, DataType>("entities.hashtags[0]", DataType.TEXT));
    +		columnDefs.add(new Pair<String, DataType>("coordinates.coordinates[0]", DataType.FLOAT8));
    +		columnDefs.add(new Pair<String, DataType>("coordinates.coordinates[1]", DataType.FLOAT8));
    +	}
    +
    +	@After
    +	public void cleanup() throws Exception {
    +		columnDefs.clear();
    +		extraParams.clear();
    +	}
    +
    +	@Test
    +	public void testCompressedMultilineJsonFile() throws Exception {
    +
    +		extraParams.add(new Pair<String, String>("IDENTIFIER", "created_at"));
    +
    +		List<String> output = new ArrayList<String>();
    +
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547115253761,text1,SpreadButter,tweetCongress,,");
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547123646465,text2,patronusdeadly,,,");
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547136233472,text3,NoSecrets_Vagas,,,");
    +
    +		super.assertOutput(new Path(System.getProperty("user.dir") + File.separator
    +				+ "src/test/resources/tweets.tar.gz"), output);
    +	}
    +
    +	@Test
    +	public void testMaxRecordLength() throws Exception {
    +
    +		// variable-size-objects.json contains 3 json objects but only 2 of them fit in the 27 byte length limitation
    +
    +		extraParams.add(new Pair<String, String>("IDENTIFIER", "key666"));
    +		extraParams.add(new Pair<String, String>("MAXLENGTH", "27"));
    +
    +		columnDefs.clear();
    +		columnDefs.add(new Pair<String, DataType>("key666", DataType.TEXT));
    +
    +		List<String> output = new ArrayList<String>();
    +
    +		output.add("small object1");
    +		// skip the large object2 XXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX
    +		output.add("small object3");
    +
    +		super.assertOutput(new Path(System.getProperty("user.dir") + File.separator
    +				+ "src/test/resources/variable-size-objects.json"), output);
    +	}
    +
    +	@Test
    +	public void testDataTypes() throws Exception {
    +
    +		// TDOO: The BYTEA type is not tested!!! Current oneField.val = val.asText().getBytes(); convention is
    --- End diff --
    
    Let's open a sub-task for it, what do you think?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by hornn <gi...@git.apache.org>.
Github user hornn commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51835836
  
    --- Diff: pxf/pxf-json/src/test/java/org/apache/hawq/pxf/plugins/json/PxfUnit.java ---
    @@ -0,0 +1,672 @@
    +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.BufferedReader;
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.InputStreamReader;
    +import java.io.OutputStream;
    +import java.lang.reflect.Constructor;
    +import java.security.InvalidParameterException;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.fs.FileSystem;
    +import org.apache.hadoop.fs.Path;
    +import org.apache.hawq.pxf.api.Fragment;
    +import org.apache.hawq.pxf.api.Fragmenter;
    +import org.apache.hawq.pxf.api.OneField;
    +import org.apache.hawq.pxf.api.OneRow;
    +import org.apache.hawq.pxf.api.ReadAccessor;
    +import org.apache.hawq.pxf.api.ReadResolver;
    +import org.apache.hawq.pxf.api.WriteAccessor;
    +import org.apache.hawq.pxf.api.WriteResolver;
    +import org.apache.hawq.pxf.api.io.DataType;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +import org.apache.hawq.pxf.service.FragmentsResponse;
    +import org.apache.hawq.pxf.service.FragmentsResponseFormatter;
    +import org.apache.hawq.pxf.service.utilities.ProtocolData;
    +import org.codehaus.jackson.JsonFactory;
    +import org.codehaus.jackson.JsonNode;
    +import org.codehaus.jackson.JsonParseException;
    +import org.codehaus.jackson.map.JsonMappingException;
    +import org.codehaus.jackson.map.ObjectMapper;
    +import org.junit.Assert;
    +
    +/**
    + * This abstract class contains a number of helpful utilities in developing a PXF extension for HAWQ. Extend this class
    + * and use the various <code>assert</code> methods to check given input against known output.
    + */
    +public abstract class PxfUnit {
    +
    +	private static final Log LOG = LogFactory.getLog(PxfUnit.class);
    +
    +	private static JsonFactory factory = new JsonFactory();
    +	private static ObjectMapper mapper = new ObjectMapper(factory);
    +
    +	protected static List<InputData> inputs = null;
    +
    +	/**
    +	 * Uses the given input directory to run through the PXF unit testing framework. Uses the lines in the file for
    +	 * output testing.
    +	 * 
    +	 * @param input
    +	 *            Input records
    +	 * @param expectedOutput
    +	 *            File containing output to check
    +	 * @throws Exception
    +	 */
    +	public void assertOutput(Path input, Path expectedOutput) throws Exception {
    +
    +		BufferedReader rdr = new BufferedReader(new InputStreamReader(FileSystem.get(new Configuration()).open(
    +				expectedOutput)));
    +
    +		List<String> outputLines = new ArrayList<String>();
    +
    +		String line;
    +		while ((line = rdr.readLine()) != null) {
    +			outputLines.add(line);
    +		}
    +
    +		assertOutput(input, outputLines);
    +
    +		rdr.close();
    +	}
    +
    +	/**
    +	 * Uses the given input directory to run through the PXF unit testing framework. Uses the lines in the given
    +	 * parameter for output testing.
    +	 * 
    +	 * @param input
    +	 *            Input records
    +	 * @param expectedOutput
    +	 *            File containing output to check
    +	 * @throws Exception
    +	 */
    +	public void assertOutput(Path input, List<String> expectedOutput) throws Exception {
    +
    +		setup(input);
    +		List<String> actualOutput = new ArrayList<String>();
    +		for (InputData data : inputs) {
    +			ReadAccessor accessor = getReadAccessor(data);
    +			ReadResolver resolver = getReadResolver(data);
    +
    +			actualOutput.addAll(getAllOutput(accessor, resolver));
    +		}
    +
    +		Assert.assertFalse("Output did not match expected output", compareOutput(expectedOutput, actualOutput));
    +	}
    +
    +	/**
    +	 * Uses the given input directory to run through the PXF unit testing framework. Uses the lines in the given
    +	 * parameter for output testing.<br>
    +	 * <br>
    +	 * Ignores order of records.
    +	 * 
    +	 * @param input
    +	 *            Input records
    +	 * @param expectedOutput
    +	 *            File containing output to check
    +	 * @throws Exception
    +	 */
    +	public void assertUnorderedOutput(Path input, Path expectedOutput) throws Exception {
    +		BufferedReader rdr = new BufferedReader(new InputStreamReader(FileSystem.get(new Configuration()).open(
    +				expectedOutput)));
    +
    +		List<String> outputLines = new ArrayList<String>();
    +
    +		String line;
    +		while ((line = rdr.readLine()) != null) {
    +			outputLines.add(line);
    +		}
    +
    +		assertUnorderedOutput(input, outputLines);
    +	}
    +
    +	/**
    +	 * Uses the given input directory to run through the PXF unit testing framework. Uses the lines in the file for
    +	 * output testing.<br>
    +	 * <br>
    +	 * Ignores order of records.
    +	 * 
    +	 * @param input
    +	 *            Input records
    +	 * @param expectedOutput
    +	 *            File containing output to check
    +	 * @throws Exception
    +	 */
    +	public void assertUnorderedOutput(Path input, List<String> expectedOutput) throws Exception {
    +
    +		setup(input);
    +
    +		List<String> actualOutput = new ArrayList<String>();
    +		for (InputData data : inputs) {
    +			ReadAccessor accessor = getReadAccessor(data);
    +			ReadResolver resolver = getReadResolver(data);
    +
    +			actualOutput.addAll(getAllOutput(accessor, resolver));
    +		}
    +
    +		Assert.assertFalse("Output did not match expected output", compareUnorderedOutput(expectedOutput, actualOutput));
    +	}
    +
    +	/**
    +	 * Writes the output to the given output stream. Comma delimiter.
    +	 * 
    +	 * @param input
    +	 *            The input file
    +	 * @param output
    +	 *            The output stream
    +	 * @throws Exception
    +	 */
    +	public void writeOutput(Path input, OutputStream output) throws Exception {
    +
    +		setup(input);
    +
    +		for (InputData data : inputs) {
    +			ReadAccessor accessor = getReadAccessor(data);
    +			ReadResolver resolver = getReadResolver(data);
    +
    +			for (String line : getAllOutput(accessor, resolver)) {
    +				output.write((line + "\n").getBytes());
    +			}
    +		}
    +
    +		output.flush();
    +	}
    +
    +	/**
    +	 * Get the class of the implementation of Fragmenter to be tested.
    +	 * 
    +	 * @return The class
    +	 */
    +	public Class<? extends Fragmenter> getFragmenterClass() {
    +		return null;
    +	}
    +
    +	/**
    +	 * Get the class of the implementation of ReadAccessor to be tested.
    +	 * 
    +	 * @return The class
    +	 */
    +	public Class<? extends ReadAccessor> getReadAccessorClass() {
    +		return null;
    +	}
    +
    +	/**
    +	 * Get the class of the implementation of WriteAccessor to be tested.
    +	 * 
    +	 * @return The class
    +	 */
    +	public Class<? extends WriteAccessor> getWriteAccessorClass() {
    +		return null;
    +	}
    +
    +	/**
    +	 * Get the class of the implementation of Resolver to be tested.
    +	 * 
    +	 * @return The class
    +	 */
    +	public Class<? extends ReadResolver> getReadResolverClass() {
    +		return null;
    +	}
    +
    +	/**
    +	 * Get the class of the implementation of WriteResolver to be tested.
    +	 * 
    +	 * @return The class
    +	 */
    +	public Class<? extends WriteResolver> getWriteResolverClass() {
    +		return null;
    +	}
    +
    +	/**
    +	 * Get any extra parameters that are meant to be specified for the "pxf" protocol. Note that "X-GP-" is prepended to
    +	 * each parameter name.
    +	 * 
    +	 * @return Any extra parameters or null if none.
    +	 */
    +	public List<Pair<String, String>> getExtraParams() {
    +		return null;
    +	}
    +
    +	/**
    +	 * Gets the column definition names and data types. Types are DataType objects
    +	 * 
    +	 * @return A list of column definition name value pairs. Cannot be null.
    +	 */
    +	public abstract List<Pair<String, DataType>> getColumnDefinitions();
    +
    +	protected InputData getInputDataForWritableTable() {
    +		return getInputDataForWritableTable(null);
    +	}
    +
    +	protected InputData getInputDataForWritableTable(Path input) {
    +
    +		if (getWriteAccessorClass() == null) {
    +			throw new IllegalArgumentException(
    +					"getWriteAccessorClass() must be overwritten to return a non-null object");
    +		}
    +
    +		if (getWriteResolverClass() == null) {
    +			throw new IllegalArgumentException(
    +					"getWriteResolverClass() must be overwritten to return a non-null object");
    +		}
    +
    +		Map<String, String> paramsMap = new HashMap<String, String>();
    +
    +		paramsMap.put("X-GP-ALIGNMENT", "what");
    +		paramsMap.put("X-GP-SEGMENT-ID", "1");
    +		paramsMap.put("X-GP-HAS-FILTER", "0");
    +		paramsMap.put("X-GP-SEGMENT-COUNT", "1");
    +
    +		paramsMap.put("X-GP-FORMAT", "GPDBWritable");
    +		paramsMap.put("X-GP-URL-HOST", "localhost");
    +		paramsMap.put("X-GP-URL-PORT", "50070");
    +
    +		if (input == null) {
    +			paramsMap.put("X-GP-DATA-DIR", "/dummydata");
    +		}
    +
    +		List<Pair<String, DataType>> params = getColumnDefinitions();
    +		paramsMap.put("X-GP-ATTRS", Integer.toString(params.size()));
    +		for (int i = 0; i < params.size(); ++i) {
    +			paramsMap.put("X-GP-ATTR-NAME" + i, params.get(i).first);
    +			paramsMap.put("X-GP-ATTR-TYPENAME" + i, params.get(i).second.name());
    +			paramsMap.put("X-GP-ATTR-TYPECODE" + i, Integer.toString(params.get(i).second.getOID()));
    +		}
    +
    +		paramsMap.put("X-GP-ACCESSOR", getWriteAccessorClass().getName());
    +		paramsMap.put("X-GP-RESOLVER", getWriteResolverClass().getName());
    +
    +		if (getExtraParams() != null) {
    +			for (Pair<String, String> param : getExtraParams()) {
    +				paramsMap.put("X-GP-" + param.first, param.second);
    +			}
    +		}
    +
    +		return new ProtocolData(paramsMap);
    +	}
    +
    +	/**
    +	 * Set all necessary parameters for GPXF framework to function. Uses the given path as a single input split.
    +	 * 
    +	 * @param input
    +	 *            The input path, relative or absolute.
    +	 * @throws Exception
    +	 */
    +	protected void setup(Path input) throws Exception {
    +
    +		if (getFragmenterClass() == null) {
    +			throw new IllegalArgumentException("getFragmenterClass() must be overwritten to return a non-null object");
    +		}
    +
    +		if (getReadAccessorClass() == null) {
    +			throw new IllegalArgumentException("getReadAccessorClass() must be overwritten to return a non-null object");
    +		}
    +
    +		if (getReadResolverClass() == null) {
    +			throw new IllegalArgumentException("getReadResolverClass() must be overwritten to return a non-null object");
    +		}
    +
    +		Map<String, String> paramsMap = new HashMap<String, String>();
    +
    +		// 2.1.0 Properties
    +		// HDMetaData parameters
    +		paramsMap.put("X-GP-ALIGNMENT", "what");
    +		paramsMap.put("X-GP-SEGMENT-ID", "1");
    +		paramsMap.put("X-GP-HAS-FILTER", "0");
    +		paramsMap.put("X-GP-SEGMENT-COUNT", "1");
    +		paramsMap.put("X-GP-FRAGMENTER", getFragmenterClass().getName());
    +		paramsMap.put("X-GP-FORMAT", "GPDBWritable");
    +		paramsMap.put("X-GP-URL-HOST", "localhost");
    +		paramsMap.put("X-GP-URL-PORT", "50070");
    +
    +		paramsMap.put("X-GP-DATA-DIR", input.toString());
    +
    +		List<Pair<String, DataType>> params = getColumnDefinitions();
    +		paramsMap.put("X-GP-ATTRS", Integer.toString(params.size()));
    +		for (int i = 0; i < params.size(); ++i) {
    +			paramsMap.put("X-GP-ATTR-NAME" + i, params.get(i).first);
    +			paramsMap.put("X-GP-ATTR-TYPENAME" + i, params.get(i).second.name());
    +			paramsMap.put("X-GP-ATTR-TYPECODE" + i, Integer.toString(params.get(i).second.getOID()));
    +		}
    +
    +		// HDFSMetaData properties
    +		paramsMap.put("X-GP-ACCESSOR", getReadAccessorClass().getName());
    +		paramsMap.put("X-GP-RESOLVER", getReadResolverClass().getName());
    +
    +		if (getExtraParams() != null) {
    +			for (Pair<String, String> param : getExtraParams()) {
    +				paramsMap.put("X-GP-" + param.first, param.second);
    +			}
    +		}
    +
    +		LocalInputData fragmentInputData = new LocalInputData(paramsMap);
    +
    +		List<Fragment> fragments = getFragmenter(fragmentInputData).getFragments();
    +
    +		FragmentsResponse fragmentsResponse = FragmentsResponseFormatter.formatResponse(fragments, input.toString());
    +
    +		ByteArrayOutputStream baos = new ByteArrayOutputStream();
    +		fragmentsResponse.write(baos);
    +
    +		String jsonOutput = baos.toString();
    +
    +		inputs = new ArrayList<InputData>();
    +
    +		JsonNode node = decodeLineToJsonNode(jsonOutput);
    +
    +		JsonNode fragmentsArray = node.get("PXFFragments");
    +		int i = 0;
    +		Iterator<JsonNode> iter = fragmentsArray.getElements();
    +		while (iter.hasNext()) {
    +			JsonNode fragNode = iter.next();
    +			String sourceData = fragNode.get("sourceName").getTextValue();
    +			if (!sourceData.startsWith("/")) {
    +				sourceData = "/" + sourceData;
    +			}
    +			paramsMap.put("X-GP-DATA-DIR", sourceData);
    +			paramsMap.put("X-GP-FRAGMENT-METADATA", fragNode.get("metadata").getTextValue());
    +			paramsMap.put("X-GP-DATA-FRAGMENT", Integer.toString(i++));
    +			inputs.add(new LocalInputData(paramsMap));
    +		}
    +	}
    +
    +	private JsonNode decodeLineToJsonNode(String line) {
    +
    +		try {
    +			return mapper.readTree(line);
    +		} catch (JsonParseException e) {
    +			e.printStackTrace();
    --- End diff --
    
    replace with LOG.debug/warn


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by tzolov <gi...@git.apache.org>.
Github user tzolov commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51345896
  
    --- Diff: pxf/build.gradle ---
    @@ -64,7 +64,7 @@ subprojects { subProject ->
                 "-Xlint:finally", "-Xlint:overrides", "-Xlint:path", "-Xlint:processing", "-Xlint:static", "-Xlint:try",
                 "-Xlint:-fallthrough", "-Xlint:-deprecation", "-Xlint:-unchecked", "-Xlint:-options"
         ]
    -
    +		
    --- End diff --
    
    Should we remove all whitespaces in the build.gradle file?  There are quite a few. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by tzolov <gi...@git.apache.org>.
Github user tzolov commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51509630
  
    --- Diff: pxf/pxf-json/src/test/java/org/apache/pxf/hawq/plugins/json/JsonExtensionTest.java ---
    @@ -0,0 +1,175 @@
    +package org.apache.pxf.hawq.plugins.json;
    --- End diff --
    
    ops. good spot!. resolved


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by adamjshook <gi...@git.apache.org>.
Github user adamjshook commented on the pull request:

    https://github.com/apache/incubator-hawq/pull/302#issuecomment-178133851
  
    @GodenYao You're very welcome!  I'll try and find some time to contribute the various prototype connectors I have written.  @tzolov shouldn't have to deal with all of my code!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by hornn <gi...@git.apache.org>.
Github user hornn commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51828253
  
    --- Diff: pxf/pxf-json/src/main/java/org/apache/hawq/pxf/plugins/json/JsonResolver.java ---
    @@ -0,0 +1,255 @@
    +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.util.ArrayList;
    +import java.util.Iterator;
    +import java.util.List;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +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.JsonFactory;
    +import org.codehaus.jackson.JsonNode;
    +import org.codehaus.jackson.map.ObjectMapper;
    +
    +/**
    + * 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 static final Log LOG = LogFactory.getLog(JsonResolver.class);
    +
    +	private ArrayList<OneField> oneFieldList;
    +	private ColumnDescriptorCache[] columnDescriptorCache;
    +	private ObjectMapper mapper;
    +
    +	public JsonResolver(InputData inputData) throws Exception {
    +		super(inputData);
    +		oneFieldList = new ArrayList<OneField>();
    +		mapper = new ObjectMapper(new JsonFactory());
    +
    +		// Pre-generate all column structure attributes concerning the JSON to column value resolution.
    +		columnDescriptorCache = new ColumnDescriptorCache[inputData.getColumns()];
    +		for (int i = 0; i < inputData.getColumns(); ++i) {
    +			ColumnDescriptor cd = inputData.getColumn(i);
    +			columnDescriptorCache[i] = new ColumnDescriptorCache(cd);
    +		}
    +	}
    +
    +	@Override
    +	public List<OneField> getFields(OneRow row) throws Exception {
    +		oneFieldList.clear();
    +
    +		String jsonRecordAsText = row.getData().toString();
    +
    +		JsonNode root = decodeLineToJsonNode(jsonRecordAsText);
    +
    +		if (root == null) {
    +			LOG.warn("Return null-fields row due to invalid JSON:" + jsonRecordAsText);
    +		}
    +
    +		// Iterate through the column definition and fetch our JSON data
    +		for (ColumnDescriptorCache column : columnDescriptorCache) {
    +
    +			// Get the current column description
    +
    +			if (root == null) {
    +				// Return empty (e.g. null) filed in case of malformed json.
    +				addNullField(column.getColumnType());
    +			} else {
    +
    +				// Move down the JSON path to the final name
    +				JsonNode node = getPriorJsonNode(root, column.getProjections());
    +
    +				// If this column is an array index, ex. "tweet.hashtags[0]"
    +				if (column.isArrayName()) {
    +
    +					// Move to the array node
    +					node = node.get(column.getArrayNodeName());
    +
    +					// If this node is null or missing, add a null value here
    +					if (node == null || node.isMissingNode()) {
    +						addNullField(column.getColumnType());
    +					} else if (node.isArray()) {
    +						// If the JSON node is an array, then add it to our list
    +						addFieldFromJsonArray(column.getColumnType(), node, column.getArrayIndex());
    +					} else {
    +						throw new IllegalStateException(column.getArrayNodeName() + " is not an array node");
    +					}
    +				} else {
    +					// This column is not an array type
    +					// Move to the final node
    +					node = node.get(column.getLastProjection());
    +
    +					// If this node is null or missing, add a null value here
    +					if (node == null || node.isMissingNode()) {
    +						addNullField(column.getColumnType());
    +					} else {
    +						// Else, add the value to the record
    +						addFieldFromJsonNode(column.getColumnType(), node);
    +					}
    +				}
    +			}
    +		}
    +
    +		return oneFieldList;
    +	}
    +
    +	/**
    +	 * Iterates down the root node to the child JSON node defined by the projs path.
    +	 * 
    +	 * @param root
    +	 *            node to to start the traversal from.
    +	 * @param projs
    +	 *            defines the path from the root to the desired child node.
    +	 * @return Returns the child node defined by the root and projs path.
    +	 */
    +	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) {
    --- End diff --
    
    just wondering - why is it `proj.length -1` and not `proj.length`? 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by hornn <gi...@git.apache.org>.
Github user hornn commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51833835
  
    --- Diff: pxf/pxf-json/src/main/java/org/apache/hawq/pxf/plugins/json/parser/PartitionedJsonParser.java ---
    @@ -0,0 +1,198 @@
    +package org.apache.hawq.pxf.plugins.json.parser;
    +
    +/*
    + * 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.io.InputStream;
    +import java.io.InputStreamReader;
    +import java.nio.charset.StandardCharsets;
    +import java.util.ArrayList;
    +import java.util.EnumSet;
    +import java.util.List;
    +
    +import org.apache.hawq.pxf.plugins.json.parser.JsonLexer.JsonLexerState;
    +
    +/**
    + * A simple parser that can support reading JSON objects from a random point in JSON text. It reads from the supplied
    + * stream (which is assumed to be positioned at any arbitrary position inside some JSON text) until it find the first
    --- End diff --
    
    typo finds


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by adamjshook <gi...@git.apache.org>.
Github user adamjshook commented on the pull request:

    https://github.com/apache/incubator-hawq/pull/302#issuecomment-177992806
  
    I thought I had handled this case in the `JsonRecordReader` code.  The `JsonStreamReader` has no knowledge of the 'end' location of the FileSplit and will read until it completes a JSON record on the call to `getJsonRecord`.  The `JsonInputStream` only returns false on a `next` call when the number of bytes read by the stream reader is greater than or equal to `end - start`.  It will ignore the split boundaries and continue to reading to make a full record.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by tzolov <gi...@git.apache.org>.
Github user tzolov commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51513424
  
    --- Diff: pxf/pxf-json/src/main/java/org/apache/hawq/pxf/plugins/json/JsonRecordReader.java ---
    @@ -0,0 +1,166 @@
    +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 static org.apache.commons.lang3.StringUtils.isEmpty;
    +
    +import java.io.BufferedInputStream;
    +import java.io.IOException;
    +import java.security.InvalidParameterException;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.hadoop.fs.FSDataInputStream;
    +import org.apache.hadoop.fs.FileSystem;
    +import org.apache.hadoop.fs.Path;
    +import org.apache.hadoop.io.LongWritable;
    +import org.apache.hadoop.io.Text;
    +import org.apache.hadoop.mapred.FileSplit;
    +import org.apache.hadoop.mapred.JobConf;
    +import org.apache.hadoop.mapred.RecordReader;
    +import org.apache.hawq.pxf.api.Fragment;
    +
    +/**
    + * {@link RecordReader} implementation that can read multiline JSON objects.
    + */
    +public class JsonRecordReader implements RecordReader<LongWritable, Text> {
    +
    +	private static final Log LOG = LogFactory.getLog(JsonRecordReader.class);
    +
    +	public static final String RECORD_IDENTIFIER = "json.input.format.record.identifier";
    +
    +	private JsonStreamReader streamReader = null;
    +	private long start = 0, end = 0;
    +	private float toRead = 0;
    +	private String identifier = null;
    +
    +	/**
    +	 * @param conf
    +	 *            Hadoop context
    +	 * @param split
    +	 *            HDFS Split as defined by the {@link Fragment}.
    +	 * @throws IOException
    +	 */
    +	public JsonRecordReader(JobConf conf, FileSplit split) throws IOException {
    +		LOG.debug("JsonRecordReader constructor called.  Conf is " + conf + ". Split is " + split);
    +		this.identifier = conf.get(RECORD_IDENTIFIER);
    +		LOG.debug("Identifier is " + this.identifier);
    +
    +		if (isEmpty(this.identifier)) {
    +			throw new InvalidParameterException(RECORD_IDENTIFIER + " is not set.");
    +		} else {
    +			LOG.debug("Initializing JsonRecordReader with identifier " + identifier);
    +		}
    +
    +		// get relevant data
    +		Path file = split.getPath();
    +
    +		LOG.debug("File is " + file);
    +
    +		start = split.getStart();
    +		end = start + split.getLength();
    +		toRead = end - start;
    +		LOG.debug("FileSystem is " + FileSystem.get(conf));
    +
    +		FSDataInputStream strm = FileSystem.get(conf).open(file);
    +
    +		LOG.debug("Retrieved file stream ");
    +
    +		if (start != 0) {
    +			strm.seek(start);
    +		}
    +
    +		streamReader = new JsonStreamReader(identifier, new BufferedInputStream(strm));
    +
    +		LOG.debug("Reader is " + streamReader);
    +	}
    +
    +	/*
    +	 * {@inheritDoc}
    +	 */
    +	@Override
    +	public boolean next(LongWritable key, Text value) throws IOException {
    +
    +		boolean retval = false;
    +		boolean keepGoing = false;
    +		do {
    +			// Exit condition (end of block/file)
    +			if (streamReader.getBytesRead() >= (end - start)) {
    +				return false;
    +			}
    +
    +			keepGoing = false;
    +			String record = streamReader.getJsonRecord();
    +			if (record != null) {
    +				if (JsonUtil.decodeLineToJsonNode(record) == null) {
    --- End diff --
    
    Did this and removed the JsonUtil all together. Parsing logic is moved directly in the resolver. Also i think it is better to log an error in case of malformed json object but to continue the processing (e.g. no exception is thrown)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by hornn <gi...@git.apache.org>.
Github user hornn commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51835237
  
    --- Diff: pxf/pxf-json/src/test/java/org/apache/hawq/pxf/plugins/json/JsonExtensionTest.java ---
    @@ -0,0 +1,288 @@
    +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.File;
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import org.apache.hadoop.fs.Path;
    +import org.apache.hawq.pxf.api.Fragmenter;
    +import org.apache.hawq.pxf.api.ReadAccessor;
    +import org.apache.hawq.pxf.api.ReadResolver;
    +import org.apache.hawq.pxf.api.io.DataType;
    +import org.apache.hawq.pxf.plugins.hdfs.HdfsDataFragmenter;
    +import org.apache.hawq.pxf.plugins.json.JsonAccessor;
    +import org.apache.hawq.pxf.plugins.json.JsonResolver;
    +import org.junit.After;
    +import org.junit.Before;
    +import org.junit.Test;
    +
    +public class JsonExtensionTest extends PxfUnit {
    +
    +	private List<Pair<String, DataType>> columnDefs = null;
    +	private List<Pair<String, String>> extraParams = new ArrayList<Pair<String, String>>();
    +
    +	@Before
    +	public void before() {
    +
    +		columnDefs = new ArrayList<Pair<String, DataType>>();
    +
    +		columnDefs.add(new Pair<String, DataType>("created_at", DataType.TEXT));
    +		columnDefs.add(new Pair<String, DataType>("id", DataType.BIGINT));
    +		columnDefs.add(new Pair<String, DataType>("text", DataType.TEXT));
    +		columnDefs.add(new Pair<String, DataType>("user.screen_name", DataType.TEXT));
    +		columnDefs.add(new Pair<String, DataType>("entities.hashtags[0]", DataType.TEXT));
    +		columnDefs.add(new Pair<String, DataType>("coordinates.coordinates[0]", DataType.FLOAT8));
    +		columnDefs.add(new Pair<String, DataType>("coordinates.coordinates[1]", DataType.FLOAT8));
    +	}
    +
    +	@After
    +	public void cleanup() throws Exception {
    +		columnDefs.clear();
    +		extraParams.clear();
    +	}
    +
    +	@Test
    +	public void testCompressedMultilineJsonFile() throws Exception {
    +
    +		extraParams.add(new Pair<String, String>("IDENTIFIER", "created_at"));
    +
    +		List<String> output = new ArrayList<String>();
    +
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547115253761,text1,SpreadButter,tweetCongress,,");
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547123646465,text2,patronusdeadly,,,");
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547136233472,text3,NoSecrets_Vagas,,,");
    +
    +		super.assertOutput(new Path(System.getProperty("user.dir") + File.separator
    +				+ "src/test/resources/tweets.tar.gz"), output);
    +	}
    +
    +	@Test
    +	public void testMaxRecordLength() throws Exception {
    +
    +		// variable-size-objects.json contains 3 json objects but only 2 of them fit in the 27 byte length limitation
    +
    +		extraParams.add(new Pair<String, String>("IDENTIFIER", "key666"));
    +		extraParams.add(new Pair<String, String>("MAXLENGTH", "27"));
    +
    +		columnDefs.clear();
    --- End diff --
    
    I think the clear is not necessary since it's also done in the @After function


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by sansanichfb <gi...@git.apache.org>.
Github user sansanichfb commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51211063
  
    --- Diff: pxf/pxf-json/src/main/java/org/apache/hawq/pxf/plugins/json/JsonInputFormat.java ---
    @@ -0,0 +1,226 @@
    +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.BufferedInputStream;
    +import java.io.IOException;
    +import java.security.InvalidParameterException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.fs.FSDataInputStream;
    +import org.apache.hadoop.fs.FileSystem;
    +import org.apache.hadoop.fs.Path;
    +import org.apache.hadoop.io.LongWritable;
    +import org.apache.hadoop.io.NullWritable;
    +import org.apache.hadoop.io.Text;
    +import org.apache.hadoop.mapred.FileInputFormat;
    +import org.apache.hadoop.mapred.FileSplit;
    +import org.apache.hadoop.mapred.InputSplit;
    +import org.apache.hadoop.mapred.JobConf;
    +import org.apache.hadoop.mapred.LineRecordReader;
    +import org.apache.hadoop.mapred.RecordReader;
    +import org.apache.hadoop.mapred.Reporter;
    +import org.apache.log4j.Logger;
    +import org.codehaus.jackson.JsonFactory;
    +import org.codehaus.jackson.JsonNode;
    +import org.codehaus.jackson.JsonParseException;
    +import org.codehaus.jackson.map.JsonMappingException;
    +import org.codehaus.jackson.map.ObjectMapper;
    +
    +public class JsonInputFormat extends FileInputFormat<Text, NullWritable> {
    +
    +	private static JsonFactory factory = new JsonFactory();
    +	private static ObjectMapper mapper = new ObjectMapper(factory);
    +
    +	public static final String ONE_RECORD_PER_LINE = "json.input.format.one.record.per.line";
    +	public static final String RECORD_IDENTIFIER = "json.input.format.record.identifier";
    +
    +	@Override
    +	public RecordReader<Text, NullWritable> getRecordReader(InputSplit split, JobConf conf, Reporter reporter)
    +			throws IOException {
    +
    +		if (conf.getBoolean(ONE_RECORD_PER_LINE, false)) {
    +
    +			return new SimpleJsonRecordReader(conf, (FileSplit) split);
    +		} else {
    +			return new JsonRecordReader(conf, (FileSplit) split);
    +		}
    +	}
    +
    +	public static class SimpleJsonRecordReader implements RecordReader<Text, NullWritable> {
    +
    +		private LineRecordReader rdr = null;
    +		private LongWritable key = new LongWritable();
    +		private Text value = new Text();
    +
    +		public SimpleJsonRecordReader(Configuration conf, FileSplit split) throws IOException {
    +			rdr = new LineRecordReader(conf, split);
    +		}
    +
    +		@Override
    +		public void close() throws IOException {
    +			rdr.close();
    +		}
    +
    +		@Override
    +		public Text createKey() {
    +			return value;
    +		}
    +
    +		@Override
    +		public NullWritable createValue() {
    +			return NullWritable.get();
    +		}
    +
    +		@Override
    +		public long getPos() throws IOException {
    +			return rdr.getPos();
    +		}
    +
    +		@Override
    +		public boolean next(Text key, NullWritable value) throws IOException {
    +			if (rdr.next(this.key, this.value)) {
    +				key.set(this.value);
    +				return true;
    +			} else {
    +				return false;
    +			}
    +		}
    +
    +		@Override
    +		public float getProgress() throws IOException {
    +			return rdr.getProgress();
    +		}
    +	}
    +
    +	public static class JsonRecordReader implements RecordReader<Text, NullWritable> {
    +
    +		private Logger LOG = Logger.getLogger(JsonRecordReader.class);
    +
    +		private JsonStreamReader rdr = null;
    +		private long start = 0, end = 0;
    +		private float toRead = 0;
    +		private String identifier = null;
    +		private Logger log = Logger.getLogger(JsonRecordReader.class);
    +
    +		public JsonRecordReader(JobConf conf, FileSplit split) throws IOException {
    +			log.info("JsonRecordReader constructor called.  Conf is " + conf + ". Split is " + split);
    +			this.identifier = conf.get(RECORD_IDENTIFIER);
    +			log.info("Identifier is " + this.identifier);
    +
    +			if (this.identifier == null || identifier.isEmpty()) {
    +				throw new InvalidParameterException(JsonInputFormat.RECORD_IDENTIFIER + " is not set.");
    +			} else {
    +				LOG.info("Initializing JsonRecordReader with identifier " + identifier);
    +			}
    +
    +			// get relevant data
    +			Path file = split.getPath();
    +
    +			log.info("File is " + file);
    +
    +			start = split.getStart();
    +			end = start + split.getLength();
    +			toRead = end - start;
    +			log.info("FileSystem is " + FileSystem.get(conf));
    +
    +			FSDataInputStream strm = FileSystem.get(conf).open(file);
    +
    +			log.info("Retrieved file stream ");
    +
    +			if (start != 0) {
    +				strm.seek(start);
    +			}
    +
    +			rdr = new JsonStreamReader(identifier, new BufferedInputStream(strm));
    +
    +			log.info("Reader is " + rdr);
    +		}
    +
    +		@Override
    +		public boolean next(Text key, NullWritable value) throws IOException {
    +
    +			boolean retval = false;
    +			boolean keepGoing = false;
    +			do {
    +				// Exit condition (end of block/file)
    +				if (rdr.getBytesRead() >= (end - start)) {
    +					return false;
    +				}
    +
    +				keepGoing = false;
    +				String record = rdr.getJsonRecord();
    +				if (record != null) {
    +					if (JsonInputFormat.decodeLineToJsonNode(record) == null) {
    +						log.error("Unable to parse JSON string.  Skipping. DEBUG to see");
    +						log.debug(record);
    +						keepGoing = true;
    +					} else {
    +						key.set(record);
    +						retval = true;
    +					}
    +				}
    +			} while (keepGoing);
    +
    +			return retval;
    +		}
    +
    +		@Override
    +		public Text createKey() {
    +			return new Text();
    +		}
    +
    +		@Override
    +		public NullWritable createValue() {
    +			return NullWritable.get();
    +		}
    +
    +		@Override
    +		public long getPos() throws IOException {
    +			return start + rdr.getBytesRead();
    +		}
    +
    +		@Override
    +		public void close() throws IOException {
    +			rdr.close();
    +		}
    +
    +		@Override
    +		public float getProgress() throws IOException {
    +			return (float) rdr.getBytesRead() / toRead;
    +		}
    +	}
    +
    +	public static synchronized JsonNode decodeLineToJsonNode(String line) {
    +
    +		try {
    +			return mapper.readTree(line);
    +		} catch (JsonParseException e) {
    +			e.printStackTrace();
    --- End diff --
    
    Using log.debug instead of?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by hornn <gi...@git.apache.org>.
Github user hornn commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51211106
  
    --- Diff: pxf/pxf-json/src/main/java/org/apache/hawq/pxf/plugins/json/JsonInputFormat.java ---
    @@ -0,0 +1,226 @@
    +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.BufferedInputStream;
    +import java.io.IOException;
    +import java.security.InvalidParameterException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.fs.FSDataInputStream;
    +import org.apache.hadoop.fs.FileSystem;
    +import org.apache.hadoop.fs.Path;
    +import org.apache.hadoop.io.LongWritable;
    +import org.apache.hadoop.io.NullWritable;
    +import org.apache.hadoop.io.Text;
    +import org.apache.hadoop.mapred.FileInputFormat;
    +import org.apache.hadoop.mapred.FileSplit;
    +import org.apache.hadoop.mapred.InputSplit;
    +import org.apache.hadoop.mapred.JobConf;
    +import org.apache.hadoop.mapred.LineRecordReader;
    +import org.apache.hadoop.mapred.RecordReader;
    +import org.apache.hadoop.mapred.Reporter;
    +import org.apache.log4j.Logger;
    +import org.codehaus.jackson.JsonFactory;
    +import org.codehaus.jackson.JsonNode;
    +import org.codehaus.jackson.JsonParseException;
    +import org.codehaus.jackson.map.JsonMappingException;
    +import org.codehaus.jackson.map.ObjectMapper;
    +
    +public class JsonInputFormat extends FileInputFormat<Text, NullWritable> {
    +
    +	private static JsonFactory factory = new JsonFactory();
    +	private static ObjectMapper mapper = new ObjectMapper(factory);
    +
    +	public static final String ONE_RECORD_PER_LINE = "json.input.format.one.record.per.line";
    +	public static final String RECORD_IDENTIFIER = "json.input.format.record.identifier";
    +
    +	@Override
    +	public RecordReader<Text, NullWritable> getRecordReader(InputSplit split, JobConf conf, Reporter reporter)
    +			throws IOException {
    +
    +		if (conf.getBoolean(ONE_RECORD_PER_LINE, false)) {
    +
    +			return new SimpleJsonRecordReader(conf, (FileSplit) split);
    +		} else {
    +			return new JsonRecordReader(conf, (FileSplit) split);
    +		}
    +	}
    +
    +	public static class SimpleJsonRecordReader implements RecordReader<Text, NullWritable> {
    +
    +		private LineRecordReader rdr = null;
    +		private LongWritable key = new LongWritable();
    +		private Text value = new Text();
    +
    +		public SimpleJsonRecordReader(Configuration conf, FileSplit split) throws IOException {
    +			rdr = new LineRecordReader(conf, split);
    +		}
    +
    +		@Override
    +		public void close() throws IOException {
    +			rdr.close();
    +		}
    +
    +		@Override
    +		public Text createKey() {
    +			return value;
    +		}
    +
    +		@Override
    +		public NullWritable createValue() {
    +			return NullWritable.get();
    +		}
    +
    +		@Override
    +		public long getPos() throws IOException {
    +			return rdr.getPos();
    +		}
    +
    +		@Override
    +		public boolean next(Text key, NullWritable value) throws IOException {
    +			if (rdr.next(this.key, this.value)) {
    +				key.set(this.value);
    +				return true;
    +			} else {
    +				return false;
    +			}
    +		}
    +
    +		@Override
    +		public float getProgress() throws IOException {
    +			return rdr.getProgress();
    +		}
    +	}
    +
    +	public static class JsonRecordReader implements RecordReader<Text, NullWritable> {
    +
    +		private Logger LOG = Logger.getLogger(JsonRecordReader.class);
    +
    +		private JsonStreamReader rdr = null;
    +		private long start = 0, end = 0;
    +		private float toRead = 0;
    +		private String identifier = null;
    +		private Logger log = Logger.getLogger(JsonRecordReader.class);
    +
    +		public JsonRecordReader(JobConf conf, FileSplit split) throws IOException {
    +			log.info("JsonRecordReader constructor called.  Conf is " + conf + ". Split is " + split);
    +			this.identifier = conf.get(RECORD_IDENTIFIER);
    +			log.info("Identifier is " + this.identifier);
    +
    +			if (this.identifier == null || identifier.isEmpty()) {
    +				throw new InvalidParameterException(JsonInputFormat.RECORD_IDENTIFIER + " is not set.");
    +			} else {
    +				LOG.info("Initializing JsonRecordReader with identifier " + identifier);
    +			}
    +
    +			// get relevant data
    +			Path file = split.getPath();
    +
    +			log.info("File is " + file);
    +
    +			start = split.getStart();
    +			end = start + split.getLength();
    +			toRead = end - start;
    +			log.info("FileSystem is " + FileSystem.get(conf));
    +
    +			FSDataInputStream strm = FileSystem.get(conf).open(file);
    +
    +			log.info("Retrieved file stream ");
    +
    +			if (start != 0) {
    +				strm.seek(start);
    +			}
    +
    +			rdr = new JsonStreamReader(identifier, new BufferedInputStream(strm));
    +
    +			log.info("Reader is " + rdr);
    +		}
    +
    +		@Override
    +		public boolean next(Text key, NullWritable value) throws IOException {
    +
    +			boolean retval = false;
    +			boolean keepGoing = false;
    +			do {
    +				// Exit condition (end of block/file)
    +				if (rdr.getBytesRead() >= (end - start)) {
    +					return false;
    +				}
    +
    +				keepGoing = false;
    +				String record = rdr.getJsonRecord();
    +				if (record != null) {
    +					if (JsonInputFormat.decodeLineToJsonNode(record) == null) {
    +						log.error("Unable to parse JSON string.  Skipping. DEBUG to see");
    +						log.debug(record);
    +						keepGoing = true;
    +					} else {
    +						key.set(record);
    +						retval = true;
    +					}
    +				}
    +			} while (keepGoing);
    +
    +			return retval;
    +		}
    +
    +		@Override
    +		public Text createKey() {
    +			return new Text();
    +		}
    +
    +		@Override
    +		public NullWritable createValue() {
    +			return NullWritable.get();
    +		}
    +
    +		@Override
    +		public long getPos() throws IOException {
    +			return start + rdr.getBytesRead();
    +		}
    +
    +		@Override
    +		public void close() throws IOException {
    +			rdr.close();
    +		}
    +
    +		@Override
    +		public float getProgress() throws IOException {
    +			return (float) rdr.getBytesRead() / toRead;
    +		}
    +	}
    +
    +	public static synchronized JsonNode decodeLineToJsonNode(String line) {
    +
    +		try {
    +			return mapper.readTree(line);
    +		} catch (JsonParseException e) {
    +			e.printStackTrace();
    --- End diff --
    
    perhaps replace e.printStackTrace() with LOG.warning()?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by tzolov <gi...@git.apache.org>.
Github user tzolov commented on the pull request:

    https://github.com/apache/incubator-hawq/pull/302#issuecomment-178252170
  
    Updated the PR. Commits should address most of the comments above. I've simplified the code. Replaced the JsonLineRecordReader in favor of the OOB LineRecordReader, removed the onerecordperline property (using the identifier property as a flag instead) and many more. 
    
    Brief [how to use doc](https://issues.apache.org/jira/secure/attachment/12785629/PXFJSONPluginforHAWQ2.0andPXF3.0.0v.2.pdf) is attached to the JIRA ticket. 
    
    Still have not resolved the use case described by @adamjshook above.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by hornn <gi...@git.apache.org>.
Github user hornn commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51511168
  
    --- Diff: pxf/pxf-json/src/main/java/org/apache/hawq/pxf/plugins/json/JsonRecordReader.java ---
    @@ -0,0 +1,166 @@
    +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 static org.apache.commons.lang3.StringUtils.isEmpty;
    +
    +import java.io.BufferedInputStream;
    +import java.io.IOException;
    +import java.security.InvalidParameterException;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.hadoop.fs.FSDataInputStream;
    +import org.apache.hadoop.fs.FileSystem;
    +import org.apache.hadoop.fs.Path;
    +import org.apache.hadoop.io.LongWritable;
    +import org.apache.hadoop.io.Text;
    +import org.apache.hadoop.mapred.FileSplit;
    +import org.apache.hadoop.mapred.JobConf;
    +import org.apache.hadoop.mapred.RecordReader;
    +import org.apache.hawq.pxf.api.Fragment;
    +
    +/**
    + * {@link RecordReader} implementation that can read multiline JSON objects.
    + */
    +public class JsonRecordReader implements RecordReader<LongWritable, Text> {
    +
    +	private static final Log LOG = LogFactory.getLog(JsonRecordReader.class);
    +
    +	public static final String RECORD_IDENTIFIER = "json.input.format.record.identifier";
    +
    +	private JsonStreamReader streamReader = null;
    +	private long start = 0, end = 0;
    +	private float toRead = 0;
    +	private String identifier = null;
    +
    +	/**
    +	 * @param conf
    +	 *            Hadoop context
    +	 * @param split
    +	 *            HDFS Split as defined by the {@link Fragment}.
    +	 * @throws IOException
    +	 */
    +	public JsonRecordReader(JobConf conf, FileSplit split) throws IOException {
    +		LOG.debug("JsonRecordReader constructor called.  Conf is " + conf + ". Split is " + split);
    +		this.identifier = conf.get(RECORD_IDENTIFIER);
    +		LOG.debug("Identifier is " + this.identifier);
    +
    +		if (isEmpty(this.identifier)) {
    +			throw new InvalidParameterException(RECORD_IDENTIFIER + " is not set.");
    +		} else {
    +			LOG.debug("Initializing JsonRecordReader with identifier " + identifier);
    +		}
    +
    +		// get relevant data
    +		Path file = split.getPath();
    +
    +		LOG.debug("File is " + file);
    +
    +		start = split.getStart();
    +		end = start + split.getLength();
    +		toRead = end - start;
    +		LOG.debug("FileSystem is " + FileSystem.get(conf));
    +
    +		FSDataInputStream strm = FileSystem.get(conf).open(file);
    +
    +		LOG.debug("Retrieved file stream ");
    +
    +		if (start != 0) {
    +			strm.seek(start);
    +		}
    +
    +		streamReader = new JsonStreamReader(identifier, new BufferedInputStream(strm));
    +
    +		LOG.debug("Reader is " + streamReader);
    +	}
    +
    +	/*
    +	 * {@inheritDoc}
    +	 */
    +	@Override
    +	public boolean next(LongWritable key, Text value) throws IOException {
    +
    +		boolean retval = false;
    +		boolean keepGoing = false;
    +		do {
    +			// Exit condition (end of block/file)
    +			if (streamReader.getBytesRead() >= (end - start)) {
    +				return false;
    +			}
    +
    +			keepGoing = false;
    +			String record = streamReader.getJsonRecord();
    +			if (record != null) {
    +				if (JsonUtil.decodeLineToJsonNode(record) == null) {
    --- End diff --
    
    We call decodeLineToJsonNode twice for each record - one time here (accessing), and one time in the resolver. I don't know if it is a heavy operation or not, but it is slow because this function is synchronized, so multiple requests will be queued.
    Consider removing this logic here - and failing in the resolver. We also have a notion of BadRecordException that can be used when there is a malformatted record. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by tzolov <gi...@git.apache.org>.
Github user tzolov commented on the pull request:

    https://github.com/apache/incubator-hawq/pull/302#issuecomment-178415466
  
    Should be done. the jsonFactory is not required as private field anymore. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by hornn <gi...@git.apache.org>.
Github user hornn commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51314932
  
    --- 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,
    +				projs[projs.length - 1].length() - 1));
    +	}
    +
    +	/**
    +	 * Iterates through the given JSON node to the proper index and adds the field of corresponding type
    +	 * 
    +	 * @param type
    +	 *            The {@link DataType} type
    +	 * @param node
    +	 *            The JSON array node
    +	 * @param index
    +	 *            The array index to iterate to
    +	 * @throws IOException
    +	 */
    +	private void addFieldFromJsonArray(DataType type, JsonNode node, int index) throws IOException {
    +
    +		int count = 0;
    +		boolean added = false;
    +		for (Iterator<JsonNode> arrayNodes = node.getElements(); arrayNodes.hasNext();) {
    +			JsonNode arrayNode = arrayNodes.next();
    +
    +			if (count == index) {
    +				added = true;
    +				addFieldFromJsonNode(type, arrayNode);
    +				break;
    +			}
    +
    +			++count;
    +		}
    +
    +		// if we reached the end of the array without adding a
    +		// field, add null
    +		if (!added) {
    +			addNullField(type);
    +		}
    +	}
    +
    +	/**
    +	 * Adds a field from a given JSON node value based on the {@link DataType} type.
    +	 * 
    +	 * @param type
    +	 *            The DataType type
    +	 * @param val
    +	 *            The JSON node to extract the value.
    +	 * @throws IOException
    +	 */
    +	private void addFieldFromJsonNode(DataType type, JsonNode val) throws IOException {
    +		OneField oneField = new OneField();
    +		oneField.type = type.getOID();
    +
    +		if (val.isNull()) {
    +			oneField.val = null;
    +		} else {
    +			switch (type) {
    +			case BIGINT:
    +				oneField.val = val.asLong();
    +				break;
    +			case BOOLEAN:
    +				oneField.val = val.asBoolean();
    +				break;
    +			case BPCHAR:
    --- End diff --
    
    BPCHAR should be treated like VARCHAR and TEXT - it's more than one character.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by tzolov <gi...@git.apache.org>.
Github user tzolov commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51346138
  
    --- Diff: pxf/pxf-json/src/main/java/org/apache/hawq/pxf/plugins/json/JsonInputFormat.java ---
    @@ -0,0 +1,226 @@
    +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.BufferedInputStream;
    +import java.io.IOException;
    +import java.security.InvalidParameterException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.fs.FSDataInputStream;
    +import org.apache.hadoop.fs.FileSystem;
    +import org.apache.hadoop.fs.Path;
    +import org.apache.hadoop.io.LongWritable;
    +import org.apache.hadoop.io.NullWritable;
    +import org.apache.hadoop.io.Text;
    +import org.apache.hadoop.mapred.FileInputFormat;
    +import org.apache.hadoop.mapred.FileSplit;
    +import org.apache.hadoop.mapred.InputSplit;
    +import org.apache.hadoop.mapred.JobConf;
    +import org.apache.hadoop.mapred.LineRecordReader;
    +import org.apache.hadoop.mapred.RecordReader;
    +import org.apache.hadoop.mapred.Reporter;
    +import org.apache.log4j.Logger;
    +import org.codehaus.jackson.JsonFactory;
    +import org.codehaus.jackson.JsonNode;
    +import org.codehaus.jackson.JsonParseException;
    +import org.codehaus.jackson.map.JsonMappingException;
    +import org.codehaus.jackson.map.ObjectMapper;
    +
    +public class JsonInputFormat extends FileInputFormat<Text, NullWritable> {
    +
    +	private static JsonFactory factory = new JsonFactory();
    +	private static ObjectMapper mapper = new ObjectMapper(factory);
    +
    +	public static final String ONE_RECORD_PER_LINE = "json.input.format.one.record.per.line";
    +	public static final String RECORD_IDENTIFIER = "json.input.format.record.identifier";
    +
    +	@Override
    +	public RecordReader<Text, NullWritable> getRecordReader(InputSplit split, JobConf conf, Reporter reporter)
    +			throws IOException {
    +
    +		if (conf.getBoolean(ONE_RECORD_PER_LINE, false)) {
    +
    +			return new SimpleJsonRecordReader(conf, (FileSplit) split);
    +		} else {
    +			return new JsonRecordReader(conf, (FileSplit) split);
    +		}
    +	}
    +
    +	public static class SimpleJsonRecordReader implements RecordReader<Text, NullWritable> {
    +
    +		private LineRecordReader rdr = null;
    +		private LongWritable key = new LongWritable();
    +		private Text value = new Text();
    +
    +		public SimpleJsonRecordReader(Configuration conf, FileSplit split) throws IOException {
    +			rdr = new LineRecordReader(conf, split);
    +		}
    +
    +		@Override
    +		public void close() throws IOException {
    +			rdr.close();
    +		}
    +
    +		@Override
    +		public Text createKey() {
    +			return value;
    +		}
    +
    +		@Override
    +		public NullWritable createValue() {
    +			return NullWritable.get();
    +		}
    +
    +		@Override
    +		public long getPos() throws IOException {
    +			return rdr.getPos();
    +		}
    +
    +		@Override
    +		public boolean next(Text key, NullWritable value) throws IOException {
    --- End diff --
    
    As those methods implement `RecordReader` i guess inherit the the doc (`{@inheritDoc}`) should be enough as documentation? Only `next()` seems to deserve additional doc.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by tzolov <gi...@git.apache.org>.
Github user tzolov commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51346971
  
    --- Diff: pxf/pxf-json/src/main/java/org/apache/hawq/pxf/plugins/json/JsonInputFormat.java ---
    @@ -0,0 +1,226 @@
    +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.BufferedInputStream;
    +import java.io.IOException;
    +import java.security.InvalidParameterException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.fs.FSDataInputStream;
    +import org.apache.hadoop.fs.FileSystem;
    +import org.apache.hadoop.fs.Path;
    +import org.apache.hadoop.io.LongWritable;
    +import org.apache.hadoop.io.NullWritable;
    +import org.apache.hadoop.io.Text;
    +import org.apache.hadoop.mapred.FileInputFormat;
    +import org.apache.hadoop.mapred.FileSplit;
    +import org.apache.hadoop.mapred.InputSplit;
    +import org.apache.hadoop.mapred.JobConf;
    +import org.apache.hadoop.mapred.LineRecordReader;
    +import org.apache.hadoop.mapred.RecordReader;
    +import org.apache.hadoop.mapred.Reporter;
    +import org.apache.log4j.Logger;
    +import org.codehaus.jackson.JsonFactory;
    +import org.codehaus.jackson.JsonNode;
    +import org.codehaus.jackson.JsonParseException;
    +import org.codehaus.jackson.map.JsonMappingException;
    +import org.codehaus.jackson.map.ObjectMapper;
    +
    +public class JsonInputFormat extends FileInputFormat<Text, NullWritable> {
    +
    +	private static JsonFactory factory = new JsonFactory();
    +	private static ObjectMapper mapper = new ObjectMapper(factory);
    +
    +	public static final String ONE_RECORD_PER_LINE = "json.input.format.one.record.per.line";
    +	public static final String RECORD_IDENTIFIER = "json.input.format.record.identifier";
    +
    +	@Override
    +	public RecordReader<Text, NullWritable> getRecordReader(InputSplit split, JobConf conf, Reporter reporter)
    +			throws IOException {
    +
    +		if (conf.getBoolean(ONE_RECORD_PER_LINE, false)) {
    +
    +			return new SimpleJsonRecordReader(conf, (FileSplit) split);
    +		} else {
    +			return new JsonRecordReader(conf, (FileSplit) split);
    +		}
    +	}
    +
    +	public static class SimpleJsonRecordReader implements RecordReader<Text, NullWritable> {
    +
    +		private LineRecordReader rdr = null;
    +		private LongWritable key = new LongWritable();
    +		private Text value = new Text();
    +
    +		public SimpleJsonRecordReader(Configuration conf, FileSplit split) throws IOException {
    +			rdr = new LineRecordReader(conf, split);
    +		}
    +
    +		@Override
    +		public void close() throws IOException {
    +			rdr.close();
    +		}
    +
    +		@Override
    +		public Text createKey() {
    +			return value;
    +		}
    +
    +		@Override
    +		public NullWritable createValue() {
    +			return NullWritable.get();
    +		}
    +
    +		@Override
    +		public long getPos() throws IOException {
    +			return rdr.getPos();
    +		}
    +
    +		@Override
    +		public boolean next(Text key, NullWritable value) throws IOException {
    +			if (rdr.next(this.key, this.value)) {
    +				key.set(this.value);
    +				return true;
    +			} else {
    +				return false;
    +			}
    +		}
    +
    +		@Override
    +		public float getProgress() throws IOException {
    +			return rdr.getProgress();
    +		}
    +	}
    +
    +	public static class JsonRecordReader implements RecordReader<Text, NullWritable> {
    +
    +		private Logger LOG = Logger.getLogger(JsonRecordReader.class);
    +
    +		private JsonStreamReader rdr = null;
    +		private long start = 0, end = 0;
    +		private float toRead = 0;
    +		private String identifier = null;
    +		private Logger log = Logger.getLogger(JsonRecordReader.class);
    +
    +		public JsonRecordReader(JobConf conf, FileSplit split) throws IOException {
    +			log.info("JsonRecordReader constructor called.  Conf is " + conf + ". Split is " + split);
    +			this.identifier = conf.get(RECORD_IDENTIFIER);
    +			log.info("Identifier is " + this.identifier);
    +
    +			if (this.identifier == null || identifier.isEmpty()) {
    +				throw new InvalidParameterException(JsonInputFormat.RECORD_IDENTIFIER + " is not set.");
    +			} else {
    +				LOG.info("Initializing JsonRecordReader with identifier " + identifier);
    +			}
    +
    +			// get relevant data
    +			Path file = split.getPath();
    +
    +			log.info("File is " + file);
    +
    +			start = split.getStart();
    +			end = start + split.getLength();
    +			toRead = end - start;
    +			log.info("FileSystem is " + FileSystem.get(conf));
    +
    +			FSDataInputStream strm = FileSystem.get(conf).open(file);
    +
    +			log.info("Retrieved file stream ");
    +
    +			if (start != 0) {
    +				strm.seek(start);
    +			}
    +
    +			rdr = new JsonStreamReader(identifier, new BufferedInputStream(strm));
    +
    +			log.info("Reader is " + rdr);
    +		}
    +
    +		@Override
    +		public boolean next(Text key, NullWritable value) throws IOException {
    +
    +			boolean retval = false;
    +			boolean keepGoing = false;
    +			do {
    +				// Exit condition (end of block/file)
    +				if (rdr.getBytesRead() >= (end - start)) {
    +					return false;
    +				}
    +
    +				keepGoing = false;
    +				String record = rdr.getJsonRecord();
    +				if (record != null) {
    +					if (JsonInputFormat.decodeLineToJsonNode(record) == null) {
    +						log.error("Unable to parse JSON string.  Skipping. DEBUG to see");
    +						log.debug(record);
    +						keepGoing = true;
    +					} else {
    +						key.set(record);
    +						retval = true;
    +					}
    +				}
    +			} while (keepGoing);
    +
    +			return retval;
    +		}
    +
    +		@Override
    +		public Text createKey() {
    +			return new Text();
    +		}
    +
    +		@Override
    +		public NullWritable createValue() {
    +			return NullWritable.get();
    +		}
    +
    +		@Override
    +		public long getPos() throws IOException {
    +			return start + rdr.getBytesRead();
    +		}
    +
    +		@Override
    +		public void close() throws IOException {
    +			rdr.close();
    +		}
    +
    +		@Override
    +		public float getProgress() throws IOException {
    +			return (float) rdr.getBytesRead() / toRead;
    +		}
    +	}
    +
    +	public static synchronized JsonNode decodeLineToJsonNode(String line) {
    +
    +		try {
    +			return mapper.readTree(line);
    +		} catch (JsonParseException e) {
    +			e.printStackTrace();
    --- End diff --
    
    addressed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by tzolov <gi...@git.apache.org>.
Github user tzolov commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51870749
  
    --- Diff: pxf/pxf-json/src/main/java/org/apache/hawq/pxf/plugins/json/JsonResolver.java ---
    @@ -0,0 +1,255 @@
    +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.util.ArrayList;
    +import java.util.Iterator;
    +import java.util.List;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +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.JsonFactory;
    +import org.codehaus.jackson.JsonNode;
    +import org.codehaus.jackson.map.ObjectMapper;
    +
    +/**
    + * 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 static final Log LOG = LogFactory.getLog(JsonResolver.class);
    +
    +	private ArrayList<OneField> oneFieldList;
    +	private ColumnDescriptorCache[] columnDescriptorCache;
    +	private ObjectMapper mapper;
    +
    +	public JsonResolver(InputData inputData) throws Exception {
    +		super(inputData);
    +		oneFieldList = new ArrayList<OneField>();
    +		mapper = new ObjectMapper(new JsonFactory());
    +
    +		// Pre-generate all column structure attributes concerning the JSON to column value resolution.
    +		columnDescriptorCache = new ColumnDescriptorCache[inputData.getColumns()];
    +		for (int i = 0; i < inputData.getColumns(); ++i) {
    +			ColumnDescriptor cd = inputData.getColumn(i);
    +			columnDescriptorCache[i] = new ColumnDescriptorCache(cd);
    +		}
    +	}
    +
    +	@Override
    +	public List<OneField> getFields(OneRow row) throws Exception {
    +		oneFieldList.clear();
    +
    +		String jsonRecordAsText = row.getData().toString();
    +
    +		JsonNode root = decodeLineToJsonNode(jsonRecordAsText);
    +
    +		if (root == null) {
    +			LOG.warn("Return null-fields row due to invalid JSON:" + jsonRecordAsText);
    +		}
    +
    +		// Iterate through the column definition and fetch our JSON data
    +		for (ColumnDescriptorCache column : columnDescriptorCache) {
    +
    +			// Get the current column description
    +
    +			if (root == null) {
    +				// Return empty (e.g. null) filed in case of malformed json.
    +				addNullField(column.getColumnType());
    +			} else {
    +
    +				// Move down the JSON path to the final name
    +				JsonNode node = getPriorJsonNode(root, column.getProjections());
    +
    +				// If this column is an array index, ex. "tweet.hashtags[0]"
    +				if (column.isArrayName()) {
    +
    +					// Move to the array node
    +					node = node.get(column.getArrayNodeName());
    +
    +					// If this node is null or missing, add a null value here
    +					if (node == null || node.isMissingNode()) {
    +						addNullField(column.getColumnType());
    +					} else if (node.isArray()) {
    +						// If the JSON node is an array, then add it to our list
    +						addFieldFromJsonArray(column.getColumnType(), node, column.getArrayIndex());
    +					} else {
    +						throw new IllegalStateException(column.getArrayNodeName() + " is not an array node");
    +					}
    +				} else {
    +					// This column is not an array type
    +					// Move to the final node
    +					node = node.get(column.getLastProjection());
    +
    +					// If this node is null or missing, add a null value here
    +					if (node == null || node.isMissingNode()) {
    +						addNullField(column.getColumnType());
    +					} else {
    +						// Else, add the value to the record
    +						addFieldFromJsonNode(column.getColumnType(), node);
    +					}
    +				}
    +			}
    +		}
    +
    +		return oneFieldList;
    +	}
    +
    +	/**
    +	 * Iterates down the root node to the child JSON node defined by the projs path.
    +	 * 
    +	 * @param root
    +	 *            node to to start the traversal from.
    +	 * @param projs
    +	 *            defines the path from the root to the desired child node.
    +	 * @return Returns the child node defined by the root and projs path.
    +	 */
    +	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) {
    --- End diff --
    
    Update: I've refactored the ColumDescriptorCache to compute a *normalized* projections. In case of array it substitutes the last path element with the name of the array. In this way the JsonResolver#getFields() is simplified. (take a look)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by GodenYao <gi...@git.apache.org>.
Github user GodenYao commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r52077060
  
    --- Diff: pxf/pxf-service/src/main/resources/pxf-profiles-default.xml ---
    @@ -128,4 +128,13 @@ under the License.
                 <resolver>org.apache.hawq.pxf.plugins.gemfirexd.GemFireXDResolver</resolver>
             </plugins>
         </profile>
    +    <profile>
    +        <name>Json</name>
    +        <description>A profile for JSON data, one JSON record per line</description>
    --- End diff --
    
    looks like we've consolidate 2 profile - should update the description to not limit it to "one JSON record per line".


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by hornn <gi...@git.apache.org>.
Github user hornn commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51835736
  
    --- Diff: pxf/pxf-json/src/test/java/org/apache/hawq/pxf/plugins/json/PxfUnit.java ---
    @@ -0,0 +1,672 @@
    +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.BufferedReader;
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.InputStreamReader;
    +import java.io.OutputStream;
    +import java.lang.reflect.Constructor;
    +import java.security.InvalidParameterException;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.fs.FileSystem;
    +import org.apache.hadoop.fs.Path;
    +import org.apache.hawq.pxf.api.Fragment;
    +import org.apache.hawq.pxf.api.Fragmenter;
    +import org.apache.hawq.pxf.api.OneField;
    +import org.apache.hawq.pxf.api.OneRow;
    +import org.apache.hawq.pxf.api.ReadAccessor;
    +import org.apache.hawq.pxf.api.ReadResolver;
    +import org.apache.hawq.pxf.api.WriteAccessor;
    +import org.apache.hawq.pxf.api.WriteResolver;
    +import org.apache.hawq.pxf.api.io.DataType;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +import org.apache.hawq.pxf.service.FragmentsResponse;
    +import org.apache.hawq.pxf.service.FragmentsResponseFormatter;
    +import org.apache.hawq.pxf.service.utilities.ProtocolData;
    +import org.codehaus.jackson.JsonFactory;
    +import org.codehaus.jackson.JsonNode;
    +import org.codehaus.jackson.JsonParseException;
    +import org.codehaus.jackson.map.JsonMappingException;
    +import org.codehaus.jackson.map.ObjectMapper;
    +import org.junit.Assert;
    +
    +/**
    + * This abstract class contains a number of helpful utilities in developing a PXF extension for HAWQ. Extend this class
    + * and use the various <code>assert</code> methods to check given input against known output.
    + */
    +public abstract class PxfUnit {
    +
    +	private static final Log LOG = LogFactory.getLog(PxfUnit.class);
    +
    +	private static JsonFactory factory = new JsonFactory();
    +	private static ObjectMapper mapper = new ObjectMapper(factory);
    +
    +	protected static List<InputData> inputs = null;
    +
    +	/**
    +	 * Uses the given input directory to run through the PXF unit testing framework. Uses the lines in the file for
    +	 * output testing.
    +	 * 
    +	 * @param input
    +	 *            Input records
    +	 * @param expectedOutput
    +	 *            File containing output to check
    +	 * @throws Exception
    +	 */
    +	public void assertOutput(Path input, Path expectedOutput) throws Exception {
    +
    +		BufferedReader rdr = new BufferedReader(new InputStreamReader(FileSystem.get(new Configuration()).open(
    +				expectedOutput)));
    +
    +		List<String> outputLines = new ArrayList<String>();
    +
    +		String line;
    +		while ((line = rdr.readLine()) != null) {
    +			outputLines.add(line);
    +		}
    +
    +		assertOutput(input, outputLines);
    +
    +		rdr.close();
    +	}
    +
    +	/**
    +	 * Uses the given input directory to run through the PXF unit testing framework. Uses the lines in the given
    +	 * parameter for output testing.
    +	 * 
    +	 * @param input
    +	 *            Input records
    +	 * @param expectedOutput
    +	 *            File containing output to check
    +	 * @throws Exception
    +	 */
    +	public void assertOutput(Path input, List<String> expectedOutput) throws Exception {
    +
    +		setup(input);
    +		List<String> actualOutput = new ArrayList<String>();
    +		for (InputData data : inputs) {
    +			ReadAccessor accessor = getReadAccessor(data);
    +			ReadResolver resolver = getReadResolver(data);
    +
    +			actualOutput.addAll(getAllOutput(accessor, resolver));
    +		}
    +
    +		Assert.assertFalse("Output did not match expected output", compareOutput(expectedOutput, actualOutput));
    +	}
    +
    +	/**
    +	 * Uses the given input directory to run through the PXF unit testing framework. Uses the lines in the given
    +	 * parameter for output testing.<br>
    +	 * <br>
    +	 * Ignores order of records.
    +	 * 
    +	 * @param input
    +	 *            Input records
    +	 * @param expectedOutput
    +	 *            File containing output to check
    +	 * @throws Exception
    +	 */
    +	public void assertUnorderedOutput(Path input, Path expectedOutput) throws Exception {
    +		BufferedReader rdr = new BufferedReader(new InputStreamReader(FileSystem.get(new Configuration()).open(
    +				expectedOutput)));
    +
    +		List<String> outputLines = new ArrayList<String>();
    +
    +		String line;
    +		while ((line = rdr.readLine()) != null) {
    +			outputLines.add(line);
    +		}
    +
    +		assertUnorderedOutput(input, outputLines);
    +	}
    --- End diff --
    
    rdr.close()


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by hornn <gi...@git.apache.org>.
Github user hornn commented on the pull request:

    https://github.com/apache/incubator-hawq/pull/302#issuecomment-179682943
  
    @tzolov regarding test coverage, what you did is very impressive.
    Normally we have an automation suite where we test end to end query using a specific plugin, but I really like the idea of PxfUnit. 
    I think after your commit is done we'll also have to add automation tests, but they will probably not be as exhaustive as what you already did.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by tzolov <gi...@git.apache.org>.
Github user tzolov commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51345914
  
    --- Diff: pxf/build.gradle ---
    @@ -286,6 +286,29 @@ project('pxf-hive') {
         }
     }
     
    +project('pxf-json') {
    +	dependencies {
    +		compile(project(':pxf-hdfs'))
    +		compile "org.apache.commons:commons-lang3:3.0"
    +
    +		testCompile 'pl.pragmatists:JUnitParams:1.0.2'
    +		configurations {
    +			// Remove hive-exec from unit tests as it causes VerifyError
    +			testRuntime.exclude module: 'hive-exec'
    --- End diff --
    
    @hornn What do you men? do we need the pxf-json project? or do we need some of the dependencies/configurations? 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by tzolov <gi...@git.apache.org>.
Github user tzolov commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51863107
  
    --- Diff: pxf/pxf-json/src/test/java/org/apache/hawq/pxf/plugins/json/JsonExtensionTest.java ---
    @@ -0,0 +1,288 @@
    +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.File;
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import org.apache.hadoop.fs.Path;
    +import org.apache.hawq.pxf.api.Fragmenter;
    +import org.apache.hawq.pxf.api.ReadAccessor;
    +import org.apache.hawq.pxf.api.ReadResolver;
    +import org.apache.hawq.pxf.api.io.DataType;
    +import org.apache.hawq.pxf.plugins.hdfs.HdfsDataFragmenter;
    +import org.apache.hawq.pxf.plugins.json.JsonAccessor;
    +import org.apache.hawq.pxf.plugins.json.JsonResolver;
    +import org.junit.After;
    +import org.junit.Before;
    +import org.junit.Test;
    +
    +public class JsonExtensionTest extends PxfUnit {
    +
    +	private List<Pair<String, DataType>> columnDefs = null;
    +	private List<Pair<String, String>> extraParams = new ArrayList<Pair<String, String>>();
    +
    +	@Before
    +	public void before() {
    +
    +		columnDefs = new ArrayList<Pair<String, DataType>>();
    +
    +		columnDefs.add(new Pair<String, DataType>("created_at", DataType.TEXT));
    +		columnDefs.add(new Pair<String, DataType>("id", DataType.BIGINT));
    +		columnDefs.add(new Pair<String, DataType>("text", DataType.TEXT));
    +		columnDefs.add(new Pair<String, DataType>("user.screen_name", DataType.TEXT));
    +		columnDefs.add(new Pair<String, DataType>("entities.hashtags[0]", DataType.TEXT));
    +		columnDefs.add(new Pair<String, DataType>("coordinates.coordinates[0]", DataType.FLOAT8));
    +		columnDefs.add(new Pair<String, DataType>("coordinates.coordinates[1]", DataType.FLOAT8));
    +	}
    +
    +	@After
    +	public void cleanup() throws Exception {
    +		columnDefs.clear();
    +		extraParams.clear();
    +	}
    +
    +	@Test
    +	public void testCompressedMultilineJsonFile() throws Exception {
    +
    +		extraParams.add(new Pair<String, String>("IDENTIFIER", "created_at"));
    +
    +		List<String> output = new ArrayList<String>();
    +
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547115253761,text1,SpreadButter,tweetCongress,,");
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547123646465,text2,patronusdeadly,,,");
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547136233472,text3,NoSecrets_Vagas,,,");
    +
    +		super.assertOutput(new Path(System.getProperty("user.dir") + File.separator
    +				+ "src/test/resources/tweets.tar.gz"), output);
    +	}
    +
    +	@Test
    +	public void testMaxRecordLength() throws Exception {
    +
    +		// variable-size-objects.json contains 3 json objects but only 2 of them fit in the 27 byte length limitation
    +
    +		extraParams.add(new Pair<String, String>("IDENTIFIER", "key666"));
    +		extraParams.add(new Pair<String, String>("MAXLENGTH", "27"));
    +
    +		columnDefs.clear();
    +		columnDefs.add(new Pair<String, DataType>("key666", DataType.TEXT));
    +
    +		List<String> output = new ArrayList<String>();
    +
    +		output.add("small object1");
    +		// skip the large object2 XXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX
    +		output.add("small object3");
    +
    +		super.assertOutput(new Path(System.getProperty("user.dir") + File.separator
    +				+ "src/test/resources/variable-size-objects.json"), output);
    +	}
    +
    +	@Test
    +	public void testDataTypes() throws Exception {
    +
    +		// TDOO: The BYTEA type is not tested!!! Current oneField.val = val.asText().getBytes(); convention is
    --- End diff --
    
    Sure. let's live it as it is for now. Just as a known issue and address it in a separate (JIRA?) task? 
    I don't know what should be the desired value but array ref. doesn't seems right to me. Perhaps @adamjshook  can help.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by hornn <gi...@git.apache.org>.
Github user hornn commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51828041
  
    --- Diff: pxf/pxf-json/src/main/java/org/apache/hawq/pxf/plugins/json/JsonResolver.java ---
    @@ -0,0 +1,255 @@
    +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.util.ArrayList;
    +import java.util.Iterator;
    +import java.util.List;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +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.JsonFactory;
    +import org.codehaus.jackson.JsonNode;
    +import org.codehaus.jackson.map.ObjectMapper;
    +
    +/**
    + * 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 static final Log LOG = LogFactory.getLog(JsonResolver.class);
    +
    +	private ArrayList<OneField> oneFieldList;
    +	private ColumnDescriptorCache[] columnDescriptorCache;
    +	private ObjectMapper mapper;
    +
    +	public JsonResolver(InputData inputData) throws Exception {
    +		super(inputData);
    +		oneFieldList = new ArrayList<OneField>();
    +		mapper = new ObjectMapper(new JsonFactory());
    +
    +		// Pre-generate all column structure attributes concerning the JSON to column value resolution.
    +		columnDescriptorCache = new ColumnDescriptorCache[inputData.getColumns()];
    +		for (int i = 0; i < inputData.getColumns(); ++i) {
    +			ColumnDescriptor cd = inputData.getColumn(i);
    +			columnDescriptorCache[i] = new ColumnDescriptorCache(cd);
    +		}
    +	}
    +
    +	@Override
    +	public List<OneField> getFields(OneRow row) throws Exception {
    +		oneFieldList.clear();
    +
    +		String jsonRecordAsText = row.getData().toString();
    +
    +		JsonNode root = decodeLineToJsonNode(jsonRecordAsText);
    +
    +		if (root == null) {
    +			LOG.warn("Return null-fields row due to invalid JSON:" + jsonRecordAsText);
    --- End diff --
    
    another possible improvement: create a final list of null fields for the case where the root is null, and immediately return that list here.
    That will save us the iteration over the columns and checking for each one if the root is null.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by tzolov <gi...@git.apache.org>.
Github user tzolov commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51494032
  
    --- Diff: pxf/build.gradle ---
    @@ -64,7 +64,7 @@ subprojects { subProject ->
                 "-Xlint:finally", "-Xlint:overrides", "-Xlint:path", "-Xlint:processing", "-Xlint:static", "-Xlint:try",
                 "-Xlint:-fallthrough", "-Xlint:-deprecation", "-Xlint:-unchecked", "-Xlint:-options"
         ]
    -
    +		
    --- End diff --
    
    done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by hornn <gi...@git.apache.org>.
Github user hornn commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51834024
  
    --- Diff: pxf/pxf-json/src/main/java/org/apache/hawq/pxf/plugins/json/parser/PartitionedJsonParser.java ---
    @@ -0,0 +1,198 @@
    +package org.apache.hawq.pxf.plugins.json.parser;
    +
    +/*
    + * 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.io.InputStream;
    +import java.io.InputStreamReader;
    +import java.nio.charset.StandardCharsets;
    +import java.util.ArrayList;
    +import java.util.EnumSet;
    +import java.util.List;
    +
    +import org.apache.hawq.pxf.plugins.json.parser.JsonLexer.JsonLexerState;
    +
    +/**
    + * A simple parser that can support reading JSON objects from a random point in JSON text. It reads from the supplied
    + * stream (which is assumed to be positioned at any arbitrary position inside some JSON text) until it find the first
    + * JSON begin-object "{". From this point on it will keep reading JSON objects until it finds one containing a member
    + * string that the user supplies.
    + * <p/>
    + * It is not recommended to use this with JSON text where individual JSON objects that can be large (MB's or larger).
    + */
    +public class PartitionedJsonParser {
    +
    +	private static final int EOF = -1;
    +	private final InputStreamReader inputStreamReader;
    +	private final JsonLexer lexer;
    +	private long bytesRead = 0;
    +	private boolean endOfStream;
    +
    +	public PartitionedJsonParser(InputStream is) {
    +		this.lexer = new JsonLexer();
    +
    +		// You need to wrap the InputStream with an InputStreamReader, so that it can encode the incoming byte stream as
    +		// UTF-8 characters
    +		this.inputStreamReader = new InputStreamReader(is, StandardCharsets.UTF_8);
    +	}
    +
    +	private boolean scanToFirstBeginObject() throws IOException {
    +		// seek until we hit the first begin-object
    +		char prev = ' ';
    +		int i;
    +		while ((i = inputStreamReader.read()) != EOF) {
    +			char c = (char) i;
    +			bytesRead++;
    +			if (c == '{' && prev != '\\') {
    --- End diff --
    
    `{` and `\\` should probably be consts


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by hornn <gi...@git.apache.org>.
Github user hornn commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51212448
  
    --- Diff: pxf/pxf-json/src/main/java/org/apache/hawq/pxf/plugins/json/JsonInputFormat.java ---
    @@ -0,0 +1,226 @@
    +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.BufferedInputStream;
    +import java.io.IOException;
    +import java.security.InvalidParameterException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.fs.FSDataInputStream;
    +import org.apache.hadoop.fs.FileSystem;
    +import org.apache.hadoop.fs.Path;
    +import org.apache.hadoop.io.LongWritable;
    +import org.apache.hadoop.io.NullWritable;
    +import org.apache.hadoop.io.Text;
    +import org.apache.hadoop.mapred.FileInputFormat;
    +import org.apache.hadoop.mapred.FileSplit;
    +import org.apache.hadoop.mapred.InputSplit;
    +import org.apache.hadoop.mapred.JobConf;
    +import org.apache.hadoop.mapred.LineRecordReader;
    +import org.apache.hadoop.mapred.RecordReader;
    +import org.apache.hadoop.mapred.Reporter;
    +import org.apache.log4j.Logger;
    +import org.codehaus.jackson.JsonFactory;
    +import org.codehaus.jackson.JsonNode;
    +import org.codehaus.jackson.JsonParseException;
    +import org.codehaus.jackson.map.JsonMappingException;
    +import org.codehaus.jackson.map.ObjectMapper;
    +
    +public class JsonInputFormat extends FileInputFormat<Text, NullWritable> {
    +
    +	private static JsonFactory factory = new JsonFactory();
    +	private static ObjectMapper mapper = new ObjectMapper(factory);
    +
    +	public static final String ONE_RECORD_PER_LINE = "json.input.format.one.record.per.line";
    +	public static final String RECORD_IDENTIFIER = "json.input.format.record.identifier";
    +
    +	@Override
    +	public RecordReader<Text, NullWritable> getRecordReader(InputSplit split, JobConf conf, Reporter reporter)
    +			throws IOException {
    +
    +		if (conf.getBoolean(ONE_RECORD_PER_LINE, false)) {
    +
    +			return new SimpleJsonRecordReader(conf, (FileSplit) split);
    +		} else {
    +			return new JsonRecordReader(conf, (FileSplit) split);
    +		}
    +	}
    +
    +	public static class SimpleJsonRecordReader implements RecordReader<Text, NullWritable> {
    +
    +		private LineRecordReader rdr = null;
    +		private LongWritable key = new LongWritable();
    +		private Text value = new Text();
    +
    +		public SimpleJsonRecordReader(Configuration conf, FileSplit split) throws IOException {
    +			rdr = new LineRecordReader(conf, split);
    +		}
    +
    +		@Override
    +		public void close() throws IOException {
    +			rdr.close();
    +		}
    +
    +		@Override
    +		public Text createKey() {
    +			return value;
    +		}
    +
    +		@Override
    +		public NullWritable createValue() {
    +			return NullWritable.get();
    +		}
    +
    +		@Override
    +		public long getPos() throws IOException {
    +			return rdr.getPos();
    +		}
    +
    +		@Override
    +		public boolean next(Text key, NullWritable value) throws IOException {
    +			if (rdr.next(this.key, this.value)) {
    +				key.set(this.value);
    +				return true;
    +			} else {
    +				return false;
    +			}
    +		}
    +
    +		@Override
    +		public float getProgress() throws IOException {
    +			return rdr.getProgress();
    +		}
    +	}
    +
    +	public static class JsonRecordReader implements RecordReader<Text, NullWritable> {
    +
    +		private Logger LOG = Logger.getLogger(JsonRecordReader.class);
    +
    +		private JsonStreamReader rdr = null;
    +		private long start = 0, end = 0;
    +		private float toRead = 0;
    +		private String identifier = null;
    +		private Logger log = Logger.getLogger(JsonRecordReader.class);
    +
    +		public JsonRecordReader(JobConf conf, FileSplit split) throws IOException {
    +			log.info("JsonRecordReader constructor called.  Conf is " + conf + ". Split is " + split);
    +			this.identifier = conf.get(RECORD_IDENTIFIER);
    +			log.info("Identifier is " + this.identifier);
    +
    +			if (this.identifier == null || identifier.isEmpty()) {
    +				throw new InvalidParameterException(JsonInputFormat.RECORD_IDENTIFIER + " is not set.");
    +			} else {
    +				LOG.info("Initializing JsonRecordReader with identifier " + identifier);
    +			}
    +
    +			// get relevant data
    +			Path file = split.getPath();
    +
    +			log.info("File is " + file);
    +
    +			start = split.getStart();
    +			end = start + split.getLength();
    +			toRead = end - start;
    +			log.info("FileSystem is " + FileSystem.get(conf));
    +
    +			FSDataInputStream strm = FileSystem.get(conf).open(file);
    +
    +			log.info("Retrieved file stream ");
    +
    +			if (start != 0) {
    +				strm.seek(start);
    +			}
    +
    +			rdr = new JsonStreamReader(identifier, new BufferedInputStream(strm));
    +
    +			log.info("Reader is " + rdr);
    +		}
    +
    +		@Override
    +		public boolean next(Text key, NullWritable value) throws IOException {
    +
    +			boolean retval = false;
    +			boolean keepGoing = false;
    +			do {
    +				// Exit condition (end of block/file)
    +				if (rdr.getBytesRead() >= (end - start)) {
    +					return false;
    +				}
    +
    +				keepGoing = false;
    +				String record = rdr.getJsonRecord();
    +				if (record != null) {
    +					if (JsonInputFormat.decodeLineToJsonNode(record) == null) {
    +						log.error("Unable to parse JSON string.  Skipping. DEBUG to see");
    +						log.debug(record);
    +						keepGoing = true;
    +					} else {
    +						key.set(record);
    +						retval = true;
    +					}
    +				}
    +			} while (keepGoing);
    +
    +			return retval;
    +		}
    +
    +		@Override
    +		public Text createKey() {
    +			return new Text();
    +		}
    +
    +		@Override
    +		public NullWritable createValue() {
    +			return NullWritable.get();
    +		}
    +
    +		@Override
    +		public long getPos() throws IOException {
    +			return start + rdr.getBytesRead();
    +		}
    +
    +		@Override
    +		public void close() throws IOException {
    +			rdr.close();
    +		}
    +
    +		@Override
    +		public float getProgress() throws IOException {
    +			return (float) rdr.getBytesRead() / toRead;
    +		}
    +	}
    +
    +	public static synchronized JsonNode decodeLineToJsonNode(String line) {
    --- End diff --
    
    why does the mapper have to be static? wouldn't it be better to make it a member of the JsonInputFormat, and then not worry about synchronizing this function?
    When handling multiple requests, it probably has a performance implication.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by tzolov <gi...@git.apache.org>.
Github user tzolov commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51861378
  
    --- Diff: pxf/pxf-json/src/test/java/org/apache/hawq/pxf/plugins/json/JsonExtensionTest.java ---
    @@ -0,0 +1,288 @@
    +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.File;
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import org.apache.hadoop.fs.Path;
    +import org.apache.hawq.pxf.api.Fragmenter;
    +import org.apache.hawq.pxf.api.ReadAccessor;
    +import org.apache.hawq.pxf.api.ReadResolver;
    +import org.apache.hawq.pxf.api.io.DataType;
    +import org.apache.hawq.pxf.plugins.hdfs.HdfsDataFragmenter;
    +import org.apache.hawq.pxf.plugins.json.JsonAccessor;
    +import org.apache.hawq.pxf.plugins.json.JsonResolver;
    +import org.junit.After;
    +import org.junit.Before;
    +import org.junit.Test;
    +
    +public class JsonExtensionTest extends PxfUnit {
    +
    +	private List<Pair<String, DataType>> columnDefs = null;
    +	private List<Pair<String, String>> extraParams = new ArrayList<Pair<String, String>>();
    +
    +	@Before
    +	public void before() {
    +
    +		columnDefs = new ArrayList<Pair<String, DataType>>();
    +
    +		columnDefs.add(new Pair<String, DataType>("created_at", DataType.TEXT));
    +		columnDefs.add(new Pair<String, DataType>("id", DataType.BIGINT));
    +		columnDefs.add(new Pair<String, DataType>("text", DataType.TEXT));
    +		columnDefs.add(new Pair<String, DataType>("user.screen_name", DataType.TEXT));
    +		columnDefs.add(new Pair<String, DataType>("entities.hashtags[0]", DataType.TEXT));
    +		columnDefs.add(new Pair<String, DataType>("coordinates.coordinates[0]", DataType.FLOAT8));
    +		columnDefs.add(new Pair<String, DataType>("coordinates.coordinates[1]", DataType.FLOAT8));
    +	}
    +
    +	@After
    +	public void cleanup() throws Exception {
    +		columnDefs.clear();
    +		extraParams.clear();
    +	}
    +
    +	@Test
    +	public void testCompressedMultilineJsonFile() throws Exception {
    +
    +		extraParams.add(new Pair<String, String>("IDENTIFIER", "created_at"));
    +
    +		List<String> output = new ArrayList<String>();
    +
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547115253761,text1,SpreadButter,tweetCongress,,");
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547123646465,text2,patronusdeadly,,,");
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547136233472,text3,NoSecrets_Vagas,,,");
    +
    +		super.assertOutput(new Path(System.getProperty("user.dir") + File.separator
    +				+ "src/test/resources/tweets.tar.gz"), output);
    +	}
    +
    +	@Test
    +	public void testMaxRecordLength() throws Exception {
    +
    +		// variable-size-objects.json contains 3 json objects but only 2 of them fit in the 27 byte length limitation
    +
    +		extraParams.add(new Pair<String, String>("IDENTIFIER", "key666"));
    +		extraParams.add(new Pair<String, String>("MAXLENGTH", "27"));
    +
    +		columnDefs.clear();
    +		columnDefs.add(new Pair<String, DataType>("key666", DataType.TEXT));
    +
    +		List<String> output = new ArrayList<String>();
    +
    +		output.add("small object1");
    +		// skip the large object2 XXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX
    +		output.add("small object3");
    +
    +		super.assertOutput(new Path(System.getProperty("user.dir") + File.separator
    +				+ "src/test/resources/variable-size-objects.json"), output);
    +	}
    +
    +	@Test
    +	public void testDataTypes() throws Exception {
    +
    +		// TDOO: The BYTEA type is not tested!!! Current oneField.val = val.asText().getBytes(); convention is
    +		// disputable!
    +
    +		extraParams.clear();
    +		extraParams.add(new Pair<String, String>("IDENTIFIER", "bintType"));
    +
    +		columnDefs.clear();
    +		columnDefs.add(new Pair<String, DataType>("bintType", DataType.BIGINT));
    +		columnDefs.add(new Pair<String, DataType>("booleanType", DataType.BOOLEAN));
    +		columnDefs.add(new Pair<String, DataType>("charType", DataType.CHAR));
    +		// columnDefs.add(new Pair<String, DataType>("byteaType", DataType.BYTEA));
    +		columnDefs.add(new Pair<String, DataType>("float8Type", DataType.FLOAT8));
    +		columnDefs.add(new Pair<String, DataType>("realType", DataType.REAL));
    +		columnDefs.add(new Pair<String, DataType>("integerType", DataType.INTEGER));
    +		columnDefs.add(new Pair<String, DataType>("smallintType", DataType.SMALLINT));
    +		columnDefs.add(new Pair<String, DataType>("bpcharType", DataType.BPCHAR));
    +		columnDefs.add(new Pair<String, DataType>("varcharType", DataType.VARCHAR));
    +		columnDefs.add(new Pair<String, DataType>("text", DataType.TEXT));
    +
    +		List<String> output = new ArrayList<String>();
    +
    +		output.add("666,true,x,3.14,3.15,999,777,bpcharType,varcharType,");
    +
    +		super.assertOutput(new Path(System.getProperty("user.dir") + File.separator
    +				+ "src/test/resources/datatypes-test.json"), output);
    +	}
    +
    +	@Test(expected = IllegalStateException.class)
    +	public void testMissingArrayJsonAttribute() throws Exception {
    +
    +		extraParams.add(new Pair<String, String>("IDENTIFIER", "created_at"));
    +
    +		columnDefs.clear();
    +
    +		columnDefs.add(new Pair<String, DataType>("created_at", DataType.TEXT));
    +		// User is not an array! An attempt to access it should throw an exception!
    +		columnDefs.add(new Pair<String, DataType>("user[0]", DataType.TEXT));
    +
    +		List<String> output = new ArrayList<String>();
    +
    +		super.assertOutput(new Path(System.getProperty("user.dir") + File.separator
    +				+ "src/test/resources/tweets-with-missing-text-attribtute.json"), output);
    +	}
    +
    +	@Test
    +	public void testMissingJsonAttribute() throws Exception {
    +
    +		List<String> output = new ArrayList<String>();
    +
    +		extraParams.add(new Pair<String, String>("IDENTIFIER", "created_at"));
    +
    +		// Missing attributes are substituted by an empty field
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547115253761,,SpreadButter,tweetCongress,,");
    +
    +		super.assertOutput(new Path(System.getProperty("user.dir") + File.separator
    +				+ "src/test/resources/tweets-with-missing-text-attribtute.json"), output);
    +	}
    +
    +	@Test
    +	public void testMalformedJsonObject() throws Exception {
    +
    +		List<String> output = new ArrayList<String>();
    +
    +		extraParams.add(new Pair<String, String>("IDENTIFIER", "created_at"));
    +
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547115253761,text1,SpreadButter,tweetCongress,,");
    +		output.add(",,,,,,"); // Expected: malformed json records are transformed into empty rows
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547136233472,text3,NoSecrets_Vagas,,,");
    +
    +		super.assertOutput(new Path(System.getProperty("user.dir") + File.separator
    +				+ "src/test/resources/tweets-broken.json"), output);
    +	}
    +
    +	@Test
    +	public void testSmallTweets() throws Exception {
    +
    +		List<String> output = new ArrayList<String>();
    +
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547115253761,text1,SpreadButter,tweetCongress,,");
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547123646465,text2,patronusdeadly,,,");
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547136233472,text3,NoSecrets_Vagas,,,");
    +		output.add("Fri Jun 07 22:45:03 +0000 2013,343136551322136576,text4,SevenStonesBuoy,,-6.1,50.103");
    +
    +		super.assertOutput(new Path(System.getProperty("user.dir") + File.separator
    +				+ "src/test/resources/tweets-small.json"), output);
    +	}
    +
    +	@Test
    +	public void testTweetsWithNull() throws Exception {
    +
    +		List<String> output = new ArrayList<String>();
    +
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,,text1,SpreadButter,tweetCongress,,");
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,,text2,patronusdeadly,,,");
    +
    +		super.assertOutput(new Path(System.getProperty("user.dir") + File.separator
    +				+ "src/test/resources/null-tweets.json"), output);
    +	}
    +
    +	@Test
    +	public void testSmallTweetsWithDelete() throws Exception {
    --- End diff --
    
    Shows that the simple (e.g. LineRecordReader) mode can handle missing or null json attributes. Those are converted into empty fields.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by hornn <gi...@git.apache.org>.
Github user hornn commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51509571
  
    --- Diff: pxf/pxf-json/src/main/java/org/apache/hawq/pxf/plugins/json/JsonUtil.java ---
    @@ -0,0 +1,55 @@
    +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 org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.codehaus.jackson.JsonFactory;
    +import org.codehaus.jackson.JsonNode;
    +import org.codehaus.jackson.map.ObjectMapper;
    +
    +/**
    + * Utility that helps to convert JSON text into {@link JsonNode} object
    + *
    + */
    +public class JsonUtil {
    --- End diff --
    
    in the other plugins, we usually put the utils classes under their own package org.apache.hawq.pxf.plugins.json.utilities.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by adamjshook <gi...@git.apache.org>.
Github user adamjshook commented on the pull request:

    https://github.com/apache/incubator-hawq/pull/302#issuecomment-178615116
  
    @tzolov A quick review of the json-mapreduce project looks good to me.  I think it'd be best to roll it in now to avoid any inconvenience as you have stated.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by hornn <gi...@git.apache.org>.
Github user hornn commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51323277
  
    --- Diff: pxf/pxf-json/src/test/java/org/apache/pxf/hawq/plugins/json/PxfUnit.java ---
    @@ -0,0 +1,666 @@
    +package org.apache.pxf.hawq.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.BufferedReader;
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.InputStreamReader;
    +import java.io.OutputStream;
    +import java.lang.reflect.Constructor;
    +import java.security.InvalidParameterException;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.fs.FileSystem;
    +import org.apache.hadoop.fs.Path;
    +import org.apache.hawq.pxf.api.Fragment;
    +import org.apache.hawq.pxf.api.Fragmenter;
    +import org.apache.hawq.pxf.api.OneField;
    +import org.apache.hawq.pxf.api.OneRow;
    +import org.apache.hawq.pxf.api.ReadAccessor;
    +import org.apache.hawq.pxf.api.ReadResolver;
    +import org.apache.hawq.pxf.api.WriteAccessor;
    +import org.apache.hawq.pxf.api.WriteResolver;
    +import org.apache.hawq.pxf.api.io.DataType;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +import org.apache.hawq.pxf.service.FragmentsResponse;
    +import org.apache.hawq.pxf.service.FragmentsResponseFormatter;
    +import org.apache.hawq.pxf.service.utilities.ProtocolData;
    +import org.codehaus.jackson.JsonFactory;
    +import org.codehaus.jackson.JsonNode;
    +import org.codehaus.jackson.JsonParseException;
    +import org.codehaus.jackson.map.JsonMappingException;
    +import org.codehaus.jackson.map.ObjectMapper;
    +import org.junit.Assert;
    +
    +/**
    + * This abstract class contains a number of helpful utilities in developing a PXF extension for HAWQ. Extend this class
    + * and use the various <code>assert</code> methods to check given input against known output.
    + */
    +public abstract class PxfUnit {
    +
    +	private static JsonFactory factory = new JsonFactory();
    +	private static ObjectMapper mapper = new ObjectMapper(factory);
    +
    +	protected static List<InputData> inputs = null;
    +
    +	/**
    +	 * Uses the given input directory to run through the PXF unit testing framework. Uses the lines in the file for
    +	 * output testing.
    +	 * 
    +	 * @param input
    +	 *            Input records
    +	 * @param expectedOutput
    +	 *            File containing output to check
    +	 * @throws Exception
    +	 */
    +	public void assertOutput(Path input, Path expectedOutput) throws Exception {
    +
    +		BufferedReader rdr = new BufferedReader(new InputStreamReader(FileSystem.get(new Configuration()).open(
    --- End diff --
    
    close the resources in the end of the function


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by hornn <gi...@git.apache.org>.
Github user hornn commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51317402
  
    --- Diff: pxf/pxf-json/src/main/java/org/apache/hawq/pxf/plugins/json/JsonStreamReader.java ---
    @@ -0,0 +1,89 @@
    +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.BufferedReader;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.io.InputStreamReader;
    +
    +public class JsonStreamReader extends BufferedReader {
    +
    +	private StringBuilder bldr = new StringBuilder();
    +	private String identifier = null;
    +	private long bytesRead = 0;
    +
    +	public JsonStreamReader(String identifier, InputStream strm) {
    +		super(new InputStreamReader(strm));
    +		this.identifier = identifier;
    --- End diff --
    
    throw an exception if it's empty or null?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by hornn <gi...@git.apache.org>.
Github user hornn commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51324302
  
    --- Diff: pxf/pxf-json/src/test/java/org/apache/pxf/hawq/plugins/json/PxfUnit.java ---
    @@ -0,0 +1,666 @@
    +package org.apache.pxf.hawq.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.BufferedReader;
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.InputStreamReader;
    +import java.io.OutputStream;
    +import java.lang.reflect.Constructor;
    +import java.security.InvalidParameterException;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.fs.FileSystem;
    +import org.apache.hadoop.fs.Path;
    +import org.apache.hawq.pxf.api.Fragment;
    +import org.apache.hawq.pxf.api.Fragmenter;
    +import org.apache.hawq.pxf.api.OneField;
    +import org.apache.hawq.pxf.api.OneRow;
    +import org.apache.hawq.pxf.api.ReadAccessor;
    +import org.apache.hawq.pxf.api.ReadResolver;
    +import org.apache.hawq.pxf.api.WriteAccessor;
    +import org.apache.hawq.pxf.api.WriteResolver;
    +import org.apache.hawq.pxf.api.io.DataType;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +import org.apache.hawq.pxf.service.FragmentsResponse;
    +import org.apache.hawq.pxf.service.FragmentsResponseFormatter;
    +import org.apache.hawq.pxf.service.utilities.ProtocolData;
    +import org.codehaus.jackson.JsonFactory;
    +import org.codehaus.jackson.JsonNode;
    +import org.codehaus.jackson.JsonParseException;
    +import org.codehaus.jackson.map.JsonMappingException;
    +import org.codehaus.jackson.map.ObjectMapper;
    +import org.junit.Assert;
    +
    +/**
    + * This abstract class contains a number of helpful utilities in developing a PXF extension for HAWQ. Extend this class
    + * and use the various <code>assert</code> methods to check given input against known output.
    + */
    +public abstract class PxfUnit {
    +
    +	private static JsonFactory factory = new JsonFactory();
    +	private static ObjectMapper mapper = new ObjectMapper(factory);
    +
    +	protected static List<InputData> inputs = null;
    +
    +	/**
    +	 * Uses the given input directory to run through the PXF unit testing framework. Uses the lines in the file for
    +	 * output testing.
    +	 * 
    +	 * @param input
    +	 *            Input records
    +	 * @param expectedOutput
    +	 *            File containing output to check
    +	 * @throws Exception
    +	 */
    +	public void assertOutput(Path input, Path expectedOutput) throws Exception {
    +
    +		BufferedReader rdr = new BufferedReader(new InputStreamReader(FileSystem.get(new Configuration()).open(
    +				expectedOutput)));
    +
    +		List<String> outputLines = new ArrayList<String>();
    +
    +		String line;
    +		while ((line = rdr.readLine()) != null) {
    +			outputLines.add(line);
    +		}
    +
    +		assertOutput(input, outputLines);
    +	}
    +
    +	/**
    +	 * Uses the given input directory to run through the PXF unit testing framework. Uses the lines in the given
    +	 * parameter for output testing.
    +	 * 
    +	 * @param input
    +	 *            Input records
    +	 * @param expectedOutput
    +	 *            File containing output to check
    +	 * @throws Exception
    +	 */
    +	public void assertOutput(Path input, List<String> expectedOutput) throws Exception {
    +
    +		setup(input);
    +		List<String> actualOutput = new ArrayList<String>();
    +		for (InputData data : inputs) {
    +			ReadAccessor accessor = getReadAccessor(data);
    +			ReadResolver resolver = getReadResolver(data);
    +
    +			actualOutput.addAll(getAllOutput(accessor, resolver));
    +		}
    +
    +		Assert.assertFalse("Output did not match expected output", compareOutput(expectedOutput, actualOutput));
    +	}
    +
    +	/**
    +	 * Uses the given input directory to run through the PXF unit testing framework. Uses the lines in the given
    +	 * parameter for output testing.<br>
    +	 * <br>
    +	 * Ignores order of records.
    +	 * 
    +	 * @param input
    +	 *            Input records
    +	 * @param expectedOutput
    +	 *            File containing output to check
    +	 * @throws Exception
    +	 */
    +	public void assertUnorderedOutput(Path input, Path expectedOutput) throws Exception {
    +		BufferedReader rdr = new BufferedReader(new InputStreamReader(FileSystem.get(new Configuration()).open(
    +				expectedOutput)));
    +
    +		List<String> outputLines = new ArrayList<String>();
    +
    +		String line;
    +		while ((line = rdr.readLine()) != null) {
    +			outputLines.add(line);
    +		}
    +
    +		assertUnorderedOutput(input, outputLines);
    +	}
    +
    +	/**
    +	 * Uses the given input directory to run through the PXF unit testing framework. Uses the lines in the file for
    +	 * output testing.<br>
    +	 * <br>
    +	 * Ignores order of records.
    +	 * 
    +	 * @param input
    +	 *            Input records
    +	 * @param expectedOutput
    +	 *            File containing output to check
    +	 * @throws Exception
    +	 */
    +	public void assertUnorderedOutput(Path input, List<String> expectedOutput) throws Exception {
    +
    +		setup(input);
    +
    +		List<String> actualOutput = new ArrayList<String>();
    +		for (InputData data : inputs) {
    +			ReadAccessor accessor = getReadAccessor(data);
    +			ReadResolver resolver = getReadResolver(data);
    +
    +			actualOutput.addAll(getAllOutput(accessor, resolver));
    +		}
    +
    +		Assert.assertFalse("Output did not match expected output", compareUnorderedOutput(expectedOutput, actualOutput));
    +	}
    +
    +	/**
    +	 * Writes the output to the given output stream. Comma delimiter.
    +	 * 
    +	 * @param input
    +	 *            The input file
    +	 * @param output
    +	 *            The output stream
    +	 * @throws Exception
    +	 */
    +	public void writeOutput(Path input, OutputStream output) throws Exception {
    +
    +		setup(input);
    +
    +		for (InputData data : inputs) {
    +			ReadAccessor accessor = getReadAccessor(data);
    +			ReadResolver resolver = getReadResolver(data);
    +
    +			for (String line : getAllOutput(accessor, resolver)) {
    +				output.write((line + "\n").getBytes());
    +			}
    +		}
    +
    +		output.flush();
    +	}
    +
    +	/**
    +	 * Get the class of the implementation of Fragmenter to be tested.
    +	 * 
    +	 * @return The class
    +	 */
    +	public Class<? extends Fragmenter> getFragmenterClass() {
    +		return null;
    +	}
    +
    +	/**
    +	 * Get the class of the implementation of ReadAccessor to be tested.
    +	 * 
    +	 * @return The class
    +	 */
    +	public Class<? extends ReadAccessor> getReadAccessorClass() {
    +		return null;
    +	}
    +
    +	/**
    +	 * Get the class of the implementation of WriteAccessor to be tested.
    +	 * 
    +	 * @return The class
    +	 */
    +	public Class<? extends WriteAccessor> getWriteAccessorClass() {
    +		return null;
    +	}
    +
    +	/**
    +	 * Get the class of the implementation of Resolver to be tested.
    +	 * 
    +	 * @return The class
    +	 */
    +	public Class<? extends ReadResolver> getReadResolverClass() {
    +		return null;
    +	}
    +
    +	/**
    +	 * Get the class of the implementation of WriteResolver to be tested.
    +	 * 
    +	 * @return The class
    +	 */
    +	public Class<? extends WriteResolver> getWriteResolverClass() {
    +		return null;
    +	}
    +
    +	/**
    +	 * Get any extra parameters that are meant to be specified for the "pxf" protocol. Note that "X-GP-" is prepended to
    +	 * each parameter name.
    +	 * 
    +	 * @return Any extra parameters or null if none.
    +	 */
    +	public List<Pair<String, String>> getExtraParams() {
    +		return null;
    +	}
    +
    +	/**
    +	 * Gets the column definition names and data types. Types are DataType objects
    +	 * 
    +	 * @return A list of column definition name value pairs. Cannot be null.
    +	 */
    +	public abstract List<Pair<String, DataType>> getColumnDefinitions();
    +
    +	protected InputData getInputDataForWritableTable() {
    +		return getInputDataForWritableTable(null);
    +	}
    +
    +	protected InputData getInputDataForWritableTable(Path input) {
    +
    +		if (getWriteAccessorClass() == null) {
    +			throw new IllegalArgumentException(
    +					"getWriteAccessorClass() must be overwritten to return a non-null object");
    +		}
    +
    +		if (getWriteResolverClass() == null) {
    +			throw new IllegalArgumentException(
    +					"getWriteResolverClass() must be overwritten to return a non-null object");
    +		}
    +
    +		Map<String, String> paramsMap = new HashMap<String, String>();
    +
    +		paramsMap.put("X-GP-ALIGNMENT", "what");
    +		paramsMap.put("X-GP-SEGMENT-ID", "1");
    +		paramsMap.put("X-GP-HAS-FILTER", "0");
    +		paramsMap.put("X-GP-SEGMENT-COUNT", "1");
    +
    +		paramsMap.put("X-GP-FORMAT", "GPDBWritable");
    +		paramsMap.put("X-GP-URL-HOST", "localhost");
    +		paramsMap.put("X-GP-URL-PORT", "50070");
    +
    +		if (input == null) {
    +			paramsMap.put("X-GP-DATA-DIR", "/dummydata");
    +		}
    +
    +		List<Pair<String, DataType>> params = getColumnDefinitions();
    +		paramsMap.put("X-GP-ATTRS", Integer.toString(params.size()));
    +		for (int i = 0; i < params.size(); ++i) {
    +			paramsMap.put("X-GP-ATTR-NAME" + i, params.get(i).first);
    +			paramsMap.put("X-GP-ATTR-TYPENAME" + i, params.get(i).second.name());
    +			paramsMap.put("X-GP-ATTR-TYPECODE" + i, Integer.toString(params.get(i).second.getOID()));
    +		}
    +
    +		paramsMap.put("X-GP-ACCESSOR", getWriteAccessorClass().getName());
    +		paramsMap.put("X-GP-RESOLVER", getWriteResolverClass().getName());
    +
    +		if (getExtraParams() != null) {
    +			for (Pair<String, String> param : getExtraParams()) {
    +				paramsMap.put("X-GP-" + param.first, param.second);
    +			}
    +		}
    +
    +		return new ProtocolData(paramsMap);
    +	}
    +
    +	/**
    +	 * Set all necessary parameters for GPXF framework to function. Uses the given path as a single input split.
    +	 * 
    +	 * @param input
    +	 *            The input path, relative or absolute.
    +	 * @throws Exception
    +	 */
    +	protected void setup(Path input) throws Exception {
    +
    +		if (getFragmenterClass() == null) {
    +			throw new IllegalArgumentException("getFragmenterClass() must be overwritten to return a non-null object");
    +		}
    +
    +		if (getReadAccessorClass() == null) {
    +			throw new IllegalArgumentException("getReadAccessorClass() must be overwritten to return a non-null object");
    +		}
    +
    +		if (getReadResolverClass() == null) {
    +			throw new IllegalArgumentException("getReadResolverClass() must be overwritten to return a non-null object");
    +		}
    +
    +		Map<String, String> paramsMap = new HashMap<String, String>();
    +
    +		// 2.1.0 Properties
    +		// HDMetaData parameters
    +		paramsMap.put("X-GP-ALIGNMENT", "what");
    +		paramsMap.put("X-GP-SEGMENT-ID", "1");
    +		paramsMap.put("X-GP-HAS-FILTER", "0");
    +		paramsMap.put("X-GP-SEGMENT-COUNT", "1");
    +		paramsMap.put("X-GP-FRAGMENTER", getFragmenterClass().getName());
    +		paramsMap.put("X-GP-FORMAT", "GPDBWritable");
    +		paramsMap.put("X-GP-URL-HOST", "localhost");
    +		paramsMap.put("X-GP-URL-PORT", "50070");
    +
    +		paramsMap.put("X-GP-DATA-DIR", input.toString());
    +
    +		List<Pair<String, DataType>> params = getColumnDefinitions();
    +		paramsMap.put("X-GP-ATTRS", Integer.toString(params.size()));
    +		for (int i = 0; i < params.size(); ++i) {
    +			paramsMap.put("X-GP-ATTR-NAME" + i, params.get(i).first);
    +			paramsMap.put("X-GP-ATTR-TYPENAME" + i, params.get(i).second.name());
    +			paramsMap.put("X-GP-ATTR-TYPECODE" + i, Integer.toString(params.get(i).second.getOID()));
    +		}
    +
    +		// HDFSMetaData properties
    +		paramsMap.put("X-GP-ACCESSOR", getReadAccessorClass().getName());
    +		paramsMap.put("X-GP-RESOLVER", getReadResolverClass().getName());
    +
    +		if (getExtraParams() != null) {
    +			for (Pair<String, String> param : getExtraParams()) {
    +				paramsMap.put("X-GP-" + param.first, param.second);
    +			}
    +		}
    +
    +		LocalInputData fragmentInputData = new LocalInputData(paramsMap);
    +
    +		List<Fragment> fragments = getFragmenter(fragmentInputData).getFragments();
    +
    +		FragmentsResponse fragmentsResponse = FragmentsResponseFormatter.formatResponse(fragments, input.toString());
    +
    +		ByteArrayOutputStream baos = new ByteArrayOutputStream();
    +		fragmentsResponse.write(baos);
    +
    +		String jsonOutput = baos.toString();
    +
    +		inputs = new ArrayList<InputData>();
    +
    +		JsonNode node = decodeLineToJsonNode(jsonOutput);
    +
    +		JsonNode fragmentsArray = node.get("PXFFragments");
    +		int i = 0;
    +		Iterator<JsonNode> iter = fragmentsArray.getElements();
    +		while (iter.hasNext()) {
    +			JsonNode fragNode = iter.next();
    +			String sourceData = fragNode.get("sourceName").getTextValue();
    +			if (!sourceData.startsWith("/")) {
    +				sourceData = "/" + sourceData;
    +			}
    +			paramsMap.put("X-GP-DATA-DIR", sourceData);
    +			paramsMap.put("X-GP-FRAGMENT-METADATA", fragNode.get("metadata").getTextValue());
    +			paramsMap.put("X-GP-DATA-FRAGMENT", Integer.toString(i++));
    +			inputs.add(new LocalInputData(paramsMap));
    +		}
    +	}
    +
    +	private JsonNode decodeLineToJsonNode(String line) {
    --- End diff --
    
    code duplication? can use the same function in json accessor (perhaps move to JsonUtils class?)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by hornn <gi...@git.apache.org>.
Github user hornn commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51322311
  
    --- Diff: pxf/pxf-json/src/main/java/org/apache/hawq/pxf/plugins/json/JsonStreamReader.java ---
    @@ -0,0 +1,89 @@
    +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.BufferedReader;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.io.InputStreamReader;
    +
    +public class JsonStreamReader extends BufferedReader {
    +
    +	private StringBuilder bldr = new StringBuilder();
    +	private String identifier = null;
    +	private long bytesRead = 0;
    +
    +	public JsonStreamReader(String identifier, InputStream strm) {
    +		super(new InputStreamReader(strm));
    +		this.identifier = identifier;
    +	}
    +
    +	public String getJsonRecord() throws IOException {
    +		bldr.delete(0, bldr.length());
    +
    +		boolean foundRecord = false;
    +
    +		int c = 0, numBraces = 1;
    +		while ((c = super.read()) != -1) {
    --- End diff --
    
    better to replace -1 with EOF const


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by tzolov <gi...@git.apache.org>.
Github user tzolov commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51856329
  
    --- Diff: pxf/pxf-json/src/main/java/org/apache/hawq/pxf/plugins/json/JsonResolver.java ---
    @@ -0,0 +1,255 @@
    +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.util.ArrayList;
    +import java.util.Iterator;
    +import java.util.List;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +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.JsonFactory;
    +import org.codehaus.jackson.JsonNode;
    +import org.codehaus.jackson.map.ObjectMapper;
    +
    +/**
    + * 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 static final Log LOG = LogFactory.getLog(JsonResolver.class);
    +
    +	private ArrayList<OneField> oneFieldList;
    +	private ColumnDescriptorCache[] columnDescriptorCache;
    +	private ObjectMapper mapper;
    +
    +	public JsonResolver(InputData inputData) throws Exception {
    +		super(inputData);
    +		oneFieldList = new ArrayList<OneField>();
    +		mapper = new ObjectMapper(new JsonFactory());
    +
    +		// Pre-generate all column structure attributes concerning the JSON to column value resolution.
    +		columnDescriptorCache = new ColumnDescriptorCache[inputData.getColumns()];
    +		for (int i = 0; i < inputData.getColumns(); ++i) {
    +			ColumnDescriptor cd = inputData.getColumn(i);
    +			columnDescriptorCache[i] = new ColumnDescriptorCache(cd);
    +		}
    +	}
    +
    +	@Override
    +	public List<OneField> getFields(OneRow row) throws Exception {
    +		oneFieldList.clear();
    +
    +		String jsonRecordAsText = row.getData().toString();
    +
    +		JsonNode root = decodeLineToJsonNode(jsonRecordAsText);
    +
    +		if (root == null) {
    +			LOG.warn("Return null-fields row due to invalid JSON:" + jsonRecordAsText);
    --- End diff --
    
    +1


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by hornn <gi...@git.apache.org>.
Github user hornn commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51210479
  
    --- Diff: pxf/pxf-json/src/main/java/org/apache/hawq/pxf/plugins/json/JsonInputFormat.java ---
    @@ -0,0 +1,226 @@
    +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.BufferedInputStream;
    +import java.io.IOException;
    +import java.security.InvalidParameterException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.fs.FSDataInputStream;
    +import org.apache.hadoop.fs.FileSystem;
    +import org.apache.hadoop.fs.Path;
    +import org.apache.hadoop.io.LongWritable;
    +import org.apache.hadoop.io.NullWritable;
    +import org.apache.hadoop.io.Text;
    +import org.apache.hadoop.mapred.FileInputFormat;
    +import org.apache.hadoop.mapred.FileSplit;
    +import org.apache.hadoop.mapred.InputSplit;
    +import org.apache.hadoop.mapred.JobConf;
    +import org.apache.hadoop.mapred.LineRecordReader;
    +import org.apache.hadoop.mapred.RecordReader;
    +import org.apache.hadoop.mapred.Reporter;
    +import org.apache.log4j.Logger;
    +import org.codehaus.jackson.JsonFactory;
    +import org.codehaus.jackson.JsonNode;
    +import org.codehaus.jackson.JsonParseException;
    +import org.codehaus.jackson.map.JsonMappingException;
    +import org.codehaus.jackson.map.ObjectMapper;
    +
    +public class JsonInputFormat extends FileInputFormat<Text, NullWritable> {
    +
    +	private static JsonFactory factory = new JsonFactory();
    +	private static ObjectMapper mapper = new ObjectMapper(factory);
    +
    +	public static final String ONE_RECORD_PER_LINE = "json.input.format.one.record.per.line";
    +	public static final String RECORD_IDENTIFIER = "json.input.format.record.identifier";
    +
    +	@Override
    +	public RecordReader<Text, NullWritable> getRecordReader(InputSplit split, JobConf conf, Reporter reporter)
    +			throws IOException {
    +
    +		if (conf.getBoolean(ONE_RECORD_PER_LINE, false)) {
    +
    +			return new SimpleJsonRecordReader(conf, (FileSplit) split);
    +		} else {
    +			return new JsonRecordReader(conf, (FileSplit) split);
    +		}
    +	}
    +
    +	public static class SimpleJsonRecordReader implements RecordReader<Text, NullWritable> {
    +
    +		private LineRecordReader rdr = null;
    +		private LongWritable key = new LongWritable();
    +		private Text value = new Text();
    +
    +		public SimpleJsonRecordReader(Configuration conf, FileSplit split) throws IOException {
    +			rdr = new LineRecordReader(conf, split);
    +		}
    +
    +		@Override
    +		public void close() throws IOException {
    +			rdr.close();
    +		}
    +
    +		@Override
    +		public Text createKey() {
    +			return value;
    +		}
    +
    +		@Override
    +		public NullWritable createValue() {
    +			return NullWritable.get();
    +		}
    +
    +		@Override
    +		public long getPos() throws IOException {
    +			return rdr.getPos();
    +		}
    +
    +		@Override
    +		public boolean next(Text key, NullWritable value) throws IOException {
    --- End diff --
    
    could you please add documentation for this function?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by tzolov <gi...@git.apache.org>.
Github user tzolov commented on the pull request:

    https://github.com/apache/incubator-hawq/pull/302#issuecomment-178314296
  
    Hopefully the last committ resolves the remaing remarks


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by GodenYao <gi...@git.apache.org>.
Github user GodenYao commented on the pull request:

    https://github.com/apache/incubator-hawq/pull/302#issuecomment-178936416
  
    yes @rvs , coincidentally, we're revamping NOTICE file for HAWQ release.
    Roman should be able to incorporate this in if you commit this week. Otherwise, we need to add it later.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by sansanichfb <gi...@git.apache.org>.
Github user sansanichfb commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r63605864
  
    --- Diff: pxf/pxf-json/src/main/java/org/apache/hawq/pxf/plugins/json/JsonResolver.java ---
    @@ -0,0 +1,256 @@
    +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.util.ArrayList;
    +import java.util.Iterator;
    +import java.util.List;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +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.JsonFactory;
    +import org.codehaus.jackson.JsonNode;
    +import org.codehaus.jackson.map.ObjectMapper;
    +
    +/**
    + * 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 static final Log LOG = LogFactory.getLog(JsonResolver.class);
    +
    +	private ArrayList<OneField> oneFieldList;
    +	private ColumnDescriptorCache[] columnDescriptorCache;
    +	private ObjectMapper mapper;
    +
    +	/**
    +	 * Row with empty fields. Returned in case of broken or malformed json records.
    +	 */
    +	private final List<OneField> emptyRow;
    +
    +	public JsonResolver(InputData inputData) throws Exception {
    +		super(inputData);
    +		oneFieldList = new ArrayList<OneField>();
    +		mapper = new ObjectMapper(new JsonFactory());
    +
    +		// Precompute the column metadata. The metadata is used for mapping column names to json nodes.
    +		columnDescriptorCache = new ColumnDescriptorCache[inputData.getColumns()];
    +		for (int i = 0; i < inputData.getColumns(); ++i) {
    +			ColumnDescriptor cd = inputData.getColumn(i);
    +			columnDescriptorCache[i] = new ColumnDescriptorCache(cd);
    +		}
    +
    +		emptyRow = createEmptyRow();
    +	}
    +
    +	@Override
    +	public List<OneField> getFields(OneRow row) throws Exception {
    +		oneFieldList.clear();
    +
    +		String jsonRecordAsText = row.getData().toString();
    +
    +		JsonNode root = decodeLineToJsonNode(jsonRecordAsText);
    +
    +		if (root == null) {
    +			LOG.warn("Return empty-fields row due to invalid JSON: " + jsonRecordAsText);
    +			return emptyRow;
    +		}
    +
    +		// Iterate through the column definition and fetch our JSON data
    +		for (ColumnDescriptorCache columnMetadata : columnDescriptorCache) {
    +
    +			JsonNode node = getChildJsonNode(root, columnMetadata.getNormalizedProjections());
    +
    +			// If this node is null or missing, add a null value here
    +			if (node == null || node.isMissingNode()) {
    +				addNullField(columnMetadata.getColumnType());
    +			} else if (columnMetadata.isArray()) {
    +				// If this column is an array index, ex. "tweet.hashtags[0]"
    +				if (node.isArray()) {
    +					// If the JSON node is an array, then add it to our list
    +					addFieldFromJsonArray(columnMetadata.getColumnType(), node, columnMetadata.getArrayNodeIndex());
    +				} else {
    +					throw new IllegalStateException(columnMetadata.getColumnName() + " is not an array node");
    +				}
    +			} else {
    +				// This column is not an array type
    +				// Add the value to the record
    +				addFieldFromJsonNode(columnMetadata.getColumnType(), node);
    +			}
    +		}
    +
    +		return oneFieldList;
    +	}
    +
    +	/**
    +	 * @return Returns a row comprised of typed, empty fields. Used as a result of broken/malformed json records.
    +	 */
    +	private List<OneField> createEmptyRow() {
    +		ArrayList<OneField> emptyFieldList = new ArrayList<OneField>();
    +		for (ColumnDescriptorCache column : columnDescriptorCache) {
    +			emptyFieldList.add(new OneField(column.getColumnType().getOID(), null));
    +		}
    +		return emptyFieldList;
    +	}
    +
    +	/**
    +	 * Iterates down the root node to the child JSON node defined by the projs path.
    +	 * 
    +	 * @param root
    +	 *            node to to start the traversal from.
    +	 * @param projs
    +	 *            defines the path from the root to the desired child node.
    +	 * @return Returns the child node defined by the root and projs path.
    +	 */
    +	private JsonNode getChildJsonNode(JsonNode root, String[] projs) {
    +
    +		// Iterate through all the tokens to the desired JSON node
    +		JsonNode node = root;
    +		for (int j = 0; j < projs.length; ++j) {
    +			node = node.path(projs[j]);
    +		}
    +
    +		return node;
    +	}
    +
    +	/**
    +	 * Iterates through the given JSON node to the proper index and adds the field of corresponding type
    +	 * 
    +	 * @param type
    +	 *            The {@link DataType} type
    +	 * @param node
    +	 *            The JSON array node
    +	 * @param index
    +	 *            The array index to iterate to
    +	 * @throws IOException
    +	 */
    +	private void addFieldFromJsonArray(DataType type, JsonNode node, int index) throws IOException {
    +
    +		int count = 0;
    +		boolean added = false;
    +		for (Iterator<JsonNode> arrayNodes = node.getElements(); arrayNodes.hasNext();) {
    +			JsonNode arrayNode = arrayNodes.next();
    +
    +			if (count == index) {
    +				added = true;
    +				addFieldFromJsonNode(type, arrayNode);
    +				break;
    +			}
    +
    +			++count;
    +		}
    +
    +		// if we reached the end of the array without adding a field, add null
    +		if (!added) {
    +			addNullField(type);
    +		}
    +	}
    +
    +	/**
    +	 * Adds a field from a given JSON node value based on the {@link DataType} type.
    +	 * 
    +	 * @param type
    +	 *            The DataType type
    +	 * @param val
    +	 *            The JSON node to extract the value.
    +	 * @throws IOException
    +	 */
    +	private void addFieldFromJsonNode(DataType type, JsonNode val) throws IOException {
    +		OneField oneField = new OneField();
    +		oneField.type = type.getOID();
    +
    +		if (val.isNull()) {
    +			oneField.val = null;
    +		} else {
    +			switch (type) {
    +			case BIGINT:
    +				oneField.val = val.asLong();
    +				break;
    +			case BOOLEAN:
    +				oneField.val = val.asBoolean();
    +				break;
    +			case CHAR:
    +				oneField.val = val.asText().charAt(0);
    +				break;
    +			case BYTEA:
    --- End diff --
    
    As far as bytea is not tested and isn't supported, should we remove this section?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by GodenYao <gi...@git.apache.org>.
Github user GodenYao commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51210646
  
    --- Diff: pxf/pxf-json/src/test/resources/tweets-small.json ---
    @@ -0,0 +1,4 @@
    +{"created_at":"Fri Jun 07 22:45:02 +0000 2013","id":343136547115253761,"text":"REPAIR THE TRUST: REMOVE OBAMA\/BIDEN FROM OFFICE. #IRS #DOJ #NSA #tcot","source":"web","user":{"id":"26643566","name":"SpreadButter","screen_name":"SpreadButter","location":"Austin, Texas"},"entities":{"hashtags":["tweetCongress","IRS"]}}
    --- End diff --
    
    better use non-political content for test example.
    I think you've fixed it in PDF guide but missed here :)
    https://issues.apache.org/jira/secure/attachment/12785045/PXFJSONPluginforHAWQ2.0andPXF3.0.0.pdf


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by hornn <gi...@git.apache.org>.
Github user hornn commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r52079820
  
    --- Diff: pxf/pxf-json/src/main/java/org/apache/hawq/pxf/plugins/json/JsonResolver.java ---
    @@ -0,0 +1,252 @@
    +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.util.ArrayList;
    +import java.util.Iterator;
    +import java.util.List;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +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.JsonFactory;
    +import org.codehaus.jackson.JsonNode;
    +import org.codehaus.jackson.map.ObjectMapper;
    +
    +/**
    + * 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 static final Log LOG = LogFactory.getLog(JsonResolver.class);
    +
    +	private ArrayList<OneField> oneFieldList;
    +	private ColumnDescriptorCache[] columnDescriptorCache;
    +	private ObjectMapper mapper;
    +
    +	/**
    +	 * Row with empty fields. Returned in case of broken or malformed json records.
    +	 */
    +	private final List<OneField> emptyRow;
    +
    +	public JsonResolver(InputData inputData) throws Exception {
    +		super(inputData);
    +		oneFieldList = new ArrayList<OneField>();
    +		mapper = new ObjectMapper(new JsonFactory());
    +
    +		// Precompute the column metadata. The metadata is used for mapping column names to json nodes.
    +		columnDescriptorCache = new ColumnDescriptorCache[inputData.getColumns()];
    +		for (int i = 0; i < inputData.getColumns(); ++i) {
    +			ColumnDescriptor cd = inputData.getColumn(i);
    +			columnDescriptorCache[i] = new ColumnDescriptorCache(cd);
    +		}
    +
    +		emptyRow = createEmptyRow();
    +	}
    +
    +	@Override
    +	public List<OneField> getFields(OneRow row) throws Exception {
    +		oneFieldList.clear();
    +
    +		String jsonRecordAsText = row.getData().toString();
    +
    +		JsonNode root = decodeLineToJsonNode(jsonRecordAsText);
    +
    +		if (root == null) {
    +			LOG.warn("Return empty-fields row due to invalid JSON: " + jsonRecordAsText);
    +			return emptyRow;
    +		}
    +
    +		// Iterate through the column definition and fetch our JSON data
    +		for (ColumnDescriptorCache columnMetadata : columnDescriptorCache) {
    +
    +			JsonNode node = getChildJsonNode(root, columnMetadata.getNormalizedProjections());
    +
    +			// If this node is null or missing, add a null value here
    +			if (node == null || node.isMissingNode()) {
    +				addNullField(columnMetadata.getColumnType());
    +			} else if (columnMetadata.isArray()) {
    +				// If this column is an array index, ex. "tweet.hashtags[0]"
    +				if (node.isArray()) {
    +					// If the JSON node is an array, then add it to our list
    +					addFieldFromJsonArray(columnMetadata.getColumnType(), node, columnMetadata.getArrayNodeIndex());
    +				} else {
    +					throw new IllegalStateException(columnMetadata.getColumnName() + " is not an array node");
    +				}
    +			} else {
    +				// This column is not an array type
    +				// Add the value to the record
    +				addFieldFromJsonNode(columnMetadata.getColumnType(), node);
    +			}
    +		}
    +
    +		return oneFieldList;
    +	}
    +
    +	/**
    +	 * @return Returns a row comprised of typed, empty fields. Used as a result of broken/malformed json records.
    +	 */
    +	private List<OneField> createEmptyRow() {
    +		ArrayList<OneField> emptyFieldList = new ArrayList<OneField>();
    +		for (ColumnDescriptorCache column : columnDescriptorCache) {
    +			emptyFieldList.add(new OneField(column.getColumnType().getOID(), null));
    +		}
    +		return emptyFieldList;
    +	}
    +
    +	/**
    +	 * Iterates down the root node to the child JSON node defined by the projs path.
    +	 * 
    +	 * @param root
    +	 *            node to to start the traversal from.
    +	 * @param projs
    +	 *            defines the path from the root to the desired child node.
    +	 * @return Returns the child node defined by the root and projs path.
    +	 */
    +	private JsonNode getChildJsonNode(JsonNode root, String[] projs) {
    +
    +		// Iterate through all the tokens to the desired JSON node
    +		JsonNode node = root;
    +		for (int j = 0; j < projs.length; ++j) {
    +			node = node.path(projs[j]);
    +		}
    +
    +		return node;
    +	}
    +
    +	/**
    +	 * Iterates through the given JSON node to the proper index and adds the field of corresponding type
    +	 * 
    +	 * @param type
    +	 *            The {@link DataType} type
    +	 * @param node
    +	 *            The JSON array node
    +	 * @param index
    +	 *            The array index to iterate to
    +	 * @throws IOException
    +	 */
    +	private void addFieldFromJsonArray(DataType type, JsonNode node, int index) throws IOException {
    +
    +		int count = 0;
    +		boolean added = false;
    +		for (Iterator<JsonNode> arrayNodes = node.getElements(); arrayNodes.hasNext();) {
    +			JsonNode arrayNode = arrayNodes.next();
    +
    +			if (count == index) {
    +				added = true;
    +				addFieldFromJsonNode(type, arrayNode);
    +				break;
    +			}
    +
    +			++count;
    +		}
    +
    +		// if we reached the end of the array without adding a field, add null
    +		if (!added) {
    +			addNullField(type);
    +		}
    +	}
    +
    +	/**
    +	 * Adds a field from a given JSON node value based on the {@link DataType} type.
    +	 * 
    +	 * @param type
    +	 *            The DataType type
    +	 * @param val
    +	 *            The JSON node to extract the value.
    +	 * @throws IOException
    +	 */
    +	private void addFieldFromJsonNode(DataType type, JsonNode val) throws IOException {
    +		OneField oneField = new OneField();
    +		oneField.type = type.getOID();
    +
    +		if (val.isNull()) {
    +			oneField.val = null;
    +		} else {
    +			switch (type) {
    +			case BIGINT:
    +				oneField.val = val.asLong();
    +				break;
    +			case BOOLEAN:
    +				oneField.val = val.asBoolean();
    +				break;
    +			case CHAR:
    +				oneField.val = val.asText().charAt(0);
    +				break;
    +			case BYTEA:
    +				oneField.val = val.asText().getBytes();
    +				break;
    +			case FLOAT8:
    +			case REAL:
    +				oneField.val = val.asDouble();
    +				break;
    +			case INTEGER:
    +			case SMALLINT:
    +				oneField.val = val.asInt();
    +				break;
    +			case BPCHAR:
    +			case TEXT:
    +			case VARCHAR:
    +				oneField.val = val.asText();
    +				break;
    +			default:
    +				throw new IOException("Unsupported type " + type);
    +			}
    +		}
    +
    +		oneFieldList.add(oneField);
    +	}
    +
    +	/**
    +	 * Adds a null field of the given type.
    +	 * 
    +	 * @param type
    +	 *            The {@link DataType} type
    +	 */
    +	private void addNullField(DataType type) {
    +		oneFieldList.add(new OneField(type.getOID(), null));
    +	}
    +
    +	/**
    +	 * Converts the input line parameter into {@link JsonNode} instance.
    +	 * 
    +	 * @param line
    +	 *            JSON text
    +	 * @return Returns a {@link JsonNode} that represents the input line or null for invalid json.
    +	 */
    +	private JsonNode decodeLineToJsonNode(String line) {
    +
    +		try {
    +			return mapper.readTree(line);
    +		} catch (Exception e) {
    +			LOG.error("Failed to pars JSON object", e);
    --- End diff --
    
    typo parse


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by hornn <gi...@git.apache.org>.
Github user hornn commented on the pull request:

    https://github.com/apache/incubator-hawq/pull/302#issuecomment-178685025
  
    I agree. We can afford to wait a little longer for a better plugin.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by tzolov <gi...@git.apache.org>.
Github user tzolov commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51492890
  
    --- Diff: pxf/pxf-json/src/test/java/org/apache/pxf/hawq/plugins/json/PxfUnit.java ---
    @@ -0,0 +1,666 @@
    +package org.apache.pxf.hawq.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.BufferedReader;
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.InputStreamReader;
    +import java.io.OutputStream;
    +import java.lang.reflect.Constructor;
    +import java.security.InvalidParameterException;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.fs.FileSystem;
    +import org.apache.hadoop.fs.Path;
    +import org.apache.hawq.pxf.api.Fragment;
    +import org.apache.hawq.pxf.api.Fragmenter;
    +import org.apache.hawq.pxf.api.OneField;
    +import org.apache.hawq.pxf.api.OneRow;
    +import org.apache.hawq.pxf.api.ReadAccessor;
    +import org.apache.hawq.pxf.api.ReadResolver;
    +import org.apache.hawq.pxf.api.WriteAccessor;
    +import org.apache.hawq.pxf.api.WriteResolver;
    +import org.apache.hawq.pxf.api.io.DataType;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +import org.apache.hawq.pxf.service.FragmentsResponse;
    +import org.apache.hawq.pxf.service.FragmentsResponseFormatter;
    +import org.apache.hawq.pxf.service.utilities.ProtocolData;
    +import org.codehaus.jackson.JsonFactory;
    +import org.codehaus.jackson.JsonNode;
    +import org.codehaus.jackson.JsonParseException;
    +import org.codehaus.jackson.map.JsonMappingException;
    +import org.codehaus.jackson.map.ObjectMapper;
    +import org.junit.Assert;
    +
    +/**
    + * This abstract class contains a number of helpful utilities in developing a PXF extension for HAWQ. Extend this class
    + * and use the various <code>assert</code> methods to check given input against known output.
    + */
    +public abstract class PxfUnit {
    +
    +	private static JsonFactory factory = new JsonFactory();
    +	private static ObjectMapper mapper = new ObjectMapper(factory);
    +
    +	protected static List<InputData> inputs = null;
    +
    +	/**
    +	 * Uses the given input directory to run through the PXF unit testing framework. Uses the lines in the file for
    +	 * output testing.
    +	 * 
    +	 * @param input
    +	 *            Input records
    +	 * @param expectedOutput
    +	 *            File containing output to check
    +	 * @throws Exception
    +	 */
    +	public void assertOutput(Path input, Path expectedOutput) throws Exception {
    +
    +		BufferedReader rdr = new BufferedReader(new InputStreamReader(FileSystem.get(new Configuration()).open(
    --- End diff --
    
    done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by tzolov <gi...@git.apache.org>.
Github user tzolov commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51509583
  
    --- Diff: pxf/pxf-json/src/main/java/org/apache/hawq/pxf/plugins/json/JsonStreamReader.java ---
    @@ -0,0 +1,121 @@
    +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.BufferedReader;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.io.InputStreamReader;
    +
    +/**
    + * Reads a JSON stream and sequentially extracts all JSON objects identified by the <b>identifier</> parameter. Returns
    + * null when there are no more objects to read.
    + *
    + */
    +public class JsonStreamReader extends BufferedReader {
    +
    +	private static char START_BRACE = '{';
    +	private static char END_BRACE = '}';
    +	private static int EOF = -1;
    +
    +	private StringBuilder bldr = new StringBuilder();
    +	private String identifier = null;
    +	private long bytesRead = 0;
    +
    +	/**
    +	 * @param identifier
    +	 *            JSON object name to extract.
    +	 * @param inputStream
    +	 *            JSON document input stream.
    +	 */
    +	public JsonStreamReader(String identifier, InputStream inputStream) {
    +		super(new InputStreamReader(inputStream));
    +		this.identifier = identifier;
    +	}
    +
    +	/**
    +	 * @return Returns next JSON object identified by the {@link JsonStreamReader#identifier} parameter or Null if no
    +	 *         more object are available.
    --- End diff --
    
    sure


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by hornn <gi...@git.apache.org>.
Github user hornn commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51452876
  
    --- Diff: pxf/build.gradle ---
    @@ -286,6 +286,29 @@ project('pxf-hive') {
         }
     }
     
    +project('pxf-json') {
    +	dependencies {
    +		compile(project(':pxf-hdfs'))
    +		compile "org.apache.commons:commons-lang3:3.0"
    +
    +		testCompile 'pl.pragmatists:JUnitParams:1.0.2'
    +		configurations {
    +			// Remove hive-exec from unit tests as it causes VerifyError
    +			testRuntime.exclude module: 'hive-exec'
    --- End diff --
    
    I mean the exclude part - it refers to hive module that is not a dependency for this plugin.
    Could you try to remove the configurations block and run make to see if it works?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by GodenYao <gi...@git.apache.org>.
Github user GodenYao commented on the pull request:

    https://github.com/apache/incubator-hawq/pull/302#issuecomment-178109535
  
    @adamjshook , thanks for your contribution to JSON plugin and now everything is open sourced , so feel free to send us your PR if any!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by hornn <gi...@git.apache.org>.
Github user hornn commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51210691
  
    --- Diff: pxf/pxf-json/src/main/java/org/apache/hawq/pxf/plugins/json/JsonInputFormat.java ---
    @@ -0,0 +1,226 @@
    +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.BufferedInputStream;
    +import java.io.IOException;
    +import java.security.InvalidParameterException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.fs.FSDataInputStream;
    +import org.apache.hadoop.fs.FileSystem;
    +import org.apache.hadoop.fs.Path;
    +import org.apache.hadoop.io.LongWritable;
    +import org.apache.hadoop.io.NullWritable;
    +import org.apache.hadoop.io.Text;
    +import org.apache.hadoop.mapred.FileInputFormat;
    +import org.apache.hadoop.mapred.FileSplit;
    +import org.apache.hadoop.mapred.InputSplit;
    +import org.apache.hadoop.mapred.JobConf;
    +import org.apache.hadoop.mapred.LineRecordReader;
    +import org.apache.hadoop.mapred.RecordReader;
    +import org.apache.hadoop.mapred.Reporter;
    +import org.apache.log4j.Logger;
    +import org.codehaus.jackson.JsonFactory;
    +import org.codehaus.jackson.JsonNode;
    +import org.codehaus.jackson.JsonParseException;
    +import org.codehaus.jackson.map.JsonMappingException;
    +import org.codehaus.jackson.map.ObjectMapper;
    +
    +public class JsonInputFormat extends FileInputFormat<Text, NullWritable> {
    +
    +	private static JsonFactory factory = new JsonFactory();
    +	private static ObjectMapper mapper = new ObjectMapper(factory);
    +
    +	public static final String ONE_RECORD_PER_LINE = "json.input.format.one.record.per.line";
    +	public static final String RECORD_IDENTIFIER = "json.input.format.record.identifier";
    +
    +	@Override
    +	public RecordReader<Text, NullWritable> getRecordReader(InputSplit split, JobConf conf, Reporter reporter)
    +			throws IOException {
    +
    +		if (conf.getBoolean(ONE_RECORD_PER_LINE, false)) {
    +
    +			return new SimpleJsonRecordReader(conf, (FileSplit) split);
    +		} else {
    +			return new JsonRecordReader(conf, (FileSplit) split);
    +		}
    +	}
    +
    +	public static class SimpleJsonRecordReader implements RecordReader<Text, NullWritable> {
    +
    +		private LineRecordReader rdr = null;
    +		private LongWritable key = new LongWritable();
    +		private Text value = new Text();
    +
    +		public SimpleJsonRecordReader(Configuration conf, FileSplit split) throws IOException {
    +			rdr = new LineRecordReader(conf, split);
    +		}
    +
    +		@Override
    +		public void close() throws IOException {
    +			rdr.close();
    +		}
    +
    +		@Override
    +		public Text createKey() {
    +			return value;
    +		}
    +
    +		@Override
    +		public NullWritable createValue() {
    +			return NullWritable.get();
    +		}
    +
    +		@Override
    +		public long getPos() throws IOException {
    +			return rdr.getPos();
    +		}
    +
    +		@Override
    +		public boolean next(Text key, NullWritable value) throws IOException {
    +			if (rdr.next(this.key, this.value)) {
    +				key.set(this.value);
    +				return true;
    +			} else {
    +				return false;
    +			}
    +		}
    +
    +		@Override
    +		public float getProgress() throws IOException {
    +			return rdr.getProgress();
    +		}
    +	}
    +
    +	public static class JsonRecordReader implements RecordReader<Text, NullWritable> {
    +
    +		private Logger LOG = Logger.getLogger(JsonRecordReader.class);
    +
    +		private JsonStreamReader rdr = null;
    +		private long start = 0, end = 0;
    +		private float toRead = 0;
    +		private String identifier = null;
    +		private Logger log = Logger.getLogger(JsonRecordReader.class);
    --- End diff --
    
    why having two loggers?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by tzolov <gi...@git.apache.org>.
Github user tzolov commented on the pull request:

    https://github.com/apache/incubator-hawq/pull/302#issuecomment-178922329
  
    @hornn , @GodenYao since we are reusing code from the [json-mapreduce](https://github.com/alexholmes/json-mapreduce) (APL2.0) do we have to mention it in the NOTICE section?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by tzolov <gi...@git.apache.org>.
Github user tzolov commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51346659
  
    --- Diff: pxf/pxf-json/src/main/java/org/apache/hawq/pxf/plugins/json/JsonInputFormat.java ---
    @@ -0,0 +1,226 @@
    +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.BufferedInputStream;
    +import java.io.IOException;
    +import java.security.InvalidParameterException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.fs.FSDataInputStream;
    +import org.apache.hadoop.fs.FileSystem;
    +import org.apache.hadoop.fs.Path;
    +import org.apache.hadoop.io.LongWritable;
    +import org.apache.hadoop.io.NullWritable;
    +import org.apache.hadoop.io.Text;
    +import org.apache.hadoop.mapred.FileInputFormat;
    +import org.apache.hadoop.mapred.FileSplit;
    +import org.apache.hadoop.mapred.InputSplit;
    +import org.apache.hadoop.mapred.JobConf;
    +import org.apache.hadoop.mapred.LineRecordReader;
    +import org.apache.hadoop.mapred.RecordReader;
    +import org.apache.hadoop.mapred.Reporter;
    +import org.apache.log4j.Logger;
    +import org.codehaus.jackson.JsonFactory;
    +import org.codehaus.jackson.JsonNode;
    +import org.codehaus.jackson.JsonParseException;
    +import org.codehaus.jackson.map.JsonMappingException;
    +import org.codehaus.jackson.map.ObjectMapper;
    +
    +public class JsonInputFormat extends FileInputFormat<Text, NullWritable> {
    +
    +	private static JsonFactory factory = new JsonFactory();
    +	private static ObjectMapper mapper = new ObjectMapper(factory);
    +
    +	public static final String ONE_RECORD_PER_LINE = "json.input.format.one.record.per.line";
    +	public static final String RECORD_IDENTIFIER = "json.input.format.record.identifier";
    +
    +	@Override
    +	public RecordReader<Text, NullWritable> getRecordReader(InputSplit split, JobConf conf, Reporter reporter)
    +			throws IOException {
    +
    +		if (conf.getBoolean(ONE_RECORD_PER_LINE, false)) {
    +
    +			return new SimpleJsonRecordReader(conf, (FileSplit) split);
    +		} else {
    +			return new JsonRecordReader(conf, (FileSplit) split);
    +		}
    +	}
    +
    +	public static class SimpleJsonRecordReader implements RecordReader<Text, NullWritable> {
    +
    +		private LineRecordReader rdr = null;
    +		private LongWritable key = new LongWritable();
    +		private Text value = new Text();
    +
    +		public SimpleJsonRecordReader(Configuration conf, FileSplit split) throws IOException {
    +			rdr = new LineRecordReader(conf, split);
    +		}
    +
    +		@Override
    +		public void close() throws IOException {
    +			rdr.close();
    +		}
    +
    +		@Override
    +		public Text createKey() {
    +			return value;
    +		}
    +
    +		@Override
    +		public NullWritable createValue() {
    +			return NullWritable.get();
    +		}
    +
    +		@Override
    +		public long getPos() throws IOException {
    +			return rdr.getPos();
    +		}
    +
    +		@Override
    +		public boolean next(Text key, NullWritable value) throws IOException {
    +			if (rdr.next(this.key, this.value)) {
    +				key.set(this.value);
    +				return true;
    +			} else {
    +				return false;
    +			}
    +		}
    +
    +		@Override
    +		public float getProgress() throws IOException {
    +			return rdr.getProgress();
    +		}
    +	}
    +
    +	public static class JsonRecordReader implements RecordReader<Text, NullWritable> {
    +
    +		private Logger LOG = Logger.getLogger(JsonRecordReader.class);
    +
    +		private JsonStreamReader rdr = null;
    +		private long start = 0, end = 0;
    +		private float toRead = 0;
    +		private String identifier = null;
    +		private Logger log = Logger.getLogger(JsonRecordReader.class);
    +
    +		public JsonRecordReader(JobConf conf, FileSplit split) throws IOException {
    +			log.info("JsonRecordReader constructor called.  Conf is " + conf + ". Split is " + split);
    --- End diff --
    
    yep will change the log level to `debug` 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by cwelton <gi...@git.apache.org>.
Github user cwelton commented on the pull request:

    https://github.com/apache/incubator-hawq/pull/302#issuecomment-185328825
  
    What is the current status on this PR?  Are there still outstanding issues to resolve, or is it ready for merge?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by tzolov <gi...@git.apache.org>.
Github user tzolov commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51400966
  
    --- 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,
    +				projs[projs.length - 1].length() - 1));
    +	}
    +
    +	/**
    +	 * Iterates through the given JSON node to the proper index and adds the field of corresponding type
    +	 * 
    +	 * @param type
    +	 *            The {@link DataType} type
    +	 * @param node
    +	 *            The JSON array node
    +	 * @param index
    +	 *            The array index to iterate to
    +	 * @throws IOException
    +	 */
    +	private void addFieldFromJsonArray(DataType type, JsonNode node, int index) throws IOException {
    +
    +		int count = 0;
    +		boolean added = false;
    +		for (Iterator<JsonNode> arrayNodes = node.getElements(); arrayNodes.hasNext();) {
    +			JsonNode arrayNode = arrayNodes.next();
    +
    +			if (count == index) {
    +				added = true;
    +				addFieldFromJsonNode(type, arrayNode);
    +				break;
    +			}
    +
    +			++count;
    +		}
    +
    +		// if we reached the end of the array without adding a
    +		// field, add null
    +		if (!added) {
    +			addNullField(type);
    +		}
    +	}
    +
    +	/**
    +	 * Adds a field from a given JSON node value based on the {@link DataType} type.
    +	 * 
    +	 * @param type
    +	 *            The DataType type
    +	 * @param val
    +	 *            The JSON node to extract the value.
    +	 * @throws IOException
    +	 */
    +	private void addFieldFromJsonNode(DataType type, JsonNode val) throws IOException {
    +		OneField oneField = new OneField();
    +		oneField.type = type.getOID();
    +
    +		if (val.isNull()) {
    +			oneField.val = null;
    +		} else {
    +			switch (type) {
    +			case BIGINT:
    +				oneField.val = val.asLong();
    +				break;
    +			case BOOLEAN:
    +				oneField.val = val.asBoolean();
    +				break;
    +			case BPCHAR:
    --- End diff --
    
    addressed!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by tzolov <gi...@git.apache.org>.
Github user tzolov commented on the pull request:

    https://github.com/apache/incubator-hawq/pull/302#issuecomment-178059442
  
    Alhtough there are no end-tags (like XLM) one can use the begingin of the next `identifer` as end-tag new-tab signal. If not mistaken this is how current implementation is meant to work? I'm also exploring this project related [json-mapreduce](https://github.com/alexholmes/json-mapreduce) that uses a similar approach. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by hornn <gi...@git.apache.org>.
Github user hornn commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51212943
  
    --- 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
    --- End diff --
    
    please fill up all the parameters in the javadoc. It causes the build to fail when compiling with java8.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by tzolov <gi...@git.apache.org>.
Github user tzolov commented on the pull request:

    https://github.com/apache/incubator-hawq/pull/302#issuecomment-178728932
  
    Here it comes. The improved JsonRecordReader implementation is committed. Mind the new identifier semantics.  Also as a safeguard you can set the max record length allowed to parse. 
    The new implementation should support compression as well (not tested)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by tzolov <gi...@git.apache.org>.
Github user tzolov commented on the pull request:

    https://github.com/apache/incubator-hawq/pull/302#issuecomment-178608613
  
    @hornn, @adamjshook  
    As an experimenti've rempimplemented/replaced the `JsonRecordReader`&`JsonStreamReader` with ideas and code boroed from the [json-mapreduce](https://github.com/alexholmes/json-mapreduce) project.  I really like the result. It comes with a (sort of) JsonLexer/Parser that solves some of the shortcomings in the current implementation. For example having the `identifier` in the nested object(s) values will be handled. 
    Furthermore they why how the identifier  is used is slightly differently and more powerful. The identifier refers to a member name which it will use to determine the encapsulating object to return.  This is  superior functionality over  what we have at the moment as you can point to multiline json objects that don't have a parent identifier. 
    
    Shall i add this change to this PR or to a separate one after this has been merged? 
    Personally i think it is better to add it now as it changes the `identifier` semantics and would be inconvenient for the potential users to learn/unlearn. Alternatively we can drop the current JsonRecordReader form this PR and introduce the new code as an extension i another one. What do you think?



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by tzolov <gi...@git.apache.org>.
Github user tzolov commented on the pull request:

    https://github.com/apache/incubator-hawq/pull/302#issuecomment-179344304
  
    In the last commit(s): 
    * Added tests to cover corner cases like malformed json records or compressed input files. The _combined_ test coverage is > 95%. @hornn is there an integration tests/environment to itest PXF plugins?
    * Improved handling of malformed json records/fields. If the json record is not parsable the Resolver will produce a row with empty fields.  If a field in the record is missing then an empty db field is created.
    * Factored out the column metadata computation logic and data in a dedicated `ColumnDescriptorCache` class. Now the cached descriptor metadata is created only once inside the `JsonResolver` constructor. Previously it use to be re-computed on each `JsonResolver.getFields()` method call. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by tzolov <gi...@git.apache.org>.
Github user tzolov commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51347426
  
    --- Diff: pxf/pxf-json/src/test/resources/tweets-small.json ---
    @@ -0,0 +1,4 @@
    +{"created_at":"Fri Jun 07 22:45:02 +0000 2013","id":343136547115253761,"text":"REPAIR THE TRUST: REMOVE OBAMA\/BIDEN FROM OFFICE. #IRS #DOJ #NSA #tcot","source":"web","user":{"id":"26643566","name":"SpreadButter","screen_name":"SpreadButter","location":"Austin, Texas"},"entities":{"hashtags":["tweetCongress","IRS"]}}
    --- End diff --
    
    sure, i will clean the tweets text content


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by hornn <gi...@git.apache.org>.
Github user hornn commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51835615
  
    --- Diff: pxf/pxf-json/src/test/java/org/apache/hawq/pxf/plugins/json/JsonExtensionTest.java ---
    @@ -0,0 +1,288 @@
    +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.File;
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import org.apache.hadoop.fs.Path;
    +import org.apache.hawq.pxf.api.Fragmenter;
    +import org.apache.hawq.pxf.api.ReadAccessor;
    +import org.apache.hawq.pxf.api.ReadResolver;
    +import org.apache.hawq.pxf.api.io.DataType;
    +import org.apache.hawq.pxf.plugins.hdfs.HdfsDataFragmenter;
    +import org.apache.hawq.pxf.plugins.json.JsonAccessor;
    +import org.apache.hawq.pxf.plugins.json.JsonResolver;
    +import org.junit.After;
    +import org.junit.Before;
    +import org.junit.Test;
    +
    +public class JsonExtensionTest extends PxfUnit {
    +
    +	private List<Pair<String, DataType>> columnDefs = null;
    +	private List<Pair<String, String>> extraParams = new ArrayList<Pair<String, String>>();
    +
    +	@Before
    +	public void before() {
    +
    +		columnDefs = new ArrayList<Pair<String, DataType>>();
    +
    +		columnDefs.add(new Pair<String, DataType>("created_at", DataType.TEXT));
    +		columnDefs.add(new Pair<String, DataType>("id", DataType.BIGINT));
    +		columnDefs.add(new Pair<String, DataType>("text", DataType.TEXT));
    +		columnDefs.add(new Pair<String, DataType>("user.screen_name", DataType.TEXT));
    +		columnDefs.add(new Pair<String, DataType>("entities.hashtags[0]", DataType.TEXT));
    +		columnDefs.add(new Pair<String, DataType>("coordinates.coordinates[0]", DataType.FLOAT8));
    +		columnDefs.add(new Pair<String, DataType>("coordinates.coordinates[1]", DataType.FLOAT8));
    +	}
    +
    +	@After
    +	public void cleanup() throws Exception {
    +		columnDefs.clear();
    +		extraParams.clear();
    +	}
    +
    +	@Test
    +	public void testCompressedMultilineJsonFile() throws Exception {
    +
    +		extraParams.add(new Pair<String, String>("IDENTIFIER", "created_at"));
    +
    +		List<String> output = new ArrayList<String>();
    +
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547115253761,text1,SpreadButter,tweetCongress,,");
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547123646465,text2,patronusdeadly,,,");
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547136233472,text3,NoSecrets_Vagas,,,");
    +
    +		super.assertOutput(new Path(System.getProperty("user.dir") + File.separator
    +				+ "src/test/resources/tweets.tar.gz"), output);
    +	}
    +
    +	@Test
    +	public void testMaxRecordLength() throws Exception {
    +
    +		// variable-size-objects.json contains 3 json objects but only 2 of them fit in the 27 byte length limitation
    +
    +		extraParams.add(new Pair<String, String>("IDENTIFIER", "key666"));
    +		extraParams.add(new Pair<String, String>("MAXLENGTH", "27"));
    +
    +		columnDefs.clear();
    +		columnDefs.add(new Pair<String, DataType>("key666", DataType.TEXT));
    +
    +		List<String> output = new ArrayList<String>();
    +
    +		output.add("small object1");
    +		// skip the large object2 XXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX
    +		output.add("small object3");
    +
    +		super.assertOutput(new Path(System.getProperty("user.dir") + File.separator
    +				+ "src/test/resources/variable-size-objects.json"), output);
    +	}
    +
    +	@Test
    +	public void testDataTypes() throws Exception {
    +
    +		// TDOO: The BYTEA type is not tested!!! Current oneField.val = val.asText().getBytes(); convention is
    +		// disputable!
    +
    +		extraParams.clear();
    +		extraParams.add(new Pair<String, String>("IDENTIFIER", "bintType"));
    +
    +		columnDefs.clear();
    +		columnDefs.add(new Pair<String, DataType>("bintType", DataType.BIGINT));
    +		columnDefs.add(new Pair<String, DataType>("booleanType", DataType.BOOLEAN));
    +		columnDefs.add(new Pair<String, DataType>("charType", DataType.CHAR));
    +		// columnDefs.add(new Pair<String, DataType>("byteaType", DataType.BYTEA));
    +		columnDefs.add(new Pair<String, DataType>("float8Type", DataType.FLOAT8));
    +		columnDefs.add(new Pair<String, DataType>("realType", DataType.REAL));
    +		columnDefs.add(new Pair<String, DataType>("integerType", DataType.INTEGER));
    +		columnDefs.add(new Pair<String, DataType>("smallintType", DataType.SMALLINT));
    +		columnDefs.add(new Pair<String, DataType>("bpcharType", DataType.BPCHAR));
    +		columnDefs.add(new Pair<String, DataType>("varcharType", DataType.VARCHAR));
    +		columnDefs.add(new Pair<String, DataType>("text", DataType.TEXT));
    +
    +		List<String> output = new ArrayList<String>();
    +
    +		output.add("666,true,x,3.14,3.15,999,777,bpcharType,varcharType,");
    +
    +		super.assertOutput(new Path(System.getProperty("user.dir") + File.separator
    +				+ "src/test/resources/datatypes-test.json"), output);
    +	}
    +
    +	@Test(expected = IllegalStateException.class)
    +	public void testMissingArrayJsonAttribute() throws Exception {
    +
    +		extraParams.add(new Pair<String, String>("IDENTIFIER", "created_at"));
    +
    +		columnDefs.clear();
    +
    +		columnDefs.add(new Pair<String, DataType>("created_at", DataType.TEXT));
    +		// User is not an array! An attempt to access it should throw an exception!
    +		columnDefs.add(new Pair<String, DataType>("user[0]", DataType.TEXT));
    +
    +		List<String> output = new ArrayList<String>();
    +
    +		super.assertOutput(new Path(System.getProperty("user.dir") + File.separator
    +				+ "src/test/resources/tweets-with-missing-text-attribtute.json"), output);
    +	}
    +
    +	@Test
    +	public void testMissingJsonAttribute() throws Exception {
    +
    +		List<String> output = new ArrayList<String>();
    +
    +		extraParams.add(new Pair<String, String>("IDENTIFIER", "created_at"));
    +
    +		// Missing attributes are substituted by an empty field
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547115253761,,SpreadButter,tweetCongress,,");
    +
    +		super.assertOutput(new Path(System.getProperty("user.dir") + File.separator
    +				+ "src/test/resources/tweets-with-missing-text-attribtute.json"), output);
    +	}
    +
    +	@Test
    +	public void testMalformedJsonObject() throws Exception {
    +
    +		List<String> output = new ArrayList<String>();
    +
    +		extraParams.add(new Pair<String, String>("IDENTIFIER", "created_at"));
    +
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547115253761,text1,SpreadButter,tweetCongress,,");
    +		output.add(",,,,,,"); // Expected: malformed json records are transformed into empty rows
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547136233472,text3,NoSecrets_Vagas,,,");
    +
    +		super.assertOutput(new Path(System.getProperty("user.dir") + File.separator
    +				+ "src/test/resources/tweets-broken.json"), output);
    +	}
    +
    +	@Test
    +	public void testSmallTweets() throws Exception {
    +
    +		List<String> output = new ArrayList<String>();
    +
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547115253761,text1,SpreadButter,tweetCongress,,");
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547123646465,text2,patronusdeadly,,,");
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547136233472,text3,NoSecrets_Vagas,,,");
    +		output.add("Fri Jun 07 22:45:03 +0000 2013,343136551322136576,text4,SevenStonesBuoy,,-6.1,50.103");
    +
    +		super.assertOutput(new Path(System.getProperty("user.dir") + File.separator
    +				+ "src/test/resources/tweets-small.json"), output);
    +	}
    +
    +	@Test
    +	public void testTweetsWithNull() throws Exception {
    +
    +		List<String> output = new ArrayList<String>();
    +
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,,text1,SpreadButter,tweetCongress,,");
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,,text2,patronusdeadly,,,");
    +
    +		super.assertOutput(new Path(System.getProperty("user.dir") + File.separator
    +				+ "src/test/resources/null-tweets.json"), output);
    +	}
    +
    +	@Test
    +	public void testSmallTweetsWithDelete() throws Exception {
    --- End diff --
    
    what is being tested here?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by tzolov <gi...@git.apache.org>.
Github user tzolov commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51496267
  
    --- Diff: pxf/pxf-json/src/main/java/org/apache/hawq/pxf/plugins/json/JsonAccessor.java ---
    @@ -0,0 +1,66 @@
    +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 org.apache.commons.lang3.StringUtils;
    +import org.apache.hadoop.mapred.FileSplit;
    +import org.apache.hadoop.mapred.InputSplit;
    +import org.apache.hadoop.mapred.JobConf;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +import org.apache.hawq.pxf.plugins.hdfs.HdfsSplittableDataAccessor;
    +
    +/**
    + * This JSON accessor for PXF will read JSON data and pass it to a {@link JsonResolver}.
    + * 
    + * This accessor supports a single JSON record per line, or a more "pretty print" format.
    + */
    +public class JsonAccessor extends HdfsSplittableDataAccessor {
    +
    +	public static final String IDENTIFIER_PARAM = "IDENTIFIER";
    +	public static final String ONERECORDPERLINE_PARAM = "ONERECORDPERLINE";
    +
    +	private String identifier = "";
    +	private boolean oneRecordPerLine = true;
    +
    +	public JsonAccessor(InputData inputData) throws Exception {
    +		super(inputData, new JsonInputFormat());
    +
    +		if (!StringUtils.isEmpty(inputData.getUserProperty(IDENTIFIER_PARAM))) {
    +			identifier = inputData.getUserProperty(IDENTIFIER_PARAM);
    --- End diff --
    
    JsonAccessor supports two modes: 
    1. json object per line -> LineRecordReader
    2. multiline json objects that can span beyond split's boundries. -> JsonRecordReader + JsonStreamReader
    Second mode (used to be called JSON-PP e.g. Pretty Print) requires an json element name (e.g. identifier) to be set to identify the boundries of a json record. 
    I've removed the JSON-PP instead the Json profile without IDENTIFIER set will default to mode 1 (e.g. json per line) and if the identifier paramter is set switches to mode 2 (multiline json records named by the identifer param).
    Will update the documentation to reflect those changes. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by GodenYao <gi...@git.apache.org>.
Github user GodenYao commented on the pull request:

    https://github.com/apache/incubator-hawq/pull/302#issuecomment-185330241
  
    it's pretty much ready - Christian still needs to run automation test (this is still proprietary as we haven't open source automation yet), he's in business trip this week and will pick it up after he returns next week.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by adamjshook <gi...@git.apache.org>.
Github user adamjshook commented on the pull request:

    https://github.com/apache/incubator-hawq/pull/302#issuecomment-178085903
  
    Good to hear from you as well Christian :)
    
    I think you're right on the `identifier `as the means to have an end-tag and new-tag.  It's slowly coming back.  The `StreamReader `seeks until it finds the identifier, then flushes the `StringBuilder` and appends characters until the final end brace is met, accounting for any nested begin brace/end brace combinations.  The big problem that happens is if the identifier is embedded as a string inside a JSON record, it would create an incomplete record.  If your identifier was `record` in the below example where the split is the horizontal line, bad stuff would happen.  This is why the expectation was one file = one split.
    
    ```
    record : {
    --------------------------
        text : "this is a record"
    }
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by hornn <gi...@git.apache.org>.
Github user hornn commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51205466
  
    --- Diff: pxf/build.gradle ---
    @@ -64,7 +64,7 @@ subprojects { subProject ->
                 "-Xlint:finally", "-Xlint:overrides", "-Xlint:path", "-Xlint:processing", "-Xlint:static", "-Xlint:try",
                 "-Xlint:-fallthrough", "-Xlint:-deprecation", "-Xlint:-unchecked", "-Xlint:-options"
         ]
    -
    +		
    --- End diff --
    
    remove white spaces?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by hornn <gi...@git.apache.org>.
Github user hornn commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51835076
  
    --- Diff: pxf/pxf-json/src/test/java/org/apache/hawq/pxf/plugins/json/JsonExtensionTest.java ---
    @@ -0,0 +1,288 @@
    +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.File;
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import org.apache.hadoop.fs.Path;
    +import org.apache.hawq.pxf.api.Fragmenter;
    +import org.apache.hawq.pxf.api.ReadAccessor;
    +import org.apache.hawq.pxf.api.ReadResolver;
    +import org.apache.hawq.pxf.api.io.DataType;
    +import org.apache.hawq.pxf.plugins.hdfs.HdfsDataFragmenter;
    +import org.apache.hawq.pxf.plugins.json.JsonAccessor;
    +import org.apache.hawq.pxf.plugins.json.JsonResolver;
    +import org.junit.After;
    +import org.junit.Before;
    +import org.junit.Test;
    +
    +public class JsonExtensionTest extends PxfUnit {
    +
    +	private List<Pair<String, DataType>> columnDefs = null;
    +	private List<Pair<String, String>> extraParams = new ArrayList<Pair<String, String>>();
    +
    +	@Before
    +	public void before() {
    +
    +		columnDefs = new ArrayList<Pair<String, DataType>>();
    +
    +		columnDefs.add(new Pair<String, DataType>("created_at", DataType.TEXT));
    +		columnDefs.add(new Pair<String, DataType>("id", DataType.BIGINT));
    +		columnDefs.add(new Pair<String, DataType>("text", DataType.TEXT));
    +		columnDefs.add(new Pair<String, DataType>("user.screen_name", DataType.TEXT));
    +		columnDefs.add(new Pair<String, DataType>("entities.hashtags[0]", DataType.TEXT));
    +		columnDefs.add(new Pair<String, DataType>("coordinates.coordinates[0]", DataType.FLOAT8));
    +		columnDefs.add(new Pair<String, DataType>("coordinates.coordinates[1]", DataType.FLOAT8));
    +	}
    +
    +	@After
    +	public void cleanup() throws Exception {
    +		columnDefs.clear();
    +		extraParams.clear();
    +	}
    +
    +	@Test
    +	public void testCompressedMultilineJsonFile() throws Exception {
    +
    +		extraParams.add(new Pair<String, String>("IDENTIFIER", "created_at"));
    +
    +		List<String> output = new ArrayList<String>();
    +
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547115253761,text1,SpreadButter,tweetCongress,,");
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547123646465,text2,patronusdeadly,,,");
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547136233472,text3,NoSecrets_Vagas,,,");
    +
    +		super.assertOutput(new Path(System.getProperty("user.dir") + File.separator
    +				+ "src/test/resources/tweets.tar.gz"), output);
    +	}
    +
    +	@Test
    +	public void testMaxRecordLength() throws Exception {
    +
    +		// variable-size-objects.json contains 3 json objects but only 2 of them fit in the 27 byte length limitation
    +
    +		extraParams.add(new Pair<String, String>("IDENTIFIER", "key666"));
    +		extraParams.add(new Pair<String, String>("MAXLENGTH", "27"));
    +
    +		columnDefs.clear();
    +		columnDefs.add(new Pair<String, DataType>("key666", DataType.TEXT));
    +
    +		List<String> output = new ArrayList<String>();
    +
    +		output.add("small object1");
    +		// skip the large object2 XXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX
    +		output.add("small object3");
    +
    +		super.assertOutput(new Path(System.getProperty("user.dir") + File.separator
    +				+ "src/test/resources/variable-size-objects.json"), output);
    +	}
    +
    +	@Test
    +	public void testDataTypes() throws Exception {
    +
    +		// TDOO: The BYTEA type is not tested!!! Current oneField.val = val.asText().getBytes(); convention is
    +		// disputable!
    +
    +		extraParams.clear();
    +		extraParams.add(new Pair<String, String>("IDENTIFIER", "bintType"));
    +
    +		columnDefs.clear();
    +		columnDefs.add(new Pair<String, DataType>("bintType", DataType.BIGINT));
    +		columnDefs.add(new Pair<String, DataType>("booleanType", DataType.BOOLEAN));
    +		columnDefs.add(new Pair<String, DataType>("charType", DataType.CHAR));
    +		// columnDefs.add(new Pair<String, DataType>("byteaType", DataType.BYTEA));
    +		columnDefs.add(new Pair<String, DataType>("float8Type", DataType.FLOAT8));
    +		columnDefs.add(new Pair<String, DataType>("realType", DataType.REAL));
    +		columnDefs.add(new Pair<String, DataType>("integerType", DataType.INTEGER));
    +		columnDefs.add(new Pair<String, DataType>("smallintType", DataType.SMALLINT));
    +		columnDefs.add(new Pair<String, DataType>("bpcharType", DataType.BPCHAR));
    +		columnDefs.add(new Pair<String, DataType>("varcharType", DataType.VARCHAR));
    +		columnDefs.add(new Pair<String, DataType>("text", DataType.TEXT));
    --- End diff --
    
    perhaps add a comment saying this field doesn't exist in the file?
    btw, does the order have to match between the table definition (columnDefs) and the json file itself? In other words, if the order between bintType and booleanType will be replaced here, will it still parse the file correctly?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by tzolov <gi...@git.apache.org>.
Github user tzolov commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51492977
  
    --- Diff: pxf/pxf-json/src/test/java/org/apache/pxf/hawq/plugins/json/JsonStreamReaderTest.java ---
    @@ -0,0 +1,53 @@
    +package org.apache.pxf.hawq.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.FileInputStream;
    +import java.io.FileNotFoundException;
    +import java.io.IOException;
    +
    +import org.apache.hawq.pxf.plugins.json.JsonStreamReader;
    +import org.junit.Assert;
    +import org.junit.Before;
    +import org.junit.Test;
    +
    +public class JsonStreamReaderTest {
    +
    +	private JsonStreamReader rdr = null;
    +
    +	private static final String FILE = "src/test/resources/sample.json";
    +
    +	@Before
    +	public void setup() throws FileNotFoundException {
    +		rdr = new JsonStreamReader("menuitem", new FileInputStream(FILE));
    --- End diff --
    
    done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by hornn <gi...@git.apache.org>.
Github user hornn commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51317339
  
    --- Diff: pxf/pxf-json/src/main/java/org/apache/hawq/pxf/plugins/json/JsonStreamReader.java ---
    @@ -0,0 +1,89 @@
    +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.BufferedReader;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.io.InputStreamReader;
    +
    +public class JsonStreamReader extends BufferedReader {
    +
    +	private StringBuilder bldr = new StringBuilder();
    +	private String identifier = null;
    +	private long bytesRead = 0;
    +
    +	public JsonStreamReader(String identifier, InputStream strm) {
    --- End diff --
    
    please add javadoc explaining what is the role of the identifier


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by tzolov <gi...@git.apache.org>.
Github user tzolov commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51513471
  
    --- Diff: pxf/pxf-json/src/main/java/org/apache/hawq/pxf/plugins/json/JsonUtil.java ---
    @@ -0,0 +1,55 @@
    +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 org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.codehaus.jackson.JsonFactory;
    +import org.codehaus.jackson.JsonNode;
    +import org.codehaus.jackson.map.ObjectMapper;
    +
    +/**
    + * Utility that helps to convert JSON text into {@link JsonNode} object
    + *
    + */
    +public class JsonUtil {
    --- End diff --
    
    i've removed the JsonUtil as the parsing logic is now part of the resolver (see below)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by tzolov <gi...@git.apache.org>.
Github user tzolov commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51494057
  
    --- Diff: pxf/build.gradle ---
    @@ -286,6 +286,29 @@ project('pxf-hive') {
         }
     }
     
    +project('pxf-json') {
    +	dependencies {
    +		compile(project(':pxf-hdfs'))
    +		compile "org.apache.commons:commons-lang3:3.0"
    +
    +		testCompile 'pl.pragmatists:JUnitParams:1.0.2'
    +		configurations {
    +			// Remove hive-exec from unit tests as it causes VerifyError
    +			testRuntime.exclude module: 'hive-exec'
    --- End diff --
    
    done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by tzolov <gi...@git.apache.org>.
Github user tzolov commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51862766
  
    --- Diff: pxf/pxf-json/src/test/java/org/apache/hawq/pxf/plugins/json/JsonExtensionTest.java ---
    @@ -0,0 +1,288 @@
    +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.File;
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import org.apache.hadoop.fs.Path;
    +import org.apache.hawq.pxf.api.Fragmenter;
    +import org.apache.hawq.pxf.api.ReadAccessor;
    +import org.apache.hawq.pxf.api.ReadResolver;
    +import org.apache.hawq.pxf.api.io.DataType;
    +import org.apache.hawq.pxf.plugins.hdfs.HdfsDataFragmenter;
    +import org.apache.hawq.pxf.plugins.json.JsonAccessor;
    +import org.apache.hawq.pxf.plugins.json.JsonResolver;
    +import org.junit.After;
    +import org.junit.Before;
    +import org.junit.Test;
    +
    +public class JsonExtensionTest extends PxfUnit {
    +
    +	private List<Pair<String, DataType>> columnDefs = null;
    +	private List<Pair<String, String>> extraParams = new ArrayList<Pair<String, String>>();
    +
    +	@Before
    +	public void before() {
    +
    +		columnDefs = new ArrayList<Pair<String, DataType>>();
    +
    +		columnDefs.add(new Pair<String, DataType>("created_at", DataType.TEXT));
    +		columnDefs.add(new Pair<String, DataType>("id", DataType.BIGINT));
    +		columnDefs.add(new Pair<String, DataType>("text", DataType.TEXT));
    +		columnDefs.add(new Pair<String, DataType>("user.screen_name", DataType.TEXT));
    +		columnDefs.add(new Pair<String, DataType>("entities.hashtags[0]", DataType.TEXT));
    +		columnDefs.add(new Pair<String, DataType>("coordinates.coordinates[0]", DataType.FLOAT8));
    +		columnDefs.add(new Pair<String, DataType>("coordinates.coordinates[1]", DataType.FLOAT8));
    +	}
    +
    +	@After
    +	public void cleanup() throws Exception {
    +		columnDefs.clear();
    +		extraParams.clear();
    +	}
    +
    +	@Test
    +	public void testCompressedMultilineJsonFile() throws Exception {
    +
    +		extraParams.add(new Pair<String, String>("IDENTIFIER", "created_at"));
    +
    +		List<String> output = new ArrayList<String>();
    +
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547115253761,text1,SpreadButter,tweetCongress,,");
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547123646465,text2,patronusdeadly,,,");
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547136233472,text3,NoSecrets_Vagas,,,");
    +
    +		super.assertOutput(new Path(System.getProperty("user.dir") + File.separator
    +				+ "src/test/resources/tweets.tar.gz"), output);
    +	}
    +
    +	@Test
    +	public void testMaxRecordLength() throws Exception {
    +
    +		// variable-size-objects.json contains 3 json objects but only 2 of them fit in the 27 byte length limitation
    +
    +		extraParams.add(new Pair<String, String>("IDENTIFIER", "key666"));
    +		extraParams.add(new Pair<String, String>("MAXLENGTH", "27"));
    +
    +		columnDefs.clear();
    +		columnDefs.add(new Pair<String, DataType>("key666", DataType.TEXT));
    +
    +		List<String> output = new ArrayList<String>();
    +
    +		output.add("small object1");
    +		// skip the large object2 XXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX
    +		output.add("small object3");
    +
    +		super.assertOutput(new Path(System.getProperty("user.dir") + File.separator
    +				+ "src/test/resources/variable-size-objects.json"), output);
    +	}
    +
    +	@Test
    +	public void testDataTypes() throws Exception {
    +
    +		// TDOO: The BYTEA type is not tested!!! Current oneField.val = val.asText().getBytes(); convention is
    +		// disputable!
    +
    +		extraParams.clear();
    +		extraParams.add(new Pair<String, String>("IDENTIFIER", "bintType"));
    +
    +		columnDefs.clear();
    +		columnDefs.add(new Pair<String, DataType>("bintType", DataType.BIGINT));
    +		columnDefs.add(new Pair<String, DataType>("booleanType", DataType.BOOLEAN));
    +		columnDefs.add(new Pair<String, DataType>("charType", DataType.CHAR));
    +		// columnDefs.add(new Pair<String, DataType>("byteaType", DataType.BYTEA));
    +		columnDefs.add(new Pair<String, DataType>("float8Type", DataType.FLOAT8));
    +		columnDefs.add(new Pair<String, DataType>("realType", DataType.REAL));
    +		columnDefs.add(new Pair<String, DataType>("integerType", DataType.INTEGER));
    +		columnDefs.add(new Pair<String, DataType>("smallintType", DataType.SMALLINT));
    +		columnDefs.add(new Pair<String, DataType>("bpcharType", DataType.BPCHAR));
    +		columnDefs.add(new Pair<String, DataType>("varcharType", DataType.VARCHAR));
    +		columnDefs.add(new Pair<String, DataType>("text", DataType.TEXT));
    --- End diff --
    
    No the order of the json attributes doesn't affect the parsing. I've modified the test (reversed the columnDefs order) to validate this case as well. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by hornn <gi...@git.apache.org>.
Github user hornn commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51505865
  
    --- Diff: pxf/pxf-json/src/main/java/org/apache/hawq/pxf/plugins/json/JsonStreamReader.java ---
    @@ -0,0 +1,121 @@
    +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.BufferedReader;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.io.InputStreamReader;
    +
    +/**
    + * Reads a JSON stream and sequentially extracts all JSON objects identified by the <b>identifier</> parameter. Returns
    + * null when there are no more objects to read.
    + *
    + */
    +public class JsonStreamReader extends BufferedReader {
    +
    +	private static char START_BRACE = '{';
    +	private static char END_BRACE = '}';
    +	private static int EOF = -1;
    +
    +	private StringBuilder bldr = new StringBuilder();
    +	private String identifier = null;
    +	private long bytesRead = 0;
    +
    +	/**
    +	 * @param identifier
    +	 *            JSON object name to extract.
    +	 * @param inputStream
    +	 *            JSON document input stream.
    +	 */
    +	public JsonStreamReader(String identifier, InputStream inputStream) {
    +		super(new InputStreamReader(inputStream));
    +		this.identifier = identifier;
    +	}
    +
    +	/**
    +	 * @return Returns next JSON object identified by the {@link JsonStreamReader#identifier} parameter or Null if no
    +	 *         more object are available.
    --- End diff --
    
    typo objects?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by adamjshook <gi...@git.apache.org>.
Github user adamjshook commented on the pull request:

    https://github.com/apache/incubator-hawq/pull/302#issuecomment-178007711
  
    Actually, scratch that, I don't even think you can comfortably build JSON records that span multiple lines.  Unlike XML, there is no cleanly identifiable "end tag" of a JSON record.  I think the intent was to set the isSplitable to false and I never did.  The preferred method of using this was to have one JSON record per line.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by hornn <gi...@git.apache.org>.
Github user hornn commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51818643
  
    --- Diff: pxf/pxf-json/src/main/java/org/apache/hawq/pxf/plugins/json/ColumnDescriptorCache.java ---
    @@ -0,0 +1,165 @@
    +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.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +
    +import org.apache.hawq.pxf.api.io.DataType;
    +import org.apache.hawq.pxf.api.utilities.ColumnDescriptor;
    +
    +/**
    + * Helper class used to retrieve all column details relevant for the json processing.
    + */
    +public class ColumnDescriptorCache {
    +
    +	private static Pattern ARRAY_PROJECTION_PATTERN = Pattern.compile("(.*)\\[([0-9]+)\\]");
    +	private static int ARRAY_NAME_GROUPID = 1;
    +	private static int ARRAY_INDEX_GROUPID = 2;
    +
    +	private final DataType columnType;
    +	private final String[] projections;
    +	private final String nodeName;
    +	private final int arrayIndex;
    +	private final boolean isArray;
    +
    +	public ColumnDescriptorCache(ColumnDescriptor columnDescriptor) {
    +
    +		this.columnType = DataType.get(columnDescriptor.columnTypeCode());
    +
    +		this.projections = columnDescriptor.columnName().split("\\.");
    +
    +		this.isArray = isArrayIndex(projections);
    --- End diff --
    
    slight improvement - if we encapsulate the whole array logic in a function, we can call `getMatchGroup` once, and use the results to populate this.isArray, this.nodeName and this.arrayIndex. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by tzolov <gi...@git.apache.org>.
Github user tzolov commented on the pull request:

    https://github.com/apache/incubator-hawq/pull/302#issuecomment-179845379
  
    Last commit address most issues except the compression thread-safety and the PxfUnit location. Also it assumes that the BYTEA type handling will be evaluated in a separate task. 
    
    @hornn let me know how i can help creating the automation tests. Is there any doc/instructions?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by tzolov <gi...@git.apache.org>.
Github user tzolov commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51346000
  
    --- Diff: pxf/pxf-service/src/main/resources/pxf-profiles-default.xml ---
    @@ -128,4 +128,26 @@ under the License.
                 <resolver>org.apache.hawq.pxf.plugins.gemfirexd.GemFireXDResolver</resolver>
             </plugins>
         </profile>
    +	<profile>
    +		<name>JSON</name>
    +		<description>A profile for JSON data, one JSON record per line
    +		</description>
    +		<plugins>
    +			<fragmenter>org.apache.hawq.pxf.plugins.hdfs.HdfsDataFragmenter</fragmenter>
    +			<accessor>org.apache.hawq.pxf.plugins.json.JsonAccessor</accessor>
    +			<resolver>org.apache.hawq.pxf.plugins.json.JsonResolver</resolver>
    +		</plugins>
    +	</profile>
    +	<profile>
    +		<name>JSON-PP</name>
    --- End diff --
    
    JSON-PP differs from JSON only by the hard coded property `onerecordperline=false`.  
    I will remove it though in favor of JSON + explicit parameter in the DDL definition.
    About the name shall i change it from `JSON` to `Json`? 



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by tzolov <gi...@git.apache.org>.
Github user tzolov commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51871264
  
    --- Diff: pxf/pxf-json/src/test/java/org/apache/hawq/pxf/plugins/json/PxfUnit.java ---
    @@ -0,0 +1,672 @@
    +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.BufferedReader;
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.InputStreamReader;
    +import java.io.OutputStream;
    +import java.lang.reflect.Constructor;
    +import java.security.InvalidParameterException;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.fs.FileSystem;
    +import org.apache.hadoop.fs.Path;
    +import org.apache.hawq.pxf.api.Fragment;
    +import org.apache.hawq.pxf.api.Fragmenter;
    +import org.apache.hawq.pxf.api.OneField;
    +import org.apache.hawq.pxf.api.OneRow;
    +import org.apache.hawq.pxf.api.ReadAccessor;
    +import org.apache.hawq.pxf.api.ReadResolver;
    +import org.apache.hawq.pxf.api.WriteAccessor;
    +import org.apache.hawq.pxf.api.WriteResolver;
    +import org.apache.hawq.pxf.api.io.DataType;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +import org.apache.hawq.pxf.service.FragmentsResponse;
    --- End diff --
    
    All the credit for the PxfUnit goes to @adamjshook 
    In the pxf-field projects the PxfUnit has its own dedicated project: `pxf-test`. But adding it under the pxf-service might be easiest way to go. Do you want me to move it now or in a follow up task?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by hornn <gi...@git.apache.org>.
Github user hornn commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51504013
  
    --- Diff: pxf/pxf-json/src/test/java/org/apache/pxf/hawq/plugins/json/JsonExtensionTest.java ---
    @@ -0,0 +1,175 @@
    +package org.apache.pxf.hawq.plugins.json;
    --- End diff --
    
    in all tests, the package name is wrong.
    should be org.apache.hawq.pxf....


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by tzolov <gi...@git.apache.org>.
Github user tzolov commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51853931
  
    --- Diff: pxf/pxf-json/src/main/java/org/apache/hawq/pxf/plugins/json/ColumnDescriptorCache.java ---
    @@ -0,0 +1,165 @@
    +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.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +
    +import org.apache.hawq.pxf.api.io.DataType;
    +import org.apache.hawq.pxf.api.utilities.ColumnDescriptor;
    +
    +/**
    + * Helper class used to retrieve all column details relevant for the json processing.
    + */
    +public class ColumnDescriptorCache {
    +
    +	private static Pattern ARRAY_PROJECTION_PATTERN = Pattern.compile("(.*)\\[([0-9]+)\\]");
    +	private static int ARRAY_NAME_GROUPID = 1;
    +	private static int ARRAY_INDEX_GROUPID = 2;
    +
    +	private final DataType columnType;
    +	private final String[] projections;
    +	private final String nodeName;
    +	private final int arrayIndex;
    +	private final boolean isArray;
    +
    +	public ColumnDescriptorCache(ColumnDescriptor columnDescriptor) {
    +
    +		this.columnType = DataType.get(columnDescriptor.columnTypeCode());
    +
    +		this.projections = columnDescriptor.columnName().split("\\.");
    +
    +		this.isArray = isArrayIndex(projections);
    --- End diff --
    
    +1


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by tzolov <gi...@git.apache.org>.
Github user tzolov commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51492471
  
    --- Diff: pxf/pxf-json/src/test/java/org/apache/pxf/hawq/plugins/json/PxfUnit.java ---
    @@ -0,0 +1,666 @@
    +package org.apache.pxf.hawq.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.BufferedReader;
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.InputStreamReader;
    +import java.io.OutputStream;
    +import java.lang.reflect.Constructor;
    +import java.security.InvalidParameterException;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.fs.FileSystem;
    +import org.apache.hadoop.fs.Path;
    +import org.apache.hawq.pxf.api.Fragment;
    +import org.apache.hawq.pxf.api.Fragmenter;
    +import org.apache.hawq.pxf.api.OneField;
    +import org.apache.hawq.pxf.api.OneRow;
    +import org.apache.hawq.pxf.api.ReadAccessor;
    +import org.apache.hawq.pxf.api.ReadResolver;
    +import org.apache.hawq.pxf.api.WriteAccessor;
    +import org.apache.hawq.pxf.api.WriteResolver;
    +import org.apache.hawq.pxf.api.io.DataType;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +import org.apache.hawq.pxf.service.FragmentsResponse;
    +import org.apache.hawq.pxf.service.FragmentsResponseFormatter;
    +import org.apache.hawq.pxf.service.utilities.ProtocolData;
    +import org.codehaus.jackson.JsonFactory;
    +import org.codehaus.jackson.JsonNode;
    +import org.codehaus.jackson.JsonParseException;
    +import org.codehaus.jackson.map.JsonMappingException;
    +import org.codehaus.jackson.map.ObjectMapper;
    +import org.junit.Assert;
    +
    +/**
    + * This abstract class contains a number of helpful utilities in developing a PXF extension for HAWQ. Extend this class
    + * and use the various <code>assert</code> methods to check given input against known output.
    + */
    +public abstract class PxfUnit {
    +
    +	private static JsonFactory factory = new JsonFactory();
    +	private static ObjectMapper mapper = new ObjectMapper(factory);
    +
    +	protected static List<InputData> inputs = null;
    +
    +	/**
    +	 * Uses the given input directory to run through the PXF unit testing framework. Uses the lines in the file for
    +	 * output testing.
    +	 * 
    +	 * @param input
    +	 *            Input records
    +	 * @param expectedOutput
    +	 *            File containing output to check
    +	 * @throws Exception
    +	 */
    +	public void assertOutput(Path input, Path expectedOutput) throws Exception {
    +
    +		BufferedReader rdr = new BufferedReader(new InputStreamReader(FileSystem.get(new Configuration()).open(
    +				expectedOutput)));
    +
    +		List<String> outputLines = new ArrayList<String>();
    +
    +		String line;
    +		while ((line = rdr.readLine()) != null) {
    +			outputLines.add(line);
    +		}
    +
    +		assertOutput(input, outputLines);
    +	}
    +
    +	/**
    +	 * Uses the given input directory to run through the PXF unit testing framework. Uses the lines in the given
    +	 * parameter for output testing.
    +	 * 
    +	 * @param input
    +	 *            Input records
    +	 * @param expectedOutput
    +	 *            File containing output to check
    +	 * @throws Exception
    +	 */
    +	public void assertOutput(Path input, List<String> expectedOutput) throws Exception {
    +
    +		setup(input);
    +		List<String> actualOutput = new ArrayList<String>();
    +		for (InputData data : inputs) {
    +			ReadAccessor accessor = getReadAccessor(data);
    +			ReadResolver resolver = getReadResolver(data);
    +
    +			actualOutput.addAll(getAllOutput(accessor, resolver));
    +		}
    +
    +		Assert.assertFalse("Output did not match expected output", compareOutput(expectedOutput, actualOutput));
    +	}
    +
    +	/**
    +	 * Uses the given input directory to run through the PXF unit testing framework. Uses the lines in the given
    +	 * parameter for output testing.<br>
    +	 * <br>
    +	 * Ignores order of records.
    +	 * 
    +	 * @param input
    +	 *            Input records
    +	 * @param expectedOutput
    +	 *            File containing output to check
    +	 * @throws Exception
    +	 */
    +	public void assertUnorderedOutput(Path input, Path expectedOutput) throws Exception {
    +		BufferedReader rdr = new BufferedReader(new InputStreamReader(FileSystem.get(new Configuration()).open(
    +				expectedOutput)));
    +
    +		List<String> outputLines = new ArrayList<String>();
    +
    +		String line;
    +		while ((line = rdr.readLine()) != null) {
    +			outputLines.add(line);
    +		}
    +
    +		assertUnorderedOutput(input, outputLines);
    +	}
    +
    +	/**
    +	 * Uses the given input directory to run through the PXF unit testing framework. Uses the lines in the file for
    +	 * output testing.<br>
    +	 * <br>
    +	 * Ignores order of records.
    +	 * 
    +	 * @param input
    +	 *            Input records
    +	 * @param expectedOutput
    +	 *            File containing output to check
    +	 * @throws Exception
    +	 */
    +	public void assertUnorderedOutput(Path input, List<String> expectedOutput) throws Exception {
    +
    +		setup(input);
    +
    +		List<String> actualOutput = new ArrayList<String>();
    +		for (InputData data : inputs) {
    +			ReadAccessor accessor = getReadAccessor(data);
    +			ReadResolver resolver = getReadResolver(data);
    +
    +			actualOutput.addAll(getAllOutput(accessor, resolver));
    +		}
    +
    +		Assert.assertFalse("Output did not match expected output", compareUnorderedOutput(expectedOutput, actualOutput));
    +	}
    +
    +	/**
    +	 * Writes the output to the given output stream. Comma delimiter.
    +	 * 
    +	 * @param input
    +	 *            The input file
    +	 * @param output
    +	 *            The output stream
    +	 * @throws Exception
    +	 */
    +	public void writeOutput(Path input, OutputStream output) throws Exception {
    +
    +		setup(input);
    +
    +		for (InputData data : inputs) {
    +			ReadAccessor accessor = getReadAccessor(data);
    +			ReadResolver resolver = getReadResolver(data);
    +
    +			for (String line : getAllOutput(accessor, resolver)) {
    +				output.write((line + "\n").getBytes());
    +			}
    +		}
    +
    +		output.flush();
    +	}
    +
    +	/**
    +	 * Get the class of the implementation of Fragmenter to be tested.
    +	 * 
    +	 * @return The class
    +	 */
    +	public Class<? extends Fragmenter> getFragmenterClass() {
    +		return null;
    +	}
    +
    +	/**
    +	 * Get the class of the implementation of ReadAccessor to be tested.
    +	 * 
    +	 * @return The class
    +	 */
    +	public Class<? extends ReadAccessor> getReadAccessorClass() {
    +		return null;
    +	}
    +
    +	/**
    +	 * Get the class of the implementation of WriteAccessor to be tested.
    +	 * 
    +	 * @return The class
    +	 */
    +	public Class<? extends WriteAccessor> getWriteAccessorClass() {
    +		return null;
    +	}
    +
    +	/**
    +	 * Get the class of the implementation of Resolver to be tested.
    +	 * 
    +	 * @return The class
    +	 */
    +	public Class<? extends ReadResolver> getReadResolverClass() {
    +		return null;
    +	}
    +
    +	/**
    +	 * Get the class of the implementation of WriteResolver to be tested.
    +	 * 
    +	 * @return The class
    +	 */
    +	public Class<? extends WriteResolver> getWriteResolverClass() {
    +		return null;
    +	}
    +
    +	/**
    +	 * Get any extra parameters that are meant to be specified for the "pxf" protocol. Note that "X-GP-" is prepended to
    +	 * each parameter name.
    +	 * 
    +	 * @return Any extra parameters or null if none.
    +	 */
    +	public List<Pair<String, String>> getExtraParams() {
    +		return null;
    +	}
    +
    +	/**
    +	 * Gets the column definition names and data types. Types are DataType objects
    +	 * 
    +	 * @return A list of column definition name value pairs. Cannot be null.
    +	 */
    +	public abstract List<Pair<String, DataType>> getColumnDefinitions();
    +
    +	protected InputData getInputDataForWritableTable() {
    +		return getInputDataForWritableTable(null);
    +	}
    +
    +	protected InputData getInputDataForWritableTable(Path input) {
    +
    +		if (getWriteAccessorClass() == null) {
    +			throw new IllegalArgumentException(
    +					"getWriteAccessorClass() must be overwritten to return a non-null object");
    +		}
    +
    +		if (getWriteResolverClass() == null) {
    +			throw new IllegalArgumentException(
    +					"getWriteResolverClass() must be overwritten to return a non-null object");
    +		}
    +
    +		Map<String, String> paramsMap = new HashMap<String, String>();
    +
    +		paramsMap.put("X-GP-ALIGNMENT", "what");
    +		paramsMap.put("X-GP-SEGMENT-ID", "1");
    +		paramsMap.put("X-GP-HAS-FILTER", "0");
    +		paramsMap.put("X-GP-SEGMENT-COUNT", "1");
    +
    +		paramsMap.put("X-GP-FORMAT", "GPDBWritable");
    +		paramsMap.put("X-GP-URL-HOST", "localhost");
    +		paramsMap.put("X-GP-URL-PORT", "50070");
    +
    +		if (input == null) {
    +			paramsMap.put("X-GP-DATA-DIR", "/dummydata");
    +		}
    +
    +		List<Pair<String, DataType>> params = getColumnDefinitions();
    +		paramsMap.put("X-GP-ATTRS", Integer.toString(params.size()));
    +		for (int i = 0; i < params.size(); ++i) {
    +			paramsMap.put("X-GP-ATTR-NAME" + i, params.get(i).first);
    +			paramsMap.put("X-GP-ATTR-TYPENAME" + i, params.get(i).second.name());
    +			paramsMap.put("X-GP-ATTR-TYPECODE" + i, Integer.toString(params.get(i).second.getOID()));
    +		}
    +
    +		paramsMap.put("X-GP-ACCESSOR", getWriteAccessorClass().getName());
    +		paramsMap.put("X-GP-RESOLVER", getWriteResolverClass().getName());
    +
    +		if (getExtraParams() != null) {
    +			for (Pair<String, String> param : getExtraParams()) {
    +				paramsMap.put("X-GP-" + param.first, param.second);
    +			}
    +		}
    +
    +		return new ProtocolData(paramsMap);
    +	}
    +
    +	/**
    +	 * Set all necessary parameters for GPXF framework to function. Uses the given path as a single input split.
    +	 * 
    +	 * @param input
    +	 *            The input path, relative or absolute.
    +	 * @throws Exception
    +	 */
    +	protected void setup(Path input) throws Exception {
    +
    +		if (getFragmenterClass() == null) {
    +			throw new IllegalArgumentException("getFragmenterClass() must be overwritten to return a non-null object");
    +		}
    +
    +		if (getReadAccessorClass() == null) {
    +			throw new IllegalArgumentException("getReadAccessorClass() must be overwritten to return a non-null object");
    +		}
    +
    +		if (getReadResolverClass() == null) {
    +			throw new IllegalArgumentException("getReadResolverClass() must be overwritten to return a non-null object");
    +		}
    +
    +		Map<String, String> paramsMap = new HashMap<String, String>();
    +
    +		// 2.1.0 Properties
    +		// HDMetaData parameters
    +		paramsMap.put("X-GP-ALIGNMENT", "what");
    +		paramsMap.put("X-GP-SEGMENT-ID", "1");
    +		paramsMap.put("X-GP-HAS-FILTER", "0");
    +		paramsMap.put("X-GP-SEGMENT-COUNT", "1");
    +		paramsMap.put("X-GP-FRAGMENTER", getFragmenterClass().getName());
    +		paramsMap.put("X-GP-FORMAT", "GPDBWritable");
    +		paramsMap.put("X-GP-URL-HOST", "localhost");
    +		paramsMap.put("X-GP-URL-PORT", "50070");
    +
    +		paramsMap.put("X-GP-DATA-DIR", input.toString());
    +
    +		List<Pair<String, DataType>> params = getColumnDefinitions();
    +		paramsMap.put("X-GP-ATTRS", Integer.toString(params.size()));
    +		for (int i = 0; i < params.size(); ++i) {
    +			paramsMap.put("X-GP-ATTR-NAME" + i, params.get(i).first);
    +			paramsMap.put("X-GP-ATTR-TYPENAME" + i, params.get(i).second.name());
    +			paramsMap.put("X-GP-ATTR-TYPECODE" + i, Integer.toString(params.get(i).second.getOID()));
    +		}
    +
    +		// HDFSMetaData properties
    +		paramsMap.put("X-GP-ACCESSOR", getReadAccessorClass().getName());
    +		paramsMap.put("X-GP-RESOLVER", getReadResolverClass().getName());
    +
    +		if (getExtraParams() != null) {
    +			for (Pair<String, String> param : getExtraParams()) {
    +				paramsMap.put("X-GP-" + param.first, param.second);
    +			}
    +		}
    +
    +		LocalInputData fragmentInputData = new LocalInputData(paramsMap);
    +
    +		List<Fragment> fragments = getFragmenter(fragmentInputData).getFragments();
    +
    +		FragmentsResponse fragmentsResponse = FragmentsResponseFormatter.formatResponse(fragments, input.toString());
    +
    +		ByteArrayOutputStream baos = new ByteArrayOutputStream();
    +		fragmentsResponse.write(baos);
    +
    +		String jsonOutput = baos.toString();
    +
    +		inputs = new ArrayList<InputData>();
    +
    +		JsonNode node = decodeLineToJsonNode(jsonOutput);
    +
    +		JsonNode fragmentsArray = node.get("PXFFragments");
    +		int i = 0;
    +		Iterator<JsonNode> iter = fragmentsArray.getElements();
    +		while (iter.hasNext()) {
    +			JsonNode fragNode = iter.next();
    +			String sourceData = fragNode.get("sourceName").getTextValue();
    +			if (!sourceData.startsWith("/")) {
    +				sourceData = "/" + sourceData;
    +			}
    +			paramsMap.put("X-GP-DATA-DIR", sourceData);
    +			paramsMap.put("X-GP-FRAGMENT-METADATA", fragNode.get("metadata").getTextValue());
    +			paramsMap.put("X-GP-DATA-FRAGMENT", Integer.toString(i++));
    +			inputs.add(new LocalInputData(paramsMap));
    +		}
    +	}
    +
    +	private JsonNode decodeLineToJsonNode(String line) {
    --- End diff --
    
    The PxfUnit should go in its own standalone project and be used only for testing. So better tackle any code duplications in PxfUnit once it is factored out from the pxf-json. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by hornn <gi...@git.apache.org>.
Github user hornn commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51311755
  
    --- 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
    --- End diff --
    
    unfinished sentence?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by tzolov <gi...@git.apache.org>.
Github user tzolov commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51346673
  
    --- Diff: pxf/pxf-json/src/main/java/org/apache/hawq/pxf/plugins/json/JsonInputFormat.java ---
    @@ -0,0 +1,226 @@
    +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.BufferedInputStream;
    +import java.io.IOException;
    +import java.security.InvalidParameterException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.fs.FSDataInputStream;
    +import org.apache.hadoop.fs.FileSystem;
    +import org.apache.hadoop.fs.Path;
    +import org.apache.hadoop.io.LongWritable;
    +import org.apache.hadoop.io.NullWritable;
    +import org.apache.hadoop.io.Text;
    +import org.apache.hadoop.mapred.FileInputFormat;
    +import org.apache.hadoop.mapred.FileSplit;
    +import org.apache.hadoop.mapred.InputSplit;
    +import org.apache.hadoop.mapred.JobConf;
    +import org.apache.hadoop.mapred.LineRecordReader;
    +import org.apache.hadoop.mapred.RecordReader;
    +import org.apache.hadoop.mapred.Reporter;
    +import org.apache.log4j.Logger;
    +import org.codehaus.jackson.JsonFactory;
    +import org.codehaus.jackson.JsonNode;
    +import org.codehaus.jackson.JsonParseException;
    +import org.codehaus.jackson.map.JsonMappingException;
    +import org.codehaus.jackson.map.ObjectMapper;
    +
    +public class JsonInputFormat extends FileInputFormat<Text, NullWritable> {
    +
    +	private static JsonFactory factory = new JsonFactory();
    +	private static ObjectMapper mapper = new ObjectMapper(factory);
    +
    +	public static final String ONE_RECORD_PER_LINE = "json.input.format.one.record.per.line";
    +	public static final String RECORD_IDENTIFIER = "json.input.format.record.identifier";
    +
    +	@Override
    +	public RecordReader<Text, NullWritable> getRecordReader(InputSplit split, JobConf conf, Reporter reporter)
    +			throws IOException {
    +
    +		if (conf.getBoolean(ONE_RECORD_PER_LINE, false)) {
    +
    +			return new SimpleJsonRecordReader(conf, (FileSplit) split);
    +		} else {
    +			return new JsonRecordReader(conf, (FileSplit) split);
    +		}
    +	}
    +
    +	public static class SimpleJsonRecordReader implements RecordReader<Text, NullWritable> {
    +
    +		private LineRecordReader rdr = null;
    +		private LongWritable key = new LongWritable();
    +		private Text value = new Text();
    +
    +		public SimpleJsonRecordReader(Configuration conf, FileSplit split) throws IOException {
    +			rdr = new LineRecordReader(conf, split);
    +		}
    +
    +		@Override
    +		public void close() throws IOException {
    +			rdr.close();
    +		}
    +
    +		@Override
    +		public Text createKey() {
    +			return value;
    +		}
    +
    +		@Override
    +		public NullWritable createValue() {
    +			return NullWritable.get();
    +		}
    +
    +		@Override
    +		public long getPos() throws IOException {
    +			return rdr.getPos();
    +		}
    +
    +		@Override
    +		public boolean next(Text key, NullWritable value) throws IOException {
    +			if (rdr.next(this.key, this.value)) {
    +				key.set(this.value);
    +				return true;
    +			} else {
    +				return false;
    +			}
    +		}
    +
    +		@Override
    +		public float getProgress() throws IOException {
    +			return rdr.getProgress();
    +		}
    +	}
    +
    +	public static class JsonRecordReader implements RecordReader<Text, NullWritable> {
    +
    +		private Logger LOG = Logger.getLogger(JsonRecordReader.class);
    +
    +		private JsonStreamReader rdr = null;
    +		private long start = 0, end = 0;
    +		private float toRead = 0;
    +		private String identifier = null;
    +		private Logger log = Logger.getLogger(JsonRecordReader.class);
    --- End diff --
    
    Will unify them under the apache commons Log.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by hornn <gi...@git.apache.org>.
Github user hornn commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51835642
  
    --- Diff: pxf/pxf-json/src/test/java/org/apache/hawq/pxf/plugins/json/JsonExtensionTest.java ---
    @@ -0,0 +1,288 @@
    +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.File;
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import org.apache.hadoop.fs.Path;
    +import org.apache.hawq.pxf.api.Fragmenter;
    +import org.apache.hawq.pxf.api.ReadAccessor;
    +import org.apache.hawq.pxf.api.ReadResolver;
    +import org.apache.hawq.pxf.api.io.DataType;
    +import org.apache.hawq.pxf.plugins.hdfs.HdfsDataFragmenter;
    +import org.apache.hawq.pxf.plugins.json.JsonAccessor;
    +import org.apache.hawq.pxf.plugins.json.JsonResolver;
    +import org.junit.After;
    +import org.junit.Before;
    +import org.junit.Test;
    +
    +public class JsonExtensionTest extends PxfUnit {
    +
    +	private List<Pair<String, DataType>> columnDefs = null;
    +	private List<Pair<String, String>> extraParams = new ArrayList<Pair<String, String>>();
    +
    +	@Before
    +	public void before() {
    +
    +		columnDefs = new ArrayList<Pair<String, DataType>>();
    +
    +		columnDefs.add(new Pair<String, DataType>("created_at", DataType.TEXT));
    +		columnDefs.add(new Pair<String, DataType>("id", DataType.BIGINT));
    +		columnDefs.add(new Pair<String, DataType>("text", DataType.TEXT));
    +		columnDefs.add(new Pair<String, DataType>("user.screen_name", DataType.TEXT));
    +		columnDefs.add(new Pair<String, DataType>("entities.hashtags[0]", DataType.TEXT));
    +		columnDefs.add(new Pair<String, DataType>("coordinates.coordinates[0]", DataType.FLOAT8));
    +		columnDefs.add(new Pair<String, DataType>("coordinates.coordinates[1]", DataType.FLOAT8));
    +	}
    +
    +	@After
    +	public void cleanup() throws Exception {
    +		columnDefs.clear();
    +		extraParams.clear();
    +	}
    +
    +	@Test
    +	public void testCompressedMultilineJsonFile() throws Exception {
    +
    +		extraParams.add(new Pair<String, String>("IDENTIFIER", "created_at"));
    +
    +		List<String> output = new ArrayList<String>();
    +
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547115253761,text1,SpreadButter,tweetCongress,,");
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547123646465,text2,patronusdeadly,,,");
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547136233472,text3,NoSecrets_Vagas,,,");
    +
    +		super.assertOutput(new Path(System.getProperty("user.dir") + File.separator
    +				+ "src/test/resources/tweets.tar.gz"), output);
    +	}
    +
    +	@Test
    +	public void testMaxRecordLength() throws Exception {
    +
    +		// variable-size-objects.json contains 3 json objects but only 2 of them fit in the 27 byte length limitation
    +
    +		extraParams.add(new Pair<String, String>("IDENTIFIER", "key666"));
    +		extraParams.add(new Pair<String, String>("MAXLENGTH", "27"));
    +
    +		columnDefs.clear();
    +		columnDefs.add(new Pair<String, DataType>("key666", DataType.TEXT));
    +
    +		List<String> output = new ArrayList<String>();
    +
    +		output.add("small object1");
    +		// skip the large object2 XXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX
    +		output.add("small object3");
    +
    +		super.assertOutput(new Path(System.getProperty("user.dir") + File.separator
    +				+ "src/test/resources/variable-size-objects.json"), output);
    +	}
    +
    +	@Test
    +	public void testDataTypes() throws Exception {
    +
    +		// TDOO: The BYTEA type is not tested!!! Current oneField.val = val.asText().getBytes(); convention is
    +		// disputable!
    +
    +		extraParams.clear();
    +		extraParams.add(new Pair<String, String>("IDENTIFIER", "bintType"));
    +
    +		columnDefs.clear();
    +		columnDefs.add(new Pair<String, DataType>("bintType", DataType.BIGINT));
    +		columnDefs.add(new Pair<String, DataType>("booleanType", DataType.BOOLEAN));
    +		columnDefs.add(new Pair<String, DataType>("charType", DataType.CHAR));
    +		// columnDefs.add(new Pair<String, DataType>("byteaType", DataType.BYTEA));
    +		columnDefs.add(new Pair<String, DataType>("float8Type", DataType.FLOAT8));
    +		columnDefs.add(new Pair<String, DataType>("realType", DataType.REAL));
    +		columnDefs.add(new Pair<String, DataType>("integerType", DataType.INTEGER));
    +		columnDefs.add(new Pair<String, DataType>("smallintType", DataType.SMALLINT));
    +		columnDefs.add(new Pair<String, DataType>("bpcharType", DataType.BPCHAR));
    +		columnDefs.add(new Pair<String, DataType>("varcharType", DataType.VARCHAR));
    +		columnDefs.add(new Pair<String, DataType>("text", DataType.TEXT));
    +
    +		List<String> output = new ArrayList<String>();
    +
    +		output.add("666,true,x,3.14,3.15,999,777,bpcharType,varcharType,");
    +
    +		super.assertOutput(new Path(System.getProperty("user.dir") + File.separator
    +				+ "src/test/resources/datatypes-test.json"), output);
    +	}
    +
    +	@Test(expected = IllegalStateException.class)
    +	public void testMissingArrayJsonAttribute() throws Exception {
    +
    +		extraParams.add(new Pair<String, String>("IDENTIFIER", "created_at"));
    +
    +		columnDefs.clear();
    +
    +		columnDefs.add(new Pair<String, DataType>("created_at", DataType.TEXT));
    +		// User is not an array! An attempt to access it should throw an exception!
    +		columnDefs.add(new Pair<String, DataType>("user[0]", DataType.TEXT));
    +
    +		List<String> output = new ArrayList<String>();
    +
    +		super.assertOutput(new Path(System.getProperty("user.dir") + File.separator
    +				+ "src/test/resources/tweets-with-missing-text-attribtute.json"), output);
    +	}
    +
    +	@Test
    +	public void testMissingJsonAttribute() throws Exception {
    +
    +		List<String> output = new ArrayList<String>();
    +
    +		extraParams.add(new Pair<String, String>("IDENTIFIER", "created_at"));
    +
    +		// Missing attributes are substituted by an empty field
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547115253761,,SpreadButter,tweetCongress,,");
    +
    +		super.assertOutput(new Path(System.getProperty("user.dir") + File.separator
    +				+ "src/test/resources/tweets-with-missing-text-attribtute.json"), output);
    +	}
    +
    +	@Test
    +	public void testMalformedJsonObject() throws Exception {
    +
    +		List<String> output = new ArrayList<String>();
    +
    +		extraParams.add(new Pair<String, String>("IDENTIFIER", "created_at"));
    +
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547115253761,text1,SpreadButter,tweetCongress,,");
    +		output.add(",,,,,,"); // Expected: malformed json records are transformed into empty rows
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547136233472,text3,NoSecrets_Vagas,,,");
    +
    +		super.assertOutput(new Path(System.getProperty("user.dir") + File.separator
    +				+ "src/test/resources/tweets-broken.json"), output);
    +	}
    +
    +	@Test
    +	public void testSmallTweets() throws Exception {
    +
    +		List<String> output = new ArrayList<String>();
    +
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547115253761,text1,SpreadButter,tweetCongress,,");
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547123646465,text2,patronusdeadly,,,");
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547136233472,text3,NoSecrets_Vagas,,,");
    +		output.add("Fri Jun 07 22:45:03 +0000 2013,343136551322136576,text4,SevenStonesBuoy,,-6.1,50.103");
    +
    +		super.assertOutput(new Path(System.getProperty("user.dir") + File.separator
    +				+ "src/test/resources/tweets-small.json"), output);
    +	}
    +
    +	@Test
    +	public void testTweetsWithNull() throws Exception {
    +
    +		List<String> output = new ArrayList<String>();
    +
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,,text1,SpreadButter,tweetCongress,,");
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,,text2,patronusdeadly,,,");
    +
    +		super.assertOutput(new Path(System.getProperty("user.dir") + File.separator
    +				+ "src/test/resources/null-tweets.json"), output);
    +	}
    +
    +	@Test
    +	public void testSmallTweetsWithDelete() throws Exception {
    +
    +		List<String> output = new ArrayList<String>();
    +
    +		output.add(",,,,,,");
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547115253761,text1,SpreadButter,tweetCongress,,");
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547123646465,text2,patronusdeadly,,,");
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547136233472,text3,NoSecrets_Vagas,,,");
    +
    +		super.assertOutput(new Path(System.getProperty("user.dir") + File.separator
    +				+ "src/test/resources/tweets-small-with-delete.json"), output);
    +	}
    +
    +	@Test
    +	public void testWellFormedJson() throws Exception {
    +
    +		extraParams.add(new Pair<String, String>("IDENTIFIER", "created_at"));
    +
    +		List<String> output = new ArrayList<String>();
    +
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547115253761,text1,SpreadButter,tweetCongress,,");
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547123646465,text2,patronusdeadly,,,");
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547136233472,text3,NoSecrets_Vagas,,,");
    +
    +		super.assertOutput(new Path(System.getProperty("user.dir") + File.separator
    +				+ "src/test/resources/tweets-pp.json"), output);
    +	}
    +
    +	@Test
    +	public void testWellFormedJsonWithDelete() throws Exception {
    +
    +		extraParams.add(new Pair<String, String>("IDENTIFIER", "created_at"));
    +
    +		List<String> output = new ArrayList<String>();
    +
    +		// output.add(",,,,,,");
    --- End diff --
    
    remove comment?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by hornn <gi...@git.apache.org>.
Github user hornn commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51509421
  
    --- Diff: pxf/pxf-json/src/main/java/org/apache/hawq/pxf/plugins/json/JsonStreamReader.java ---
    @@ -0,0 +1,121 @@
    +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.BufferedReader;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.io.InputStreamReader;
    +
    +/**
    + * Reads a JSON stream and sequentially extracts all JSON objects identified by the <b>identifier</> parameter. Returns
    + * null when there are no more objects to read.
    + *
    + */
    +public class JsonStreamReader extends BufferedReader {
    +
    +	private static char START_BRACE = '{';
    +	private static char END_BRACE = '}';
    +	private static int EOF = -1;
    +
    +	private StringBuilder bldr = new StringBuilder();
    +	private String identifier = null;
    +	private long bytesRead = 0;
    +
    +	/**
    +	 * @param identifier
    +	 *            JSON object name to extract.
    +	 * @param inputStream
    +	 *            JSON document input stream.
    +	 */
    +	public JsonStreamReader(String identifier, InputStream inputStream) {
    +		super(new InputStreamReader(inputStream));
    +		this.identifier = identifier;
    --- End diff --
    
    I'm sorry if I already mentioned it, but I think we should check also here that identifier is not null or empty, even if it's checked by the accessor. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by tzolov <gi...@git.apache.org>.
Github user tzolov commented on the pull request:

    https://github.com/apache/incubator-hawq/pull/302#issuecomment-177895864
  
    @hornn , @GodenYao 
    The `pxf-json` code was implemented by @adamjshook. In this PR i've barely ported it to the HAWQ pxf project structure. My idea was to port the existing code and then improve it if needed.
    But the excellent comments above made me review the code and find a significant issue with the JsonRecordReader  - e.g. the multiline JSON objects support (also called Pretty Print - PP). Current implementation will not work when the JSON documents spans multiple HDFS Splits! 
    So i will remove the multiline-JSON code from the PR, leaving in only the LineRecordReader version (e.g assuming that json object per line). 
    Also i will open a discussion in the dev mailing list about how to handle in PXF documents that spans across Splits. 



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by tzolov <gi...@git.apache.org>.
Github user tzolov commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51509574
  
    --- Diff: pxf/pxf-json/src/main/java/org/apache/hawq/pxf/plugins/json/JsonStreamReader.java ---
    @@ -0,0 +1,121 @@
    +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.BufferedReader;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.io.InputStreamReader;
    +
    +/**
    + * Reads a JSON stream and sequentially extracts all JSON objects identified by the <b>identifier</> parameter. Returns
    + * null when there are no more objects to read.
    + *
    + */
    +public class JsonStreamReader extends BufferedReader {
    +
    +	private static char START_BRACE = '{';
    +	private static char END_BRACE = '}';
    +	private static int EOF = -1;
    +
    +	private StringBuilder bldr = new StringBuilder();
    +	private String identifier = null;
    +	private long bytesRead = 0;
    +
    +	/**
    +	 * @param identifier
    +	 *            JSON object name to extract.
    +	 * @param inputStream
    +	 *            JSON document input stream.
    +	 */
    +	public JsonStreamReader(String identifier, InputStream inputStream) {
    +		super(new InputStreamReader(inputStream));
    +		this.identifier = identifier;
    +	}
    +
    +	/**
    +	 * @return Returns next JSON object identified by the {@link JsonStreamReader#identifier} parameter or Null if no
    +	 *         more object are available.
    +	 * @throws IOException
    +	 */
    +	public String getJsonRecord() throws IOException {
    +		bldr.delete(0, bldr.length());
    +
    +		boolean foundRecord = false;
    +
    +		int c = 0, numBraces = 1;
    +		while ((c = super.read()) != EOF) {
    +			++bytesRead;
    +			if (!foundRecord) {
    +				bldr.append((char) c);
    +
    +				if (bldr.toString().contains(identifier)) {
    +					if (forwardToStartBrace()) {
    +						foundRecord = true;
    +
    +						bldr.delete(0, bldr.length());
    +						bldr.append(START_BRACE);
    +					}
    +				}
    +			} else {
    +				bldr.append((char) c);
    +
    +				if (c == START_BRACE) {
    +					++numBraces;
    +				} else if (c == END_BRACE) {
    +					--numBraces;
    +				}
    +
    +				if (numBraces == 0) {
    +					break;
    +				}
    +			}
    +		}
    +
    +		if (foundRecord && numBraces == 0) {
    +			return bldr.toString();
    +		} else {
    +			return null;
    --- End diff --
    
    yes it makes sense and i will add it. But mind that the current stream reader is not aware of the semantic of the json document and can miss chunks from the documents if later is  malformed. We need to implement a proper stream reading to minimize such issues.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by tzolov <gi...@git.apache.org>.
Github user tzolov commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51424607
  
    --- 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 --
    
    will chanage the array name/index retrieval to to use the Pattern.compile("(.*)\\[([0-9]+)\\]") matching groups instead


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by hornn <gi...@git.apache.org>.
Github user hornn commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51827170
  
    --- Diff: pxf/pxf-json/src/main/java/org/apache/hawq/pxf/plugins/json/JsonRecordReader.java ---
    @@ -0,0 +1,176 @@
    +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.io.InputStream;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.hadoop.fs.FSDataInputStream;
    +import org.apache.hadoop.fs.FileSystem;
    +import org.apache.hadoop.fs.Path;
    +import org.apache.hadoop.io.LongWritable;
    +import org.apache.hadoop.io.Text;
    +import org.apache.hadoop.io.compress.CompressionCodec;
    +import org.apache.hadoop.io.compress.CompressionCodecFactory;
    +import org.apache.hadoop.mapred.FileSplit;
    +import org.apache.hadoop.mapred.JobConf;
    +import org.apache.hadoop.mapred.RecordReader;
    +import org.apache.hawq.pxf.plugins.json.parser.PartitionedJsonParser;
    +
    +/**
    + * Multi-line json object reader. JsonRecordReader uses a member name (set by the <b>IDENTIFIER</b> PXF parameter) to
    + * determine the encapsulating object to extract and read.
    + * 
    + * JsonRecordReader supports compressed input files as well.
    + * 
    + * As a safe guard set the optional <b>MAXLENGTH</b> parameter to limit the max size of a record.
    + */
    +public class JsonRecordReader implements RecordReader<LongWritable, Text> {
    +
    +	private static final Log LOG = LogFactory.getLog(JsonRecordReader.class);
    +
    +	public static final String RECORD_MEMBER_IDENTIFIER = "json.input.format.record.identifier";
    +	public static final String RECORD_MAX_LENGTH = "multilinejsonrecordreader.maxlength";
    +
    +	private CompressionCodecFactory compressionCodecs = null;
    +	private long start;
    +	private long pos;
    +	private long end;
    +	private int maxObjectLength;
    +	private InputStream is;
    +	private PartitionedJsonParser parser;
    +	private final String jsonMemberName;
    +
    +	/**
    +	 * Create new multi-line json object reader.
    +	 * 
    +	 * @param conf
    +	 *            Hadoop context
    +	 * @param split
    +	 *            HDFS split to start the reading from
    +	 * @throws IOException
    +	 */
    +	public JsonRecordReader(JobConf conf, FileSplit split) throws IOException {
    +
    +		this.jsonMemberName = conf.get(RECORD_MEMBER_IDENTIFIER);
    +		this.maxObjectLength = conf.getInt(RECORD_MAX_LENGTH, Integer.MAX_VALUE);
    +
    +		start = split.getStart();
    +		end = start + split.getLength();
    +		final Path file = split.getPath();
    +		compressionCodecs = new CompressionCodecFactory(conf);
    +		final CompressionCodec codec = compressionCodecs.getCodec(file);
    --- End diff --
    
    Regarding compression, we support it for hdfs files, and the implementation here looks very similar to the one in LineRecordReader. The issue worth mentioning here is that BZip2 Codec is not thread safe. For map-reduce jobs it doesn't matter, because each job is running in its own JVM, but we are handling multiple concurrent requests through tomcat, so that introduced a problem.
    For Hdfs files, we "solved" it by forcing the processing of bzip2 files to be done in a single thread. See HdfsUtilities.isThreadSafe() which is called by HdfsSplittableDataAccessor. I see that the JsonAccessor inherits from this class, so I think it should be fine, but it's worth verifying.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by hornn <gi...@git.apache.org>.
Github user hornn commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51322531
  
    --- Diff: pxf/pxf-json/src/main/java/org/apache/hawq/pxf/plugins/json/JsonStreamReader.java ---
    @@ -0,0 +1,89 @@
    +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.BufferedReader;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.io.InputStreamReader;
    +
    +public class JsonStreamReader extends BufferedReader {
    +
    +	private StringBuilder bldr = new StringBuilder();
    +	private String identifier = null;
    +	private long bytesRead = 0;
    +
    +	public JsonStreamReader(String identifier, InputStream strm) {
    +		super(new InputStreamReader(strm));
    +		this.identifier = identifier;
    +	}
    +
    +	public String getJsonRecord() throws IOException {
    +		bldr.delete(0, bldr.length());
    +
    +		boolean foundRecord = false;
    +
    +		int c = 0, numBraces = 1;
    +		while ((c = super.read()) != -1) {
    +			++bytesRead;
    +			if (!foundRecord) {
    +				bldr.append((char) c);
    +
    +				if (bldr.toString().contains(identifier)) {
    +					forwardToBrace();
    --- End diff --
    
    what if forwardToBrace runs to the end of the stream without finding any '{'?
    In that case wouldn't we return a "record" that only consists of "{"? 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by tzolov <gi...@git.apache.org>.
Github user tzolov commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51513494
  
    --- Diff: pxf/pxf-json/src/main/java/org/apache/hawq/pxf/plugins/json/JsonStreamReader.java ---
    @@ -0,0 +1,121 @@
    +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.BufferedReader;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.io.InputStreamReader;
    +
    +/**
    + * Reads a JSON stream and sequentially extracts all JSON objects identified by the <b>identifier</> parameter. Returns
    + * null when there are no more objects to read.
    + *
    + */
    +public class JsonStreamReader extends BufferedReader {
    +
    +	private static char START_BRACE = '{';
    +	private static char END_BRACE = '}';
    +	private static int EOF = -1;
    +
    +	private StringBuilder bldr = new StringBuilder();
    +	private String identifier = null;
    +	private long bytesRead = 0;
    +
    +	/**
    +	 * @param identifier
    +	 *            JSON object name to extract.
    +	 * @param inputStream
    +	 *            JSON document input stream.
    +	 */
    +	public JsonStreamReader(String identifier, InputStream inputStream) {
    +		super(new InputStreamReader(inputStream));
    +		this.identifier = identifier;
    --- End diff --
    
    makes sense 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by scottcode <gi...@git.apache.org>.
Github user scottcode commented on the pull request:

    https://github.com/apache/incubator-hawq/pull/302#issuecomment-206002524
  
    I have a feature request within the JSON plugin. (Not sure if this is the right venue for this...) Right now, if you specify an element within the JSON that corresponds to further nested objects or arrays, you just get null. I'd like to be able to coerce the nested object as a JSON string. This would be especially useful for structures that have variably-sized nested contents. For example, the Twitter API's `entities` object has an array under the `hashtags` key. It would be nice if specifying `"entities.hashtags" text` within the external table would give you a text representation of the array (e.g. `["tag1", "tag2", "etc"]`). 
    
    If this isn't the right place, let me know where to post such a suggestion. Thanks.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by tzolov <gi...@git.apache.org>.
Github user tzolov commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51493935
  
    --- Diff: pxf/pxf-json/src/test/java/org/apache/pxf/hawq/plugins/json/JsonStreamReaderTest.java ---
    @@ -0,0 +1,53 @@
    +package org.apache.pxf.hawq.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.FileInputStream;
    +import java.io.FileNotFoundException;
    +import java.io.IOException;
    +
    +import org.apache.hawq.pxf.plugins.json.JsonStreamReader;
    +import org.junit.Assert;
    +import org.junit.Before;
    +import org.junit.Test;
    +
    +public class JsonStreamReaderTest {
    --- End diff --
    
    addressed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by shivzone <gi...@git.apache.org>.
Github user shivzone commented on the pull request:

    https://github.com/apache/incubator-hawq/pull/302#issuecomment-206031571
  
    @scottcode the appropriate forum for having questions/discussions such as the one you have is hawq user mailing list. You can find the available mailing lists here http://hawq.incubator.apache.org/#mailing-lists


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by hornn <gi...@git.apache.org>.
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.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by hornn <gi...@git.apache.org>.
Github user hornn commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51906457
  
    --- Diff: pxf/pxf-json/src/test/java/org/apache/hawq/pxf/plugins/json/PxfUnit.java ---
    @@ -0,0 +1,672 @@
    +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.BufferedReader;
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.InputStreamReader;
    +import java.io.OutputStream;
    +import java.lang.reflect.Constructor;
    +import java.security.InvalidParameterException;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.fs.FileSystem;
    +import org.apache.hadoop.fs.Path;
    +import org.apache.hawq.pxf.api.Fragment;
    +import org.apache.hawq.pxf.api.Fragmenter;
    +import org.apache.hawq.pxf.api.OneField;
    +import org.apache.hawq.pxf.api.OneRow;
    +import org.apache.hawq.pxf.api.ReadAccessor;
    +import org.apache.hawq.pxf.api.ReadResolver;
    +import org.apache.hawq.pxf.api.WriteAccessor;
    +import org.apache.hawq.pxf.api.WriteResolver;
    +import org.apache.hawq.pxf.api.io.DataType;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +import org.apache.hawq.pxf.service.FragmentsResponse;
    --- End diff --
    
    Let's leave it as a follow up.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by GodenYao <gi...@git.apache.org>.
Github user GodenYao commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51213394
  
    --- Diff: pxf/pxf-json/src/main/java/org/apache/hawq/pxf/plugins/json/JsonAccessor.java ---
    @@ -0,0 +1,66 @@
    +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 org.apache.commons.lang3.StringUtils;
    +import org.apache.hadoop.mapred.FileSplit;
    +import org.apache.hadoop.mapred.InputSplit;
    +import org.apache.hadoop.mapred.JobConf;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +import org.apache.hawq.pxf.plugins.hdfs.HdfsSplittableDataAccessor;
    +
    +/**
    + * This JSON accessor for PXF will read JSON data and pass it to a {@link JsonResolver}.
    + * 
    + * This accessor supports a single JSON record per line, or a more "pretty print" format.
    + */
    +public class JsonAccessor extends HdfsSplittableDataAccessor {
    +
    +	public static final String IDENTIFIER_PARAM = "IDENTIFIER";
    +	public static final String ONERECORDPERLINE_PARAM = "ONERECORDPERLINE";
    +
    +	private String identifier = "";
    +	private boolean oneRecordPerLine = true;
    +
    +	public JsonAccessor(InputData inputData) throws Exception {
    +		super(inputData, new JsonInputFormat());
    +
    +		if (!StringUtils.isEmpty(inputData.getUserProperty(IDENTIFIER_PARAM))) {
    +			identifier = inputData.getUserProperty(IDENTIFIER_PARAM);
    --- End diff --
    
    Looking at description of JSON-PP plugin:
    
    > A profile for JSON data, where a single JSON record spans multiple lines. Must specify the IDENTIFIER parameter in DDL.
    
    Identifier is required?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by tzolov <gi...@git.apache.org>.
Github user tzolov commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51493009
  
    --- Diff: pxf/pxf-json/src/test/java/org/apache/pxf/hawq/plugins/json/JsonStreamReaderTest.java ---
    @@ -0,0 +1,53 @@
    +package org.apache.pxf.hawq.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.FileInputStream;
    +import java.io.FileNotFoundException;
    +import java.io.IOException;
    +
    +import org.apache.hawq.pxf.plugins.json.JsonStreamReader;
    +import org.junit.Assert;
    +import org.junit.Before;
    +import org.junit.Test;
    +
    +public class JsonStreamReaderTest {
    +
    +	private JsonStreamReader rdr = null;
    +
    +	private static final String FILE = "src/test/resources/sample.json";
    +
    +	@Before
    +	public void setup() throws FileNotFoundException {
    +		rdr = new JsonStreamReader("menuitem", new FileInputStream(FILE));
    +	}
    +
    +	@Test
    +	public void testReadRecords() throws IOException {
    +		int count = 0;
    +		String record = null;
    +		while ((record = rdr.getJsonRecord()) != null) {
    +			++count;
    +			System.out.println(record);
    --- End diff --
    
    done


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by hornn <gi...@git.apache.org>.
Github user hornn commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51322920
  
    --- Diff: pxf/pxf-json/src/test/java/org/apache/pxf/hawq/plugins/json/JsonStreamReaderTest.java ---
    @@ -0,0 +1,53 @@
    +package org.apache.pxf.hawq.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.FileInputStream;
    +import java.io.FileNotFoundException;
    +import java.io.IOException;
    +
    +import org.apache.hawq.pxf.plugins.json.JsonStreamReader;
    +import org.junit.Assert;
    +import org.junit.Before;
    +import org.junit.Test;
    +
    +public class JsonStreamReaderTest {
    --- End diff --
    
    Good test - would it be possible to add a negative test with a malformatted json file?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by tzolov <gi...@git.apache.org>.
Github user tzolov commented on the pull request:

    https://github.com/apache/incubator-hawq/pull/302#issuecomment-178099326
  
    Actually I've been thinking about similar use cases that the current stream reader doesn’t handle. For this we can re-implementing the stream reader using the Jackson JSON Stream API or prehaps to resue some parts of the [json-mapreduce](https://github.com/alexholmes/json-mapreduce) project.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by hornn <gi...@git.apache.org>.
Github user hornn commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51818210
  
    --- Diff: pxf/pxf-json/src/main/java/org/apache/hawq/pxf/plugins/json/ColumnDescriptorCache.java ---
    @@ -0,0 +1,165 @@
    +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.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +
    +import org.apache.hawq.pxf.api.io.DataType;
    +import org.apache.hawq.pxf.api.utilities.ColumnDescriptor;
    +
    +/**
    + * Helper class used to retrieve all column details relevant for the json processing.
    + */
    +public class ColumnDescriptorCache {
    +
    +	private static Pattern ARRAY_PROJECTION_PATTERN = Pattern.compile("(.*)\\[([0-9]+)\\]");
    --- End diff --
    
    just to be safe, I think the first group should be `.+` and not `.*`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by hornn <gi...@git.apache.org>.
Github user hornn commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51834454
  
    --- Diff: pxf/pxf-json/src/main/java/org/apache/hawq/pxf/plugins/json/parser/PartitionedJsonParser.java ---
    @@ -0,0 +1,198 @@
    +package org.apache.hawq.pxf.plugins.json.parser;
    +
    +/*
    + * 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.io.InputStream;
    +import java.io.InputStreamReader;
    +import java.nio.charset.StandardCharsets;
    +import java.util.ArrayList;
    +import java.util.EnumSet;
    +import java.util.List;
    +
    +import org.apache.hawq.pxf.plugins.json.parser.JsonLexer.JsonLexerState;
    +
    +/**
    + * A simple parser that can support reading JSON objects from a random point in JSON text. It reads from the supplied
    + * stream (which is assumed to be positioned at any arbitrary position inside some JSON text) until it find the first
    + * JSON begin-object "{". From this point on it will keep reading JSON objects until it finds one containing a member
    + * string that the user supplies.
    + * <p/>
    + * It is not recommended to use this with JSON text where individual JSON objects that can be large (MB's or larger).
    + */
    +public class PartitionedJsonParser {
    +
    +	private static final int EOF = -1;
    +	private final InputStreamReader inputStreamReader;
    +	private final JsonLexer lexer;
    +	private long bytesRead = 0;
    +	private boolean endOfStream;
    --- End diff --
    
    for clarify, perhaps explicitly assign `false` to it?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by tzolov <gi...@git.apache.org>.
Github user tzolov commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51348576
  
    --- Diff: pxf/pxf-json/src/main/java/org/apache/hawq/pxf/plugins/json/JsonInputFormat.java ---
    @@ -0,0 +1,226 @@
    +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.BufferedInputStream;
    +import java.io.IOException;
    +import java.security.InvalidParameterException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.fs.FSDataInputStream;
    +import org.apache.hadoop.fs.FileSystem;
    +import org.apache.hadoop.fs.Path;
    +import org.apache.hadoop.io.LongWritable;
    +import org.apache.hadoop.io.NullWritable;
    +import org.apache.hadoop.io.Text;
    +import org.apache.hadoop.mapred.FileInputFormat;
    +import org.apache.hadoop.mapred.FileSplit;
    +import org.apache.hadoop.mapred.InputSplit;
    +import org.apache.hadoop.mapred.JobConf;
    +import org.apache.hadoop.mapred.LineRecordReader;
    +import org.apache.hadoop.mapred.RecordReader;
    +import org.apache.hadoop.mapred.Reporter;
    +import org.apache.log4j.Logger;
    +import org.codehaus.jackson.JsonFactory;
    +import org.codehaus.jackson.JsonNode;
    +import org.codehaus.jackson.JsonParseException;
    +import org.codehaus.jackson.map.JsonMappingException;
    +import org.codehaus.jackson.map.ObjectMapper;
    +
    +public class JsonInputFormat extends FileInputFormat<Text, NullWritable> {
    +
    +	private static JsonFactory factory = new JsonFactory();
    +	private static ObjectMapper mapper = new ObjectMapper(factory);
    +
    +	public static final String ONE_RECORD_PER_LINE = "json.input.format.one.record.per.line";
    +	public static final String RECORD_IDENTIFIER = "json.input.format.record.identifier";
    +
    +	@Override
    +	public RecordReader<Text, NullWritable> getRecordReader(InputSplit split, JobConf conf, Reporter reporter)
    +			throws IOException {
    +
    +		if (conf.getBoolean(ONE_RECORD_PER_LINE, false)) {
    +
    +			return new SimpleJsonRecordReader(conf, (FileSplit) split);
    +		} else {
    +			return new JsonRecordReader(conf, (FileSplit) split);
    +		}
    +	}
    +
    +	public static class SimpleJsonRecordReader implements RecordReader<Text, NullWritable> {
    +
    +		private LineRecordReader rdr = null;
    +		private LongWritable key = new LongWritable();
    +		private Text value = new Text();
    +
    +		public SimpleJsonRecordReader(Configuration conf, FileSplit split) throws IOException {
    +			rdr = new LineRecordReader(conf, split);
    +		}
    +
    +		@Override
    +		public void close() throws IOException {
    +			rdr.close();
    +		}
    +
    +		@Override
    +		public Text createKey() {
    +			return value;
    +		}
    +
    +		@Override
    +		public NullWritable createValue() {
    +			return NullWritable.get();
    +		}
    +
    +		@Override
    +		public long getPos() throws IOException {
    +			return rdr.getPos();
    +		}
    +
    +		@Override
    +		public boolean next(Text key, NullWritable value) throws IOException {
    +			if (rdr.next(this.key, this.value)) {
    +				key.set(this.value);
    +				return true;
    +			} else {
    +				return false;
    +			}
    +		}
    +
    +		@Override
    +		public float getProgress() throws IOException {
    +			return rdr.getProgress();
    +		}
    +	}
    +
    +	public static class JsonRecordReader implements RecordReader<Text, NullWritable> {
    +
    +		private Logger LOG = Logger.getLogger(JsonRecordReader.class);
    --- End diff --
    
    will do


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by hornn <gi...@git.apache.org>.
Github user hornn commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51322864
  
    --- Diff: pxf/pxf-json/src/test/java/org/apache/pxf/hawq/plugins/json/JsonStreamReaderTest.java ---
    @@ -0,0 +1,53 @@
    +package org.apache.pxf.hawq.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.FileInputStream;
    +import java.io.FileNotFoundException;
    +import java.io.IOException;
    +
    +import org.apache.hawq.pxf.plugins.json.JsonStreamReader;
    +import org.junit.Assert;
    +import org.junit.Before;
    +import org.junit.Test;
    +
    +public class JsonStreamReaderTest {
    +
    +	private JsonStreamReader rdr = null;
    +
    +	private static final String FILE = "src/test/resources/sample.json";
    +
    +	@Before
    +	public void setup() throws FileNotFoundException {
    +		rdr = new JsonStreamReader("menuitem", new FileInputStream(FILE));
    --- End diff --
    
    please add @After function to close the streams


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by tzolov <gi...@git.apache.org>.
Github user tzolov commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51859883
  
    --- Diff: pxf/pxf-json/src/test/java/org/apache/hawq/pxf/plugins/json/PxfUnit.java ---
    @@ -0,0 +1,672 @@
    +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.BufferedReader;
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.InputStreamReader;
    +import java.io.OutputStream;
    +import java.lang.reflect.Constructor;
    +import java.security.InvalidParameterException;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.fs.FileSystem;
    +import org.apache.hadoop.fs.Path;
    +import org.apache.hawq.pxf.api.Fragment;
    +import org.apache.hawq.pxf.api.Fragmenter;
    +import org.apache.hawq.pxf.api.OneField;
    +import org.apache.hawq.pxf.api.OneRow;
    +import org.apache.hawq.pxf.api.ReadAccessor;
    +import org.apache.hawq.pxf.api.ReadResolver;
    +import org.apache.hawq.pxf.api.WriteAccessor;
    +import org.apache.hawq.pxf.api.WriteResolver;
    +import org.apache.hawq.pxf.api.io.DataType;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +import org.apache.hawq.pxf.service.FragmentsResponse;
    +import org.apache.hawq.pxf.service.FragmentsResponseFormatter;
    +import org.apache.hawq.pxf.service.utilities.ProtocolData;
    +import org.codehaus.jackson.JsonFactory;
    +import org.codehaus.jackson.JsonNode;
    +import org.codehaus.jackson.JsonParseException;
    +import org.codehaus.jackson.map.JsonMappingException;
    +import org.codehaus.jackson.map.ObjectMapper;
    +import org.junit.Assert;
    +
    +/**
    + * This abstract class contains a number of helpful utilities in developing a PXF extension for HAWQ. Extend this class
    + * and use the various <code>assert</code> methods to check given input against known output.
    + */
    +public abstract class PxfUnit {
    +
    +	private static final Log LOG = LogFactory.getLog(PxfUnit.class);
    +
    +	private static JsonFactory factory = new JsonFactory();
    +	private static ObjectMapper mapper = new ObjectMapper(factory);
    +
    +	protected static List<InputData> inputs = null;
    +
    +	/**
    +	 * Uses the given input directory to run through the PXF unit testing framework. Uses the lines in the file for
    +	 * output testing.
    +	 * 
    +	 * @param input
    +	 *            Input records
    +	 * @param expectedOutput
    +	 *            File containing output to check
    +	 * @throws Exception
    +	 */
    +	public void assertOutput(Path input, Path expectedOutput) throws Exception {
    +
    +		BufferedReader rdr = new BufferedReader(new InputStreamReader(FileSystem.get(new Configuration()).open(
    +				expectedOutput)));
    +
    +		List<String> outputLines = new ArrayList<String>();
    +
    +		String line;
    +		while ((line = rdr.readLine()) != null) {
    +			outputLines.add(line);
    +		}
    +
    +		assertOutput(input, outputLines);
    +
    +		rdr.close();
    +	}
    +
    +	/**
    +	 * Uses the given input directory to run through the PXF unit testing framework. Uses the lines in the given
    +	 * parameter for output testing.
    +	 * 
    +	 * @param input
    +	 *            Input records
    +	 * @param expectedOutput
    +	 *            File containing output to check
    +	 * @throws Exception
    +	 */
    +	public void assertOutput(Path input, List<String> expectedOutput) throws Exception {
    +
    +		setup(input);
    +		List<String> actualOutput = new ArrayList<String>();
    +		for (InputData data : inputs) {
    +			ReadAccessor accessor = getReadAccessor(data);
    +			ReadResolver resolver = getReadResolver(data);
    +
    +			actualOutput.addAll(getAllOutput(accessor, resolver));
    +		}
    +
    +		Assert.assertFalse("Output did not match expected output", compareOutput(expectedOutput, actualOutput));
    +	}
    +
    +	/**
    +	 * Uses the given input directory to run through the PXF unit testing framework. Uses the lines in the given
    +	 * parameter for output testing.<br>
    +	 * <br>
    +	 * Ignores order of records.
    +	 * 
    +	 * @param input
    +	 *            Input records
    +	 * @param expectedOutput
    +	 *            File containing output to check
    +	 * @throws Exception
    +	 */
    +	public void assertUnorderedOutput(Path input, Path expectedOutput) throws Exception {
    +		BufferedReader rdr = new BufferedReader(new InputStreamReader(FileSystem.get(new Configuration()).open(
    +				expectedOutput)));
    +
    +		List<String> outputLines = new ArrayList<String>();
    +
    +		String line;
    +		while ((line = rdr.readLine()) != null) {
    +			outputLines.add(line);
    +		}
    +
    +		assertUnorderedOutput(input, outputLines);
    +	}
    --- End diff --
    
    +1


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by hornn <gi...@git.apache.org>.
Github user hornn commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51453322
  
    --- Diff: pxf/pxf-json/src/main/java/org/apache/hawq/pxf/plugins/json/JsonInputFormat.java ---
    @@ -0,0 +1,226 @@
    +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.BufferedInputStream;
    +import java.io.IOException;
    +import java.security.InvalidParameterException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.fs.FSDataInputStream;
    +import org.apache.hadoop.fs.FileSystem;
    +import org.apache.hadoop.fs.Path;
    +import org.apache.hadoop.io.LongWritable;
    +import org.apache.hadoop.io.NullWritable;
    +import org.apache.hadoop.io.Text;
    +import org.apache.hadoop.mapred.FileInputFormat;
    +import org.apache.hadoop.mapred.FileSplit;
    +import org.apache.hadoop.mapred.InputSplit;
    +import org.apache.hadoop.mapred.JobConf;
    +import org.apache.hadoop.mapred.LineRecordReader;
    +import org.apache.hadoop.mapred.RecordReader;
    +import org.apache.hadoop.mapred.Reporter;
    +import org.apache.log4j.Logger;
    +import org.codehaus.jackson.JsonFactory;
    +import org.codehaus.jackson.JsonNode;
    +import org.codehaus.jackson.JsonParseException;
    +import org.codehaus.jackson.map.JsonMappingException;
    +import org.codehaus.jackson.map.ObjectMapper;
    +
    +public class JsonInputFormat extends FileInputFormat<Text, NullWritable> {
    +
    +	private static JsonFactory factory = new JsonFactory();
    +	private static ObjectMapper mapper = new ObjectMapper(factory);
    +
    +	public static final String ONE_RECORD_PER_LINE = "json.input.format.one.record.per.line";
    +	public static final String RECORD_IDENTIFIER = "json.input.format.record.identifier";
    +
    +	@Override
    +	public RecordReader<Text, NullWritable> getRecordReader(InputSplit split, JobConf conf, Reporter reporter)
    +			throws IOException {
    +
    +		if (conf.getBoolean(ONE_RECORD_PER_LINE, false)) {
    +
    +			return new SimpleJsonRecordReader(conf, (FileSplit) split);
    +		} else {
    +			return new JsonRecordReader(conf, (FileSplit) split);
    +		}
    +	}
    +
    +	public static class SimpleJsonRecordReader implements RecordReader<Text, NullWritable> {
    +
    +		private LineRecordReader rdr = null;
    +		private LongWritable key = new LongWritable();
    +		private Text value = new Text();
    +
    +		public SimpleJsonRecordReader(Configuration conf, FileSplit split) throws IOException {
    +			rdr = new LineRecordReader(conf, split);
    +		}
    +
    +		@Override
    +		public void close() throws IOException {
    +			rdr.close();
    +		}
    +
    +		@Override
    +		public Text createKey() {
    +			return value;
    +		}
    +
    +		@Override
    +		public NullWritable createValue() {
    +			return NullWritable.get();
    +		}
    +
    +		@Override
    +		public long getPos() throws IOException {
    +			return rdr.getPos();
    +		}
    +
    +		@Override
    +		public boolean next(Text key, NullWritable value) throws IOException {
    --- End diff --
    
    I agree, the rest is more clear. Perhaps add a comment also for the class, explaining the general flow.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by hornn <gi...@git.apache.org>.
Github user hornn commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51452980
  
    --- Diff: pxf/pxf-service/src/main/resources/pxf-profiles-default.xml ---
    @@ -128,4 +128,26 @@ under the License.
                 <resolver>org.apache.hawq.pxf.plugins.gemfirexd.GemFireXDResolver</resolver>
             </plugins>
         </profile>
    +	<profile>
    +		<name>JSON</name>
    +		<description>A profile for JSON data, one JSON record per line
    +		</description>
    +		<plugins>
    +			<fragmenter>org.apache.hawq.pxf.plugins.hdfs.HdfsDataFragmenter</fragmenter>
    +			<accessor>org.apache.hawq.pxf.plugins.json.JsonAccessor</accessor>
    +			<resolver>org.apache.hawq.pxf.plugins.json.JsonResolver</resolver>
    +		</plugins>
    +	</profile>
    +	<profile>
    +		<name>JSON-PP</name>
    --- End diff --
    
    Both JSON and Json is fine, but I like Json better.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by tzolov <gi...@git.apache.org>.
Github user tzolov commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51347433
  
    --- Diff: pxf/pxf-json/src/test/java/org/apache/pxf/hawq/plugins/json/JsonExtensionTest.java ---
    @@ -0,0 +1,173 @@
    +package org.apache.pxf.hawq.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.util.ArrayList;
    +import java.util.List;
    +
    +import org.apache.hadoop.fs.Path;
    +import org.apache.hawq.pxf.api.Fragmenter;
    +import org.apache.hawq.pxf.api.ReadAccessor;
    +import org.apache.hawq.pxf.api.ReadResolver;
    +import org.apache.hawq.pxf.api.io.DataType;
    +import org.apache.hawq.pxf.plugins.hdfs.HdfsDataFragmenter;
    +import org.apache.hawq.pxf.plugins.json.JsonAccessor;
    +import org.apache.hawq.pxf.plugins.json.JsonResolver;
    +import org.junit.After;
    +import org.junit.Test;
    +
    +public class JsonExtensionTest extends PxfUnit {
    +
    +	private static List<Pair<String, DataType>> columnDefs = null;
    +	private static List<Pair<String, String>> extraParams = new ArrayList<Pair<String, String>>();
    +
    +	static {
    +
    +		columnDefs = new ArrayList<Pair<String, DataType>>();
    +
    +		columnDefs.add(new Pair<String, DataType>("created_at", DataType.TEXT));
    +		columnDefs.add(new Pair<String, DataType>("id", DataType.BIGINT));
    +		columnDefs.add(new Pair<String, DataType>("text", DataType.TEXT));
    +		columnDefs.add(new Pair<String, DataType>("user.screen_name", DataType.TEXT));
    +		columnDefs.add(new Pair<String, DataType>("entities.hashtags[0]", DataType.TEXT));
    +		columnDefs.add(new Pair<String, DataType>("coordinates.coordinates[0]", DataType.FLOAT8));
    +		columnDefs.add(new Pair<String, DataType>("coordinates.coordinates[1]", DataType.FLOAT8));
    +	}
    +
    +	@After
    +	public void cleanup() throws Exception {
    +		extraParams.clear();
    +	}
    +
    +	@Test
    +	public void testSmallTweets() throws Exception {
    +
    +		List<String> output = new ArrayList<String>();
    +
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547115253761,REPAIR THE TRUST: REMOVE OBAMA/BIDEN FROM OFFICE. #IRS #DOJ #NSA #tcot,SpreadButter,tweetCongress,,");
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547123646465,@marshafitrie dibagi 1000 aja sha :P,patronusdeadly,,,");
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547136233472,Vaga: Supervisor de Almoxarifado. Confira em http://t.co/hK5cy5B2oS,NoSecrets_Vagas,,,");
    +		output.add("Fri Jun 07 22:45:03 +0000 2013,343136551322136576,It's Jun 7, 2013 @ 11pm ; Wind = NNE (30,0) 14.0 knots; Swell = 2.6 ft @ 5 seconds....,SevenStonesBuoy,,-6.1,50.103");
    +
    +		super.assertOutput(new Path(System.getProperty("user.dir") + "/" + "src/test/resources/tweets-small.json"),
    --- End diff --
    
    yep


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by adamjshook <gi...@git.apache.org>.
Github user adamjshook commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r52078650
  
    --- Diff: pxf/pxf-json/src/test/java/org/apache/hawq/pxf/plugins/json/PxfUnit.java ---
    @@ -0,0 +1,672 @@
    +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.BufferedReader;
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.InputStreamReader;
    +import java.io.OutputStream;
    +import java.lang.reflect.Constructor;
    +import java.security.InvalidParameterException;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.fs.FileSystem;
    +import org.apache.hadoop.fs.Path;
    +import org.apache.hawq.pxf.api.Fragment;
    +import org.apache.hawq.pxf.api.Fragmenter;
    +import org.apache.hawq.pxf.api.OneField;
    +import org.apache.hawq.pxf.api.OneRow;
    +import org.apache.hawq.pxf.api.ReadAccessor;
    +import org.apache.hawq.pxf.api.ReadResolver;
    +import org.apache.hawq.pxf.api.WriteAccessor;
    +import org.apache.hawq.pxf.api.WriteResolver;
    +import org.apache.hawq.pxf.api.io.DataType;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +import org.apache.hawq.pxf.service.FragmentsResponse;
    --- End diff --
    
    I took some inspiration from MRUnit and wanted to mock the execution framework of PXF.  I imagine it could use some work, but I think it would be very beneficial in the long run for developers to do some testing of their extensions without having to install it on a HAWQ system.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by hornn <gi...@git.apache.org>.
Github user hornn commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51212800
  
    --- Diff: pxf/pxf-json/src/main/java/org/apache/hawq/pxf/plugins/json/JsonAccessor.java ---
    @@ -0,0 +1,66 @@
    +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 org.apache.commons.lang3.StringUtils;
    +import org.apache.hadoop.mapred.FileSplit;
    +import org.apache.hadoop.mapred.InputSplit;
    +import org.apache.hadoop.mapred.JobConf;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +import org.apache.hawq.pxf.plugins.hdfs.HdfsSplittableDataAccessor;
    +
    +/**
    + * This JSON accessor for PXF will read JSON data and pass it to a {@link JsonResolver}.
    + * 
    + * This accessor supports a single JSON record per line, or a more "pretty print" format.
    + */
    +public class JsonAccessor extends HdfsSplittableDataAccessor {
    +
    +	public static final String IDENTIFIER_PARAM = "IDENTIFIER";
    +	public static final String ONERECORDPERLINE_PARAM = "ONERECORDPERLINE";
    +
    +	private String identifier = "";
    +	private boolean oneRecordPerLine = true;
    +
    +	public JsonAccessor(InputData inputData) throws Exception {
    +		super(inputData, new JsonInputFormat());
    +
    +		if (!StringUtils.isEmpty(inputData.getUserProperty(IDENTIFIER_PARAM))) {
    +			identifier = inputData.getUserProperty(IDENTIFIER_PARAM);
    +		}
    +
    +		if (!StringUtils.isEmpty(inputData.getUserProperty(ONERECORDPERLINE_PARAM))) {
    +			oneRecordPerLine = Boolean.parseBoolean(inputData.getUserProperty(ONERECORDPERLINE_PARAM));
    +		}
    +	}
    +
    +	@Override
    +	protected Object getReader(JobConf conf, InputSplit split) throws IOException {
    +		conf.set(JsonInputFormat.RECORD_IDENTIFIER, identifier);
    --- End diff --
    
    same comment - if identifier is "", should we set it here or not?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by hornn <gi...@git.apache.org>.
Github user hornn commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51452719
  
    --- Diff: pxf/build.gradle ---
    @@ -64,7 +64,7 @@ subprojects { subProject ->
                 "-Xlint:finally", "-Xlint:overrides", "-Xlint:path", "-Xlint:processing", "-Xlint:static", "-Xlint:try",
                 "-Xlint:-fallthrough", "-Xlint:-deprecation", "-Xlint:-unchecked", "-Xlint:-options"
         ]
    -
    +		
    --- End diff --
    
    sure :)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by hornn <gi...@git.apache.org>.
Github user hornn commented on the pull request:

    https://github.com/apache/incubator-hawq/pull/302#issuecomment-185331246
  
    There are also a few minor fixes to be merged: https://github.com/tzolov/incubator-hawq/pull/1


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by sansanichfb <gi...@git.apache.org>.
Github user sansanichfb commented on the pull request:

    https://github.com/apache/incubator-hawq/pull/302#issuecomment-220192722
  
    Merging this PR, will address bytea type in separate story.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by tzolov <gi...@git.apache.org>.
Github user tzolov commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r52080654
  
    --- Diff: pxf/pxf-service/src/main/resources/pxf-profiles-default.xml ---
    @@ -128,4 +128,13 @@ under the License.
                 <resolver>org.apache.hawq.pxf.plugins.gemfirexd.GemFireXDResolver</resolver>
             </plugins>
         </profile>
    +    <profile>
    +        <name>Json</name>
    +        <description>A profile for JSON data, one JSON record per line</description>
    --- End diff --
    
    Thanks for spotting this @GodenYao! Just fixed it
    
    P.S. as an experiment i've implemented a Json Resolver that leverages [JsonPath](https://github.com/jayway/JsonPath). It allows you to  traverse arbitrary complex json records. (current expression mechanism is quite limited). But this an experimental work and we should leave a possible future improvement. Perhaps as an additional 3rd mode -  the most flexible but potentially slowest. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by hornn <gi...@git.apache.org>.
Github user hornn commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51210807
  
    --- Diff: pxf/pxf-json/src/main/java/org/apache/hawq/pxf/plugins/json/JsonInputFormat.java ---
    @@ -0,0 +1,226 @@
    +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.BufferedInputStream;
    +import java.io.IOException;
    +import java.security.InvalidParameterException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.fs.FSDataInputStream;
    +import org.apache.hadoop.fs.FileSystem;
    +import org.apache.hadoop.fs.Path;
    +import org.apache.hadoop.io.LongWritable;
    +import org.apache.hadoop.io.NullWritable;
    +import org.apache.hadoop.io.Text;
    +import org.apache.hadoop.mapred.FileInputFormat;
    +import org.apache.hadoop.mapred.FileSplit;
    +import org.apache.hadoop.mapred.InputSplit;
    +import org.apache.hadoop.mapred.JobConf;
    +import org.apache.hadoop.mapred.LineRecordReader;
    +import org.apache.hadoop.mapred.RecordReader;
    +import org.apache.hadoop.mapred.Reporter;
    +import org.apache.log4j.Logger;
    +import org.codehaus.jackson.JsonFactory;
    +import org.codehaus.jackson.JsonNode;
    +import org.codehaus.jackson.JsonParseException;
    +import org.codehaus.jackson.map.JsonMappingException;
    +import org.codehaus.jackson.map.ObjectMapper;
    +
    +public class JsonInputFormat extends FileInputFormat<Text, NullWritable> {
    +
    +	private static JsonFactory factory = new JsonFactory();
    +	private static ObjectMapper mapper = new ObjectMapper(factory);
    +
    +	public static final String ONE_RECORD_PER_LINE = "json.input.format.one.record.per.line";
    +	public static final String RECORD_IDENTIFIER = "json.input.format.record.identifier";
    +
    +	@Override
    +	public RecordReader<Text, NullWritable> getRecordReader(InputSplit split, JobConf conf, Reporter reporter)
    +			throws IOException {
    +
    +		if (conf.getBoolean(ONE_RECORD_PER_LINE, false)) {
    +
    +			return new SimpleJsonRecordReader(conf, (FileSplit) split);
    +		} else {
    +			return new JsonRecordReader(conf, (FileSplit) split);
    +		}
    +	}
    +
    +	public static class SimpleJsonRecordReader implements RecordReader<Text, NullWritable> {
    +
    +		private LineRecordReader rdr = null;
    +		private LongWritable key = new LongWritable();
    +		private Text value = new Text();
    +
    +		public SimpleJsonRecordReader(Configuration conf, FileSplit split) throws IOException {
    +			rdr = new LineRecordReader(conf, split);
    +		}
    +
    +		@Override
    +		public void close() throws IOException {
    +			rdr.close();
    +		}
    +
    +		@Override
    +		public Text createKey() {
    +			return value;
    +		}
    +
    +		@Override
    +		public NullWritable createValue() {
    +			return NullWritable.get();
    +		}
    +
    +		@Override
    +		public long getPos() throws IOException {
    +			return rdr.getPos();
    +		}
    +
    +		@Override
    +		public boolean next(Text key, NullWritable value) throws IOException {
    +			if (rdr.next(this.key, this.value)) {
    +				key.set(this.value);
    +				return true;
    +			} else {
    +				return false;
    +			}
    +		}
    +
    +		@Override
    +		public float getProgress() throws IOException {
    +			return rdr.getProgress();
    +		}
    +	}
    +
    +	public static class JsonRecordReader implements RecordReader<Text, NullWritable> {
    +
    +		private Logger LOG = Logger.getLogger(JsonRecordReader.class);
    +
    +		private JsonStreamReader rdr = null;
    +		private long start = 0, end = 0;
    +		private float toRead = 0;
    +		private String identifier = null;
    +		private Logger log = Logger.getLogger(JsonRecordReader.class);
    +
    +		public JsonRecordReader(JobConf conf, FileSplit split) throws IOException {
    +			log.info("JsonRecordReader constructor called.  Conf is " + conf + ". Split is " + split);
    --- End diff --
    
    how often is this ctor called? consider changing all logs to debug level


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by tzolov <gi...@git.apache.org>.
Github user tzolov commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51859823
  
    --- Diff: pxf/pxf-json/src/test/java/org/apache/hawq/pxf/plugins/json/PxfUnit.java ---
    @@ -0,0 +1,672 @@
    +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.BufferedReader;
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.InputStreamReader;
    +import java.io.OutputStream;
    +import java.lang.reflect.Constructor;
    +import java.security.InvalidParameterException;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.fs.FileSystem;
    +import org.apache.hadoop.fs.Path;
    +import org.apache.hawq.pxf.api.Fragment;
    +import org.apache.hawq.pxf.api.Fragmenter;
    +import org.apache.hawq.pxf.api.OneField;
    +import org.apache.hawq.pxf.api.OneRow;
    +import org.apache.hawq.pxf.api.ReadAccessor;
    +import org.apache.hawq.pxf.api.ReadResolver;
    +import org.apache.hawq.pxf.api.WriteAccessor;
    +import org.apache.hawq.pxf.api.WriteResolver;
    +import org.apache.hawq.pxf.api.io.DataType;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +import org.apache.hawq.pxf.service.FragmentsResponse;
    +import org.apache.hawq.pxf.service.FragmentsResponseFormatter;
    +import org.apache.hawq.pxf.service.utilities.ProtocolData;
    +import org.codehaus.jackson.JsonFactory;
    +import org.codehaus.jackson.JsonNode;
    +import org.codehaus.jackson.JsonParseException;
    +import org.codehaus.jackson.map.JsonMappingException;
    +import org.codehaus.jackson.map.ObjectMapper;
    +import org.junit.Assert;
    +
    +/**
    + * This abstract class contains a number of helpful utilities in developing a PXF extension for HAWQ. Extend this class
    + * and use the various <code>assert</code> methods to check given input against known output.
    + */
    +public abstract class PxfUnit {
    +
    +	private static final Log LOG = LogFactory.getLog(PxfUnit.class);
    +
    +	private static JsonFactory factory = new JsonFactory();
    +	private static ObjectMapper mapper = new ObjectMapper(factory);
    +
    +	protected static List<InputData> inputs = null;
    +
    +	/**
    +	 * Uses the given input directory to run through the PXF unit testing framework. Uses the lines in the file for
    +	 * output testing.
    +	 * 
    +	 * @param input
    +	 *            Input records
    +	 * @param expectedOutput
    +	 *            File containing output to check
    +	 * @throws Exception
    +	 */
    +	public void assertOutput(Path input, Path expectedOutput) throws Exception {
    +
    +		BufferedReader rdr = new BufferedReader(new InputStreamReader(FileSystem.get(new Configuration()).open(
    +				expectedOutput)));
    +
    +		List<String> outputLines = new ArrayList<String>();
    +
    +		String line;
    +		while ((line = rdr.readLine()) != null) {
    +			outputLines.add(line);
    +		}
    +
    +		assertOutput(input, outputLines);
    +
    +		rdr.close();
    +	}
    +
    +	/**
    +	 * Uses the given input directory to run through the PXF unit testing framework. Uses the lines in the given
    +	 * parameter for output testing.
    +	 * 
    +	 * @param input
    +	 *            Input records
    +	 * @param expectedOutput
    +	 *            File containing output to check
    +	 * @throws Exception
    +	 */
    +	public void assertOutput(Path input, List<String> expectedOutput) throws Exception {
    +
    +		setup(input);
    +		List<String> actualOutput = new ArrayList<String>();
    +		for (InputData data : inputs) {
    +			ReadAccessor accessor = getReadAccessor(data);
    +			ReadResolver resolver = getReadResolver(data);
    +
    +			actualOutput.addAll(getAllOutput(accessor, resolver));
    +		}
    +
    +		Assert.assertFalse("Output did not match expected output", compareOutput(expectedOutput, actualOutput));
    +	}
    +
    +	/**
    +	 * Uses the given input directory to run through the PXF unit testing framework. Uses the lines in the given
    +	 * parameter for output testing.<br>
    +	 * <br>
    +	 * Ignores order of records.
    +	 * 
    +	 * @param input
    +	 *            Input records
    +	 * @param expectedOutput
    +	 *            File containing output to check
    +	 * @throws Exception
    +	 */
    +	public void assertUnorderedOutput(Path input, Path expectedOutput) throws Exception {
    +		BufferedReader rdr = new BufferedReader(new InputStreamReader(FileSystem.get(new Configuration()).open(
    +				expectedOutput)));
    +
    +		List<String> outputLines = new ArrayList<String>();
    +
    +		String line;
    +		while ((line = rdr.readLine()) != null) {
    +			outputLines.add(line);
    +		}
    +
    +		assertUnorderedOutput(input, outputLines);
    +	}
    +
    +	/**
    +	 * Uses the given input directory to run through the PXF unit testing framework. Uses the lines in the file for
    +	 * output testing.<br>
    +	 * <br>
    +	 * Ignores order of records.
    +	 * 
    +	 * @param input
    +	 *            Input records
    +	 * @param expectedOutput
    +	 *            File containing output to check
    +	 * @throws Exception
    +	 */
    +	public void assertUnorderedOutput(Path input, List<String> expectedOutput) throws Exception {
    +
    +		setup(input);
    +
    +		List<String> actualOutput = new ArrayList<String>();
    +		for (InputData data : inputs) {
    +			ReadAccessor accessor = getReadAccessor(data);
    +			ReadResolver resolver = getReadResolver(data);
    +
    +			actualOutput.addAll(getAllOutput(accessor, resolver));
    +		}
    +
    +		Assert.assertFalse("Output did not match expected output", compareUnorderedOutput(expectedOutput, actualOutput));
    +	}
    +
    +	/**
    +	 * Writes the output to the given output stream. Comma delimiter.
    +	 * 
    +	 * @param input
    +	 *            The input file
    +	 * @param output
    +	 *            The output stream
    +	 * @throws Exception
    +	 */
    +	public void writeOutput(Path input, OutputStream output) throws Exception {
    +
    +		setup(input);
    +
    +		for (InputData data : inputs) {
    +			ReadAccessor accessor = getReadAccessor(data);
    +			ReadResolver resolver = getReadResolver(data);
    +
    +			for (String line : getAllOutput(accessor, resolver)) {
    +				output.write((line + "\n").getBytes());
    +			}
    +		}
    +
    +		output.flush();
    +	}
    +
    +	/**
    +	 * Get the class of the implementation of Fragmenter to be tested.
    +	 * 
    +	 * @return The class
    +	 */
    +	public Class<? extends Fragmenter> getFragmenterClass() {
    +		return null;
    +	}
    +
    +	/**
    +	 * Get the class of the implementation of ReadAccessor to be tested.
    +	 * 
    +	 * @return The class
    +	 */
    +	public Class<? extends ReadAccessor> getReadAccessorClass() {
    +		return null;
    +	}
    +
    +	/**
    +	 * Get the class of the implementation of WriteAccessor to be tested.
    +	 * 
    +	 * @return The class
    +	 */
    +	public Class<? extends WriteAccessor> getWriteAccessorClass() {
    +		return null;
    +	}
    +
    +	/**
    +	 * Get the class of the implementation of Resolver to be tested.
    +	 * 
    +	 * @return The class
    +	 */
    +	public Class<? extends ReadResolver> getReadResolverClass() {
    +		return null;
    +	}
    +
    +	/**
    +	 * Get the class of the implementation of WriteResolver to be tested.
    +	 * 
    +	 * @return The class
    +	 */
    +	public Class<? extends WriteResolver> getWriteResolverClass() {
    +		return null;
    +	}
    +
    +	/**
    +	 * Get any extra parameters that are meant to be specified for the "pxf" protocol. Note that "X-GP-" is prepended to
    +	 * each parameter name.
    +	 * 
    +	 * @return Any extra parameters or null if none.
    +	 */
    +	public List<Pair<String, String>> getExtraParams() {
    +		return null;
    +	}
    +
    +	/**
    +	 * Gets the column definition names and data types. Types are DataType objects
    +	 * 
    +	 * @return A list of column definition name value pairs. Cannot be null.
    +	 */
    +	public abstract List<Pair<String, DataType>> getColumnDefinitions();
    +
    +	protected InputData getInputDataForWritableTable() {
    +		return getInputDataForWritableTable(null);
    +	}
    +
    +	protected InputData getInputDataForWritableTable(Path input) {
    +
    +		if (getWriteAccessorClass() == null) {
    +			throw new IllegalArgumentException(
    +					"getWriteAccessorClass() must be overwritten to return a non-null object");
    +		}
    +
    +		if (getWriteResolverClass() == null) {
    +			throw new IllegalArgumentException(
    +					"getWriteResolverClass() must be overwritten to return a non-null object");
    +		}
    +
    +		Map<String, String> paramsMap = new HashMap<String, String>();
    +
    +		paramsMap.put("X-GP-ALIGNMENT", "what");
    +		paramsMap.put("X-GP-SEGMENT-ID", "1");
    +		paramsMap.put("X-GP-HAS-FILTER", "0");
    +		paramsMap.put("X-GP-SEGMENT-COUNT", "1");
    +
    +		paramsMap.put("X-GP-FORMAT", "GPDBWritable");
    +		paramsMap.put("X-GP-URL-HOST", "localhost");
    +		paramsMap.put("X-GP-URL-PORT", "50070");
    +
    +		if (input == null) {
    +			paramsMap.put("X-GP-DATA-DIR", "/dummydata");
    +		}
    +
    +		List<Pair<String, DataType>> params = getColumnDefinitions();
    +		paramsMap.put("X-GP-ATTRS", Integer.toString(params.size()));
    +		for (int i = 0; i < params.size(); ++i) {
    +			paramsMap.put("X-GP-ATTR-NAME" + i, params.get(i).first);
    +			paramsMap.put("X-GP-ATTR-TYPENAME" + i, params.get(i).second.name());
    +			paramsMap.put("X-GP-ATTR-TYPECODE" + i, Integer.toString(params.get(i).second.getOID()));
    +		}
    +
    +		paramsMap.put("X-GP-ACCESSOR", getWriteAccessorClass().getName());
    +		paramsMap.put("X-GP-RESOLVER", getWriteResolverClass().getName());
    +
    +		if (getExtraParams() != null) {
    +			for (Pair<String, String> param : getExtraParams()) {
    +				paramsMap.put("X-GP-" + param.first, param.second);
    +			}
    +		}
    +
    +		return new ProtocolData(paramsMap);
    +	}
    +
    +	/**
    +	 * Set all necessary parameters for GPXF framework to function. Uses the given path as a single input split.
    +	 * 
    +	 * @param input
    +	 *            The input path, relative or absolute.
    +	 * @throws Exception
    +	 */
    +	protected void setup(Path input) throws Exception {
    +
    +		if (getFragmenterClass() == null) {
    +			throw new IllegalArgumentException("getFragmenterClass() must be overwritten to return a non-null object");
    +		}
    +
    +		if (getReadAccessorClass() == null) {
    +			throw new IllegalArgumentException("getReadAccessorClass() must be overwritten to return a non-null object");
    +		}
    +
    +		if (getReadResolverClass() == null) {
    +			throw new IllegalArgumentException("getReadResolverClass() must be overwritten to return a non-null object");
    +		}
    +
    +		Map<String, String> paramsMap = new HashMap<String, String>();
    +
    +		// 2.1.0 Properties
    +		// HDMetaData parameters
    +		paramsMap.put("X-GP-ALIGNMENT", "what");
    +		paramsMap.put("X-GP-SEGMENT-ID", "1");
    +		paramsMap.put("X-GP-HAS-FILTER", "0");
    +		paramsMap.put("X-GP-SEGMENT-COUNT", "1");
    +		paramsMap.put("X-GP-FRAGMENTER", getFragmenterClass().getName());
    +		paramsMap.put("X-GP-FORMAT", "GPDBWritable");
    +		paramsMap.put("X-GP-URL-HOST", "localhost");
    +		paramsMap.put("X-GP-URL-PORT", "50070");
    +
    +		paramsMap.put("X-GP-DATA-DIR", input.toString());
    +
    +		List<Pair<String, DataType>> params = getColumnDefinitions();
    +		paramsMap.put("X-GP-ATTRS", Integer.toString(params.size()));
    +		for (int i = 0; i < params.size(); ++i) {
    +			paramsMap.put("X-GP-ATTR-NAME" + i, params.get(i).first);
    +			paramsMap.put("X-GP-ATTR-TYPENAME" + i, params.get(i).second.name());
    +			paramsMap.put("X-GP-ATTR-TYPECODE" + i, Integer.toString(params.get(i).second.getOID()));
    +		}
    +
    +		// HDFSMetaData properties
    +		paramsMap.put("X-GP-ACCESSOR", getReadAccessorClass().getName());
    +		paramsMap.put("X-GP-RESOLVER", getReadResolverClass().getName());
    +
    +		if (getExtraParams() != null) {
    +			for (Pair<String, String> param : getExtraParams()) {
    +				paramsMap.put("X-GP-" + param.first, param.second);
    +			}
    +		}
    +
    +		LocalInputData fragmentInputData = new LocalInputData(paramsMap);
    +
    +		List<Fragment> fragments = getFragmenter(fragmentInputData).getFragments();
    +
    +		FragmentsResponse fragmentsResponse = FragmentsResponseFormatter.formatResponse(fragments, input.toString());
    +
    +		ByteArrayOutputStream baos = new ByteArrayOutputStream();
    +		fragmentsResponse.write(baos);
    +
    +		String jsonOutput = baos.toString();
    +
    +		inputs = new ArrayList<InputData>();
    +
    +		JsonNode node = decodeLineToJsonNode(jsonOutput);
    +
    +		JsonNode fragmentsArray = node.get("PXFFragments");
    +		int i = 0;
    +		Iterator<JsonNode> iter = fragmentsArray.getElements();
    +		while (iter.hasNext()) {
    +			JsonNode fragNode = iter.next();
    +			String sourceData = fragNode.get("sourceName").getTextValue();
    +			if (!sourceData.startsWith("/")) {
    +				sourceData = "/" + sourceData;
    +			}
    +			paramsMap.put("X-GP-DATA-DIR", sourceData);
    +			paramsMap.put("X-GP-FRAGMENT-METADATA", fragNode.get("metadata").getTextValue());
    +			paramsMap.put("X-GP-DATA-FRAGMENT", Integer.toString(i++));
    +			inputs.add(new LocalInputData(paramsMap));
    +		}
    +	}
    +
    +	private JsonNode decodeLineToJsonNode(String line) {
    +
    +		try {
    +			return mapper.readTree(line);
    +		} catch (JsonParseException e) {
    +			e.printStackTrace();
    --- End diff --
    
    +1. missed this one


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by tzolov <gi...@git.apache.org>.
Github user tzolov commented on the pull request:

    https://github.com/apache/incubator-hawq/pull/302#issuecomment-178057412
  
    @adamjshook, it's nice to hear from you!  
    You are right the `JsonRecordReader`,`JsonStreamReader` reads beyond the `Split` boundries. I was missleaded by the `JsonInputFormat` which is never been used. The `HdfsSplittableDataAccessor` (the JsonAccessor parent) doesn't use the InptuFormat and getSplits\getRecordReader methods are not called. 
    The `HdfsSplittableDataAccessor` generates exactly **one** split!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by tzolov <gi...@git.apache.org>.
Github user tzolov commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51853867
  
    --- Diff: pxf/pxf-json/src/main/java/org/apache/hawq/pxf/plugins/json/ColumnDescriptorCache.java ---
    @@ -0,0 +1,165 @@
    +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.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +
    +import org.apache.hawq.pxf.api.io.DataType;
    +import org.apache.hawq.pxf.api.utilities.ColumnDescriptor;
    +
    +/**
    + * Helper class used to retrieve all column details relevant for the json processing.
    + */
    +public class ColumnDescriptorCache {
    +
    +	private static Pattern ARRAY_PROJECTION_PATTERN = Pattern.compile("(.*)\\[([0-9]+)\\]");
    --- End diff --
    
    +1


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by hornn <gi...@git.apache.org>.
Github user hornn commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51322662
  
    --- Diff: pxf/pxf-json/src/test/java/org/apache/pxf/hawq/plugins/json/JsonStreamReaderTest.java ---
    @@ -0,0 +1,53 @@
    +package org.apache.pxf.hawq.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.FileInputStream;
    +import java.io.FileNotFoundException;
    +import java.io.IOException;
    +
    +import org.apache.hawq.pxf.plugins.json.JsonStreamReader;
    +import org.junit.Assert;
    +import org.junit.Before;
    +import org.junit.Test;
    +
    +public class JsonStreamReaderTest {
    +
    +	private JsonStreamReader rdr = null;
    +
    +	private static final String FILE = "src/test/resources/sample.json";
    +
    +	@Before
    +	public void setup() throws FileNotFoundException {
    +		rdr = new JsonStreamReader("menuitem", new FileInputStream(FILE));
    +	}
    +
    +	@Test
    +	public void testReadRecords() throws IOException {
    +		int count = 0;
    +		String record = null;
    +		while ((record = rdr.getJsonRecord()) != null) {
    +			++count;
    +			System.out.println(record);
    --- End diff --
    
    instead of printing, better to assert the content of each record.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by tzolov <gi...@git.apache.org>.
Github user tzolov commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51859667
  
    --- Diff: pxf/pxf-json/src/test/java/org/apache/hawq/pxf/plugins/json/parser/PartitionedJsonParserOffsetTest.java ---
    @@ -0,0 +1,54 @@
    +package org.apache.hawq.pxf.plugins.json.parser;
    +
    +/*
    + * 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 static org.junit.Assert.assertEquals;
    +import static org.junit.Assert.assertNotNull;
    +
    +import java.io.ByteArrayInputStream;
    +import java.io.IOException;
    +import java.io.InputStream;
    +
    +import org.junit.Test;
    +
    +public class PartitionedJsonParserOffsetTest {
    +	/*
    +	 * [{"color": "red","v": "vv"},{"color": "red","v": "vv"}]
    +	 */
    +	public static String json2 = "[{\"color\": \"red\",\"v\": \"vv\"},{\"color\": \"red\",\"v\": \"vv\"}]";
    +
    +	@Test
    +	public void testOffset() throws IOException {
    +		InputStream jsonInputStream = createFromString(json2);
    --- End diff --
    
    +1


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by tzolov <gi...@git.apache.org>.
Github user tzolov commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51859447
  
    --- Diff: pxf/pxf-json/src/main/java/org/apache/hawq/pxf/plugins/json/parser/PartitionedJsonParser.java ---
    @@ -0,0 +1,198 @@
    +package org.apache.hawq.pxf.plugins.json.parser;
    +
    +/*
    + * 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.io.InputStream;
    +import java.io.InputStreamReader;
    +import java.nio.charset.StandardCharsets;
    +import java.util.ArrayList;
    +import java.util.EnumSet;
    +import java.util.List;
    +
    +import org.apache.hawq.pxf.plugins.json.parser.JsonLexer.JsonLexerState;
    +
    +/**
    + * A simple parser that can support reading JSON objects from a random point in JSON text. It reads from the supplied
    + * stream (which is assumed to be positioned at any arbitrary position inside some JSON text) until it find the first
    + * JSON begin-object "{". From this point on it will keep reading JSON objects until it finds one containing a member
    + * string that the user supplies.
    + * <p/>
    + * It is not recommended to use this with JSON text where individual JSON objects that can be large (MB's or larger).
    + */
    +public class PartitionedJsonParser {
    +
    +	private static final int EOF = -1;
    +	private final InputStreamReader inputStreamReader;
    +	private final JsonLexer lexer;
    +	private long bytesRead = 0;
    +	private boolean endOfStream;
    --- End diff --
    
    ok, will set it for consistency with the rest preinitialized attributes. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by hornn <gi...@git.apache.org>.
Github user hornn commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51836013
  
    --- Diff: pxf/pxf-json/src/test/java/org/apache/hawq/pxf/plugins/json/PxfUnit.java ---
    @@ -0,0 +1,672 @@
    +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.BufferedReader;
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.InputStreamReader;
    +import java.io.OutputStream;
    +import java.lang.reflect.Constructor;
    +import java.security.InvalidParameterException;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.fs.FileSystem;
    +import org.apache.hadoop.fs.Path;
    +import org.apache.hawq.pxf.api.Fragment;
    +import org.apache.hawq.pxf.api.Fragmenter;
    +import org.apache.hawq.pxf.api.OneField;
    +import org.apache.hawq.pxf.api.OneRow;
    +import org.apache.hawq.pxf.api.ReadAccessor;
    +import org.apache.hawq.pxf.api.ReadResolver;
    +import org.apache.hawq.pxf.api.WriteAccessor;
    +import org.apache.hawq.pxf.api.WriteResolver;
    +import org.apache.hawq.pxf.api.io.DataType;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +import org.apache.hawq.pxf.service.FragmentsResponse;
    --- End diff --
    
    This test file breaks the convention of separating between pxf-service and the plugins. Only pxf-api is supposed to be used by the plugins.
    I think it's a really good test mechanism, and we should use it, but it definitely should be moved to another location - perhaps its own package, perhaps under pxf-service.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in code base

Posted by GodenYao <gi...@git.apache.org>.
Github user GodenYao commented on the pull request:

    https://github.com/apache/incubator-hawq/pull/302
  
    @tzolov , can you close the PR.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by tzolov <gi...@git.apache.org>.
Github user tzolov commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r52077552
  
    --- Diff: pxf/pxf-json/src/test/java/org/apache/hawq/pxf/plugins/json/PxfUnit.java ---
    @@ -0,0 +1,672 @@
    +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.BufferedReader;
    +import java.io.ByteArrayOutputStream;
    +import java.io.IOException;
    +import java.io.InputStreamReader;
    +import java.io.OutputStream;
    +import java.lang.reflect.Constructor;
    +import java.security.InvalidParameterException;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.fs.FileSystem;
    +import org.apache.hadoop.fs.Path;
    +import org.apache.hawq.pxf.api.Fragment;
    +import org.apache.hawq.pxf.api.Fragmenter;
    +import org.apache.hawq.pxf.api.OneField;
    +import org.apache.hawq.pxf.api.OneRow;
    +import org.apache.hawq.pxf.api.ReadAccessor;
    +import org.apache.hawq.pxf.api.ReadResolver;
    +import org.apache.hawq.pxf.api.WriteAccessor;
    +import org.apache.hawq.pxf.api.WriteResolver;
    +import org.apache.hawq.pxf.api.io.DataType;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +import org.apache.hawq.pxf.service.FragmentsResponse;
    --- End diff --
    
    ok


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by hornn <gi...@git.apache.org>.
Github user hornn commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51210625
  
    --- Diff: pxf/pxf-json/src/main/java/org/apache/hawq/pxf/plugins/json/JsonInputFormat.java ---
    @@ -0,0 +1,226 @@
    +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.BufferedInputStream;
    +import java.io.IOException;
    +import java.security.InvalidParameterException;
    +
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.fs.FSDataInputStream;
    +import org.apache.hadoop.fs.FileSystem;
    +import org.apache.hadoop.fs.Path;
    +import org.apache.hadoop.io.LongWritable;
    +import org.apache.hadoop.io.NullWritable;
    +import org.apache.hadoop.io.Text;
    +import org.apache.hadoop.mapred.FileInputFormat;
    +import org.apache.hadoop.mapred.FileSplit;
    +import org.apache.hadoop.mapred.InputSplit;
    +import org.apache.hadoop.mapred.JobConf;
    +import org.apache.hadoop.mapred.LineRecordReader;
    +import org.apache.hadoop.mapred.RecordReader;
    +import org.apache.hadoop.mapred.Reporter;
    +import org.apache.log4j.Logger;
    +import org.codehaus.jackson.JsonFactory;
    +import org.codehaus.jackson.JsonNode;
    +import org.codehaus.jackson.JsonParseException;
    +import org.codehaus.jackson.map.JsonMappingException;
    +import org.codehaus.jackson.map.ObjectMapper;
    +
    +public class JsonInputFormat extends FileInputFormat<Text, NullWritable> {
    +
    +	private static JsonFactory factory = new JsonFactory();
    +	private static ObjectMapper mapper = new ObjectMapper(factory);
    +
    +	public static final String ONE_RECORD_PER_LINE = "json.input.format.one.record.per.line";
    +	public static final String RECORD_IDENTIFIER = "json.input.format.record.identifier";
    +
    +	@Override
    +	public RecordReader<Text, NullWritable> getRecordReader(InputSplit split, JobConf conf, Reporter reporter)
    +			throws IOException {
    +
    +		if (conf.getBoolean(ONE_RECORD_PER_LINE, false)) {
    +
    +			return new SimpleJsonRecordReader(conf, (FileSplit) split);
    +		} else {
    +			return new JsonRecordReader(conf, (FileSplit) split);
    +		}
    +	}
    +
    +	public static class SimpleJsonRecordReader implements RecordReader<Text, NullWritable> {
    +
    +		private LineRecordReader rdr = null;
    +		private LongWritable key = new LongWritable();
    +		private Text value = new Text();
    +
    +		public SimpleJsonRecordReader(Configuration conf, FileSplit split) throws IOException {
    +			rdr = new LineRecordReader(conf, split);
    +		}
    +
    +		@Override
    +		public void close() throws IOException {
    +			rdr.close();
    +		}
    +
    +		@Override
    +		public Text createKey() {
    +			return value;
    +		}
    +
    +		@Override
    +		public NullWritable createValue() {
    +			return NullWritable.get();
    +		}
    +
    +		@Override
    +		public long getPos() throws IOException {
    +			return rdr.getPos();
    +		}
    +
    +		@Override
    +		public boolean next(Text key, NullWritable value) throws IOException {
    +			if (rdr.next(this.key, this.value)) {
    +				key.set(this.value);
    +				return true;
    +			} else {
    +				return false;
    +			}
    +		}
    +
    +		@Override
    +		public float getProgress() throws IOException {
    +			return rdr.getProgress();
    +		}
    +	}
    +
    +	public static class JsonRecordReader implements RecordReader<Text, NullWritable> {
    +
    +		private Logger LOG = Logger.getLogger(JsonRecordReader.class);
    --- End diff --
    
    We use apache.commons.logging.Log in PXF:
    import org.apache.commons.logging.Log;
    import org.apache.commons.logging.LogFactory;
     private static final Log LOG = LogFactory.getLog(JsonRecordReader.class);



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by hornn <gi...@git.apache.org>.
Github user hornn commented on the pull request:

    https://github.com/apache/incubator-hawq/pull/302#issuecomment-178107477
  
    It sounds very similar to CSV with quoted data, which is not splittable. The way we do it today is by ensuring we process each file by a single accessor, even if it actually consists of multiple splits. (see HdfsTextMulti profile and [QuotedLineBreakAccessor](https://github.com/apache/incubator-hawq/blob/master/pxf/pxf-hdfs/src/main/java/org/apache/hawq/pxf/plugins/hdfs/QuotedLineBreakAccessor.java)). The problem, of course, is that we lose parallelism and performance.
    @tzolov, if it requires too much re-writing I agree that we can make it in two stages - first the splittable case (one record per line), and then the more complex cases.
    @adamjshook, good to hear from you :


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by tzolov <gi...@git.apache.org>.
Github user tzolov commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51492625
  
    --- Diff: pxf/pxf-service/src/main/resources/pxf-profiles-default.xml ---
    @@ -128,4 +128,26 @@ under the License.
                 <resolver>org.apache.hawq.pxf.plugins.gemfirexd.GemFireXDResolver</resolver>
             </plugins>
         </profile>
    +	<profile>
    +		<name>JSON</name>
    +		<description>A profile for JSON data, one JSON record per line
    +		</description>
    +		<plugins>
    +			<fragmenter>org.apache.hawq.pxf.plugins.hdfs.HdfsDataFragmenter</fragmenter>
    +			<accessor>org.apache.hawq.pxf.plugins.json.JsonAccessor</accessor>
    +			<resolver>org.apache.hawq.pxf.plugins.json.JsonResolver</resolver>
    +		</plugins>
    +	</profile>
    +	<profile>
    +		<name>JSON-PP</name>
    --- End diff --
    
    So Json it is


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by hornn <gi...@git.apache.org>.
Github user hornn commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51836238
  
    --- Diff: pxf/pxf-json/src/test/java/org/apache/hawq/pxf/plugins/json/parser/PartitionedJsonParserOffsetTest.java ---
    @@ -0,0 +1,54 @@
    +package org.apache.hawq.pxf.plugins.json.parser;
    +
    +/*
    + * 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 static org.junit.Assert.assertEquals;
    +import static org.junit.Assert.assertNotNull;
    +
    +import java.io.ByteArrayInputStream;
    +import java.io.IOException;
    +import java.io.InputStream;
    +
    +import org.junit.Test;
    +
    +public class PartitionedJsonParserOffsetTest {
    +	/*
    +	 * [{"color": "red","v": "vv"},{"color": "red","v": "vv"}]
    +	 */
    +	public static String json2 = "[{\"color\": \"red\",\"v\": \"vv\"},{\"color\": \"red\",\"v\": \"vv\"}]";
    +
    +	@Test
    +	public void testOffset() throws IOException {
    +		InputStream jsonInputStream = createFromString(json2);
    --- End diff --
    
    close stream


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by hornn <gi...@git.apache.org>.
Github user hornn commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51212745
  
    --- Diff: pxf/pxf-json/src/main/java/org/apache/hawq/pxf/plugins/json/JsonAccessor.java ---
    @@ -0,0 +1,66 @@
    +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 org.apache.commons.lang3.StringUtils;
    +import org.apache.hadoop.mapred.FileSplit;
    +import org.apache.hadoop.mapred.InputSplit;
    +import org.apache.hadoop.mapred.JobConf;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +import org.apache.hawq.pxf.plugins.hdfs.HdfsSplittableDataAccessor;
    +
    +/**
    + * This JSON accessor for PXF will read JSON data and pass it to a {@link JsonResolver}.
    + * 
    + * This accessor supports a single JSON record per line, or a more "pretty print" format.
    + */
    +public class JsonAccessor extends HdfsSplittableDataAccessor {
    +
    +	public static final String IDENTIFIER_PARAM = "IDENTIFIER";
    +	public static final String ONERECORDPERLINE_PARAM = "ONERECORDPERLINE";
    +
    +	private String identifier = "";
    +	private boolean oneRecordPerLine = true;
    +
    +	public JsonAccessor(InputData inputData) throws Exception {
    +		super(inputData, new JsonInputFormat());
    +
    +		if (!StringUtils.isEmpty(inputData.getUserProperty(IDENTIFIER_PARAM))) {
    +			identifier = inputData.getUserProperty(IDENTIFIER_PARAM);
    --- End diff --
    
    is it ok not to have an identifier? what happens if we leave it as ""?
    If it's an error, it should probably be handled here by throwing an exception.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by tzolov <gi...@git.apache.org>.
Github user tzolov commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51870529
  
    --- Diff: pxf/pxf-json/src/main/java/org/apache/hawq/pxf/plugins/json/JsonResolver.java ---
    @@ -0,0 +1,255 @@
    +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.util.ArrayList;
    +import java.util.Iterator;
    +import java.util.List;
    +
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +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.JsonFactory;
    +import org.codehaus.jackson.JsonNode;
    +import org.codehaus.jackson.map.ObjectMapper;
    +
    +/**
    + * 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 static final Log LOG = LogFactory.getLog(JsonResolver.class);
    +
    +	private ArrayList<OneField> oneFieldList;
    +	private ColumnDescriptorCache[] columnDescriptorCache;
    +	private ObjectMapper mapper;
    +
    +	public JsonResolver(InputData inputData) throws Exception {
    +		super(inputData);
    +		oneFieldList = new ArrayList<OneField>();
    +		mapper = new ObjectMapper(new JsonFactory());
    +
    +		// Pre-generate all column structure attributes concerning the JSON to column value resolution.
    +		columnDescriptorCache = new ColumnDescriptorCache[inputData.getColumns()];
    +		for (int i = 0; i < inputData.getColumns(); ++i) {
    +			ColumnDescriptor cd = inputData.getColumn(i);
    +			columnDescriptorCache[i] = new ColumnDescriptorCache(cd);
    +		}
    +	}
    +
    +	@Override
    +	public List<OneField> getFields(OneRow row) throws Exception {
    +		oneFieldList.clear();
    +
    +		String jsonRecordAsText = row.getData().toString();
    +
    +		JsonNode root = decodeLineToJsonNode(jsonRecordAsText);
    +
    +		if (root == null) {
    +			LOG.warn("Return null-fields row due to invalid JSON:" + jsonRecordAsText);
    +		}
    +
    +		// Iterate through the column definition and fetch our JSON data
    +		for (ColumnDescriptorCache column : columnDescriptorCache) {
    +
    +			// Get the current column description
    +
    +			if (root == null) {
    +				// Return empty (e.g. null) filed in case of malformed json.
    +				addNullField(column.getColumnType());
    +			} else {
    +
    +				// Move down the JSON path to the final name
    +				JsonNode node = getPriorJsonNode(root, column.getProjections());
    +
    +				// If this column is an array index, ex. "tweet.hashtags[0]"
    +				if (column.isArrayName()) {
    +
    +					// Move to the array node
    +					node = node.get(column.getArrayNodeName());
    +
    +					// If this node is null or missing, add a null value here
    +					if (node == null || node.isMissingNode()) {
    +						addNullField(column.getColumnType());
    +					} else if (node.isArray()) {
    +						// If the JSON node is an array, then add it to our list
    +						addFieldFromJsonArray(column.getColumnType(), node, column.getArrayIndex());
    +					} else {
    +						throw new IllegalStateException(column.getArrayNodeName() + " is not an array node");
    +					}
    +				} else {
    +					// This column is not an array type
    +					// Move to the final node
    +					node = node.get(column.getLastProjection());
    +
    +					// If this node is null or missing, add a null value here
    +					if (node == null || node.isMissingNode()) {
    +						addNullField(column.getColumnType());
    +					} else {
    +						// Else, add the value to the record
    +						addFieldFromJsonNode(column.getColumnType(), node);
    +					}
    +				}
    +			}
    +		}
    +
    +		return oneFieldList;
    +	}
    +
    +	/**
    +	 * Iterates down the root node to the child JSON node defined by the projs path.
    +	 * 
    +	 * @param root
    +	 *            node to to start the traversal from.
    +	 * @param projs
    +	 *            defines the path from the root to the desired child node.
    +	 * @return Returns the child node defined by the root and projs path.
    +	 */
    +	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) {
    --- End diff --
    
    Not it points to the element before the last. This is because the last element could either a simple node name or array reference (e.g. name[index]).  Later the JsornResovler#getFields handles both cases separately to resolve the last node element. 
    



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by hornn <gi...@git.apache.org>.
Github user hornn commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51209928
  
    --- Diff: pxf/pxf-service/src/main/resources/pxf-profiles-default.xml ---
    @@ -128,4 +128,26 @@ under the License.
                 <resolver>org.apache.hawq.pxf.plugins.gemfirexd.GemFireXDResolver</resolver>
             </plugins>
         </profile>
    +	<profile>
    +		<name>JSON</name>
    +		<description>A profile for JSON data, one JSON record per line
    +		</description>
    +		<plugins>
    +			<fragmenter>org.apache.hawq.pxf.plugins.hdfs.HdfsDataFragmenter</fragmenter>
    +			<accessor>org.apache.hawq.pxf.plugins.json.JsonAccessor</accessor>
    +			<resolver>org.apache.hawq.pxf.plugins.json.JsonResolver</resolver>
    +		</plugins>
    +	</profile>
    +	<profile>
    +		<name>JSON-PP</name>
    --- End diff --
    
    what does PP stand for?
    Other profiles use CamelCase as their naming standing, like HdfsTextSimple or HiveRc - perhaps change it to something like that?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by hornn <gi...@git.apache.org>.
Github user hornn commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51205768
  
    --- Diff: pxf/build.gradle ---
    @@ -286,6 +286,29 @@ project('pxf-hive') {
         }
     }
     
    +project('pxf-json') {
    +	dependencies {
    +		compile(project(':pxf-hdfs'))
    +		compile "org.apache.commons:commons-lang3:3.0"
    +
    +		testCompile 'pl.pragmatists:JUnitParams:1.0.2'
    +		configurations {
    +			// Remove hive-exec from unit tests as it causes VerifyError
    +			testRuntime.exclude module: 'hive-exec'
    --- End diff --
    
    is this necessary?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by hornn <gi...@git.apache.org>.
Github user hornn commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51506608
  
    --- Diff: pxf/pxf-json/src/main/java/org/apache/hawq/pxf/plugins/json/JsonStreamReader.java ---
    @@ -0,0 +1,121 @@
    +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.BufferedReader;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.io.InputStreamReader;
    +
    +/**
    + * Reads a JSON stream and sequentially extracts all JSON objects identified by the <b>identifier</> parameter. Returns
    + * null when there are no more objects to read.
    + *
    + */
    +public class JsonStreamReader extends BufferedReader {
    +
    +	private static char START_BRACE = '{';
    +	private static char END_BRACE = '}';
    +	private static int EOF = -1;
    +
    +	private StringBuilder bldr = new StringBuilder();
    +	private String identifier = null;
    +	private long bytesRead = 0;
    +
    +	/**
    +	 * @param identifier
    +	 *            JSON object name to extract.
    +	 * @param inputStream
    +	 *            JSON document input stream.
    +	 */
    +	public JsonStreamReader(String identifier, InputStream inputStream) {
    +		super(new InputStreamReader(inputStream));
    +		this.identifier = identifier;
    +	}
    +
    +	/**
    +	 * @return Returns next JSON object identified by the {@link JsonStreamReader#identifier} parameter or Null if no
    +	 *         more object are available.
    +	 * @throws IOException
    +	 */
    +	public String getJsonRecord() throws IOException {
    +		bldr.delete(0, bldr.length());
    +
    +		boolean foundRecord = false;
    +
    +		int c = 0, numBraces = 1;
    +		while ((c = super.read()) != EOF) {
    +			++bytesRead;
    +			if (!foundRecord) {
    +				bldr.append((char) c);
    +
    +				if (bldr.toString().contains(identifier)) {
    +					if (forwardToStartBrace()) {
    +						foundRecord = true;
    +
    +						bldr.delete(0, bldr.length());
    +						bldr.append(START_BRACE);
    +					}
    +				}
    +			} else {
    +				bldr.append((char) c);
    +
    +				if (c == START_BRACE) {
    +					++numBraces;
    +				} else if (c == END_BRACE) {
    +					--numBraces;
    +				}
    +
    +				if (numBraces == 0) {
    +					break;
    +				}
    +			}
    +		}
    +
    +		if (foundRecord && numBraces == 0) {
    +			return bldr.toString();
    +		} else {
    +			return null;
    --- End diff --
    
    if bldr is not empty, should we perhaps log a warning/error? 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by tzolov <gi...@git.apache.org>.
Github user tzolov commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51424582
  
    --- 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("]");
    --- End diff --
    
    will chanage the array name/index retrieval to to use the Pattern.compile("(.*)\\[([0-9]+)\\]") matching groups instead


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by hornn <gi...@git.apache.org>.
Github user hornn commented on the pull request:

    https://github.com/apache/incubator-hawq/pull/302#issuecomment-178308048
  
    Looks really good. I will try to test the plugins using your doc.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by sansanichfb <gi...@git.apache.org>.
Github user sansanichfb commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51211413
  
    --- Diff: pxf/pxf-json/src/test/java/org/apache/pxf/hawq/plugins/json/JsonExtensionTest.java ---
    @@ -0,0 +1,173 @@
    +package org.apache.pxf.hawq.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.util.ArrayList;
    +import java.util.List;
    +
    +import org.apache.hadoop.fs.Path;
    +import org.apache.hawq.pxf.api.Fragmenter;
    +import org.apache.hawq.pxf.api.ReadAccessor;
    +import org.apache.hawq.pxf.api.ReadResolver;
    +import org.apache.hawq.pxf.api.io.DataType;
    +import org.apache.hawq.pxf.plugins.hdfs.HdfsDataFragmenter;
    +import org.apache.hawq.pxf.plugins.json.JsonAccessor;
    +import org.apache.hawq.pxf.plugins.json.JsonResolver;
    +import org.junit.After;
    +import org.junit.Test;
    +
    +public class JsonExtensionTest extends PxfUnit {
    +
    +	private static List<Pair<String, DataType>> columnDefs = null;
    +	private static List<Pair<String, String>> extraParams = new ArrayList<Pair<String, String>>();
    +
    +	static {
    +
    +		columnDefs = new ArrayList<Pair<String, DataType>>();
    +
    +		columnDefs.add(new Pair<String, DataType>("created_at", DataType.TEXT));
    +		columnDefs.add(new Pair<String, DataType>("id", DataType.BIGINT));
    +		columnDefs.add(new Pair<String, DataType>("text", DataType.TEXT));
    +		columnDefs.add(new Pair<String, DataType>("user.screen_name", DataType.TEXT));
    +		columnDefs.add(new Pair<String, DataType>("entities.hashtags[0]", DataType.TEXT));
    +		columnDefs.add(new Pair<String, DataType>("coordinates.coordinates[0]", DataType.FLOAT8));
    +		columnDefs.add(new Pair<String, DataType>("coordinates.coordinates[1]", DataType.FLOAT8));
    +	}
    +
    +	@After
    +	public void cleanup() throws Exception {
    +		extraParams.clear();
    +	}
    +
    +	@Test
    +	public void testSmallTweets() throws Exception {
    +
    +		List<String> output = new ArrayList<String>();
    +
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547115253761,REPAIR THE TRUST: REMOVE OBAMA/BIDEN FROM OFFICE. #IRS #DOJ #NSA #tcot,SpreadButter,tweetCongress,,");
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547123646465,@marshafitrie dibagi 1000 aja sha :P,patronusdeadly,,,");
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547136233472,Vaga: Supervisor de Almoxarifado. Confira em http://t.co/hK5cy5B2oS,NoSecrets_Vagas,,,");
    +		output.add("Fri Jun 07 22:45:03 +0000 2013,343136551322136576,It's Jun 7, 2013 @ 11pm ; Wind = NNE (30,0) 14.0 knots; Swell = 2.6 ft @ 5 seconds....,SevenStonesBuoy,,-6.1,50.103");
    +
    +		super.assertOutput(new Path(System.getProperty("user.dir") + "/" + "src/test/resources/tweets-small.json"),
    --- End diff --
    
    Probably we can use File.separator instead of "/".


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-hawq pull request: HAWQ-178: Add JSON plugin support in ...

Posted by tzolov <gi...@git.apache.org>.
Github user tzolov commented on a diff in the pull request:

    https://github.com/apache/incubator-hawq/pull/302#discussion_r51860718
  
    --- Diff: pxf/pxf-json/src/test/java/org/apache/hawq/pxf/plugins/json/JsonExtensionTest.java ---
    @@ -0,0 +1,288 @@
    +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.File;
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import org.apache.hadoop.fs.Path;
    +import org.apache.hawq.pxf.api.Fragmenter;
    +import org.apache.hawq.pxf.api.ReadAccessor;
    +import org.apache.hawq.pxf.api.ReadResolver;
    +import org.apache.hawq.pxf.api.io.DataType;
    +import org.apache.hawq.pxf.plugins.hdfs.HdfsDataFragmenter;
    +import org.apache.hawq.pxf.plugins.json.JsonAccessor;
    +import org.apache.hawq.pxf.plugins.json.JsonResolver;
    +import org.junit.After;
    +import org.junit.Before;
    +import org.junit.Test;
    +
    +public class JsonExtensionTest extends PxfUnit {
    +
    +	private List<Pair<String, DataType>> columnDefs = null;
    +	private List<Pair<String, String>> extraParams = new ArrayList<Pair<String, String>>();
    +
    +	@Before
    +	public void before() {
    +
    +		columnDefs = new ArrayList<Pair<String, DataType>>();
    +
    +		columnDefs.add(new Pair<String, DataType>("created_at", DataType.TEXT));
    +		columnDefs.add(new Pair<String, DataType>("id", DataType.BIGINT));
    +		columnDefs.add(new Pair<String, DataType>("text", DataType.TEXT));
    +		columnDefs.add(new Pair<String, DataType>("user.screen_name", DataType.TEXT));
    +		columnDefs.add(new Pair<String, DataType>("entities.hashtags[0]", DataType.TEXT));
    +		columnDefs.add(new Pair<String, DataType>("coordinates.coordinates[0]", DataType.FLOAT8));
    +		columnDefs.add(new Pair<String, DataType>("coordinates.coordinates[1]", DataType.FLOAT8));
    +	}
    +
    +	@After
    +	public void cleanup() throws Exception {
    +		columnDefs.clear();
    +		extraParams.clear();
    +	}
    +
    +	@Test
    +	public void testCompressedMultilineJsonFile() throws Exception {
    +
    +		extraParams.add(new Pair<String, String>("IDENTIFIER", "created_at"));
    +
    +		List<String> output = new ArrayList<String>();
    +
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547115253761,text1,SpreadButter,tweetCongress,,");
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547123646465,text2,patronusdeadly,,,");
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547136233472,text3,NoSecrets_Vagas,,,");
    +
    +		super.assertOutput(new Path(System.getProperty("user.dir") + File.separator
    +				+ "src/test/resources/tweets.tar.gz"), output);
    +	}
    +
    +	@Test
    +	public void testMaxRecordLength() throws Exception {
    +
    +		// variable-size-objects.json contains 3 json objects but only 2 of them fit in the 27 byte length limitation
    +
    +		extraParams.add(new Pair<String, String>("IDENTIFIER", "key666"));
    +		extraParams.add(new Pair<String, String>("MAXLENGTH", "27"));
    +
    +		columnDefs.clear();
    +		columnDefs.add(new Pair<String, DataType>("key666", DataType.TEXT));
    +
    +		List<String> output = new ArrayList<String>();
    +
    +		output.add("small object1");
    +		// skip the large object2 XXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXXX
    +		output.add("small object3");
    +
    +		super.assertOutput(new Path(System.getProperty("user.dir") + File.separator
    +				+ "src/test/resources/variable-size-objects.json"), output);
    +	}
    +
    +	@Test
    +	public void testDataTypes() throws Exception {
    +
    +		// TDOO: The BYTEA type is not tested!!! Current oneField.val = val.asText().getBytes(); convention is
    +		// disputable!
    +
    +		extraParams.clear();
    +		extraParams.add(new Pair<String, String>("IDENTIFIER", "bintType"));
    +
    +		columnDefs.clear();
    +		columnDefs.add(new Pair<String, DataType>("bintType", DataType.BIGINT));
    +		columnDefs.add(new Pair<String, DataType>("booleanType", DataType.BOOLEAN));
    +		columnDefs.add(new Pair<String, DataType>("charType", DataType.CHAR));
    +		// columnDefs.add(new Pair<String, DataType>("byteaType", DataType.BYTEA));
    +		columnDefs.add(new Pair<String, DataType>("float8Type", DataType.FLOAT8));
    +		columnDefs.add(new Pair<String, DataType>("realType", DataType.REAL));
    +		columnDefs.add(new Pair<String, DataType>("integerType", DataType.INTEGER));
    +		columnDefs.add(new Pair<String, DataType>("smallintType", DataType.SMALLINT));
    +		columnDefs.add(new Pair<String, DataType>("bpcharType", DataType.BPCHAR));
    +		columnDefs.add(new Pair<String, DataType>("varcharType", DataType.VARCHAR));
    +		columnDefs.add(new Pair<String, DataType>("text", DataType.TEXT));
    +
    +		List<String> output = new ArrayList<String>();
    +
    +		output.add("666,true,x,3.14,3.15,999,777,bpcharType,varcharType,");
    +
    +		super.assertOutput(new Path(System.getProperty("user.dir") + File.separator
    +				+ "src/test/resources/datatypes-test.json"), output);
    +	}
    +
    +	@Test(expected = IllegalStateException.class)
    +	public void testMissingArrayJsonAttribute() throws Exception {
    +
    +		extraParams.add(new Pair<String, String>("IDENTIFIER", "created_at"));
    +
    +		columnDefs.clear();
    +
    +		columnDefs.add(new Pair<String, DataType>("created_at", DataType.TEXT));
    +		// User is not an array! An attempt to access it should throw an exception!
    +		columnDefs.add(new Pair<String, DataType>("user[0]", DataType.TEXT));
    +
    +		List<String> output = new ArrayList<String>();
    +
    +		super.assertOutput(new Path(System.getProperty("user.dir") + File.separator
    +				+ "src/test/resources/tweets-with-missing-text-attribtute.json"), output);
    +	}
    +
    +	@Test
    +	public void testMissingJsonAttribute() throws Exception {
    +
    +		List<String> output = new ArrayList<String>();
    +
    +		extraParams.add(new Pair<String, String>("IDENTIFIER", "created_at"));
    +
    +		// Missing attributes are substituted by an empty field
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547115253761,,SpreadButter,tweetCongress,,");
    +
    +		super.assertOutput(new Path(System.getProperty("user.dir") + File.separator
    +				+ "src/test/resources/tweets-with-missing-text-attribtute.json"), output);
    +	}
    +
    +	@Test
    +	public void testMalformedJsonObject() throws Exception {
    +
    +		List<String> output = new ArrayList<String>();
    +
    +		extraParams.add(new Pair<String, String>("IDENTIFIER", "created_at"));
    +
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547115253761,text1,SpreadButter,tweetCongress,,");
    +		output.add(",,,,,,"); // Expected: malformed json records are transformed into empty rows
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547136233472,text3,NoSecrets_Vagas,,,");
    +
    +		super.assertOutput(new Path(System.getProperty("user.dir") + File.separator
    +				+ "src/test/resources/tweets-broken.json"), output);
    +	}
    +
    +	@Test
    +	public void testSmallTweets() throws Exception {
    +
    +		List<String> output = new ArrayList<String>();
    +
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547115253761,text1,SpreadButter,tweetCongress,,");
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547123646465,text2,patronusdeadly,,,");
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547136233472,text3,NoSecrets_Vagas,,,");
    +		output.add("Fri Jun 07 22:45:03 +0000 2013,343136551322136576,text4,SevenStonesBuoy,,-6.1,50.103");
    +
    +		super.assertOutput(new Path(System.getProperty("user.dir") + File.separator
    +				+ "src/test/resources/tweets-small.json"), output);
    +	}
    +
    +	@Test
    +	public void testTweetsWithNull() throws Exception {
    +
    +		List<String> output = new ArrayList<String>();
    +
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,,text1,SpreadButter,tweetCongress,,");
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,,text2,patronusdeadly,,,");
    +
    +		super.assertOutput(new Path(System.getProperty("user.dir") + File.separator
    +				+ "src/test/resources/null-tweets.json"), output);
    +	}
    +
    +	@Test
    +	public void testSmallTweetsWithDelete() throws Exception {
    +
    +		List<String> output = new ArrayList<String>();
    +
    +		output.add(",,,,,,");
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547115253761,text1,SpreadButter,tweetCongress,,");
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547123646465,text2,patronusdeadly,,,");
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547136233472,text3,NoSecrets_Vagas,,,");
    +
    +		super.assertOutput(new Path(System.getProperty("user.dir") + File.separator
    +				+ "src/test/resources/tweets-small-with-delete.json"), output);
    +	}
    +
    +	@Test
    +	public void testWellFormedJson() throws Exception {
    +
    +		extraParams.add(new Pair<String, String>("IDENTIFIER", "created_at"));
    +
    +		List<String> output = new ArrayList<String>();
    +
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547115253761,text1,SpreadButter,tweetCongress,,");
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547123646465,text2,patronusdeadly,,,");
    +		output.add("Fri Jun 07 22:45:02 +0000 2013,343136547136233472,text3,NoSecrets_Vagas,,,");
    +
    +		super.assertOutput(new Path(System.getProperty("user.dir") + File.separator
    +				+ "src/test/resources/tweets-pp.json"), output);
    +	}
    +
    +	@Test
    +	public void testWellFormedJsonWithDelete() throws Exception {
    +
    +		extraParams.add(new Pair<String, String>("IDENTIFIER", "created_at"));
    +
    +		List<String> output = new ArrayList<String>();
    +
    +		// output.add(",,,,,,");
    --- End diff --
    
    I am not sure what was the purpose of the deleted json records in the original pxf-json code. Perhaps @adamjshook would remember? 
    But i've kept the test because it shows that the identifier will filter out the records not containing the identifier (nothing to do with deletion though).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---