You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hawq.apache.org by shivzone <gi...@git.apache.org> on 2016/09/09 00:11:39 UTC

[GitHub] incubator-hawq pull request #895: HAWQ 971

GitHub user shivzone opened a pull request:

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

    HAWQ 971

    This is based on the earlier pull request for HAWQ 931 and includes updates to return null values for columns not requested in the query

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

    $ git pull https://github.com/shivzone/incubator-hawq HAWQ-971

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

    https://github.com/apache/incubator-hawq/pull/895.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 #895
    
----
commit 3c23716c9560f3584a2c2c24523494e3be713ab6
Author: Shivram Mani <sh...@gmail.com>
Date:   2016-07-28T00:48:59Z

    HAWQ-931. ORC optimized profile for PPD/CP

commit a22602b20509d8ac44657a725ec4e8ace58aab2a
Author: Shivram Mani <sh...@gmail.com>
Date:   2016-07-28T20:57:18Z

    HAWQ-931. CR Fixes

commit 3dda224c86bee8fba48ae16004da7e6592fbba05
Author: Shivram Mani <sh...@gmail.com>
Date:   2016-07-30T00:09:10Z

    HAWQ-931. CR Fixes. Take 2

commit 9c157419d0d215c51a6950c8de94e0e1c3bd9d13
Author: Shivram Mani <sh...@gmail.com>
Date:   2016-08-02T21:57:56Z

    HAWQ-931. Removed redundant function overrides

commit 71fd7a7a0d06ebf4dc3fcf87c2015bb33b3669bd
Author: Shivram Mani <sh...@gmail.com>
Date:   2016-08-02T22:07:14Z

    HAWQ-971. Updated Hive Resolver to return partial content

commit 0357f14051c4c73f727061295390e26d5f1ffcae
Author: Shivram Mani <sh...@gmail.com>
Date:   2016-07-28T00:48:59Z

    HAWQ-931. ORC optimized profile for PPD/CP

commit ce94e7720ce97cbb2420ece97e10bc1f699d3006
Author: Shivram Mani <sh...@gmail.com>
Date:   2016-07-28T20:57:18Z

    HAWQ-931. CR Fixes

commit fd058491fce01df2e24fe00619e7b5a9aa97f5e7
Author: Shivram Mani <sh...@gmail.com>
Date:   2016-07-30T00:09:10Z

    HAWQ-931. CR Fixes. Take 2

commit 20582fde866a9e63379bb48aeef6665aa867ad6c
Author: Shivram Mani <sh...@gmail.com>
Date:   2016-08-02T21:57:56Z

    HAWQ-931. Removed redundant function overrides

commit cb978c44ccfce4a316d67ebec96eb52d5b33e574
Author: Shivram Mani <sh...@gmail.com>
Date:   2016-08-02T22:07:14Z

    HAWQ-971. Updated Hive Resolver to return partial content

commit 67005d54a4e196a656614351487a6020c41d3705
Author: Shivram Mani <sh...@gmail.com>
Date:   2016-08-13T02:04:36Z

    HAWQ-971. ORC resolver fixes and support for additional types

commit 420eee10e9a8af82e130a18979aaf6b262e947d1
Author: Shivram Mani <sh...@gmail.com>
Date:   2016-08-13T02:07:43Z

    HAWQ-971. ORC resolver fixes and support for additional types

commit 5f764886b98d3eda3a86976ccd8f73f09f7e46c0
Author: Shivram Mani <sh...@gmail.com>
Date:   2016-09-08T20:55:50Z

    HAWQ-931. CR Fixes

commit 75256e1590b5054a6b873ddb12f0846c805bdf6b
Author: Shivram Mani <sh...@gmail.com>
Date:   2016-09-09T00:01:57Z

    HAWQ-971. CR Fixes

commit 9bf29d318087d0bdceeb6e39d44a3bf5efeecf2e
Author: Shivram Mani <sh...@gmail.com>
Date:   2016-09-09T00:10:08Z

    HAWQ-971. CR Fixes

----


---
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 #895: HAWQ 971

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/895#discussion_r78232492
  
    --- Diff: pxf/pxf-hive/src/main/java/org/apache/hawq/pxf/plugins/hive/HiveORCSerdeResolver.java ---
    @@ -0,0 +1,150 @@
    +package org.apache.hawq.pxf.plugins.hive;
    +
    +/*
    + * 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.lang.CharUtils;
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.hive.common.type.HiveDecimal;
    +import org.apache.hadoop.hive.ql.io.orc.OrcSerde;
    +import org.apache.hadoop.hive.serde.serdeConstants;
    +import org.apache.hadoop.hive.serde2.objectinspector.*;
    +import org.apache.hadoop.hive.serde2.objectinspector.primitive.*;
    +import org.apache.hadoop.io.BytesWritable;
    +import org.apache.hadoop.io.Writable;
    +import org.apache.hadoop.mapred.JobConf;
    +import org.apache.hawq.pxf.api.BadRecordException;
    +import org.apache.hawq.pxf.api.OneField;
    +import org.apache.hawq.pxf.api.OneRow;
    +import org.apache.hawq.pxf.api.UnsupportedTypeException;
    +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.Utilities;
    +import org.apache.hawq.pxf.plugins.hdfs.utilities.HdfsUtilities;
    +import org.apache.hawq.pxf.plugins.hive.utilities.HiveUtilities;
    +
    +import java.io.IOException;
    +import java.math.BigDecimal;
    +import java.util.*;
    +
    +import static org.apache.hawq.pxf.api.io.DataType.*;
    +import static org.apache.hawq.pxf.api.io.DataType.DATE;
    +import static org.apache.hawq.pxf.api.io.DataType.SMALLINT;
    +
    +/**
    + * Specialized HiveResolver for a Hive table stored as RC file.
    + * Use together with HiveInputFormatFragmenter/HiveRCFileAccessor.
    + */
    +public class HiveORCSerdeResolver extends HiveResolver {
    +    private static final Log LOG = LogFactory.getLog(HiveORCSerdeResolver.class);
    +    private OrcSerde deserializer;
    +    private boolean firstColumn;
    +    private StringBuilder builder;
    +    private StringBuilder parts;
    +    private int numberOfPartitions;
    +    private HiveInputFormatFragmenter.PXF_HIVE_SERDES serdeType;
    +
    +    public HiveORCSerdeResolver(InputData input) throws Exception {
    +        super(input);
    +    }
    +
    +    /* read the data supplied by the fragmenter: inputformat name, serde name, partition keys */
    +    @Override
    +    void parseUserData(InputData input) throws Exception {
    +        String[] toks = HiveInputFormatFragmenter.parseToks(input);
    +        String serdeEnumStr = toks[HiveInputFormatFragmenter.TOK_SERDE];
    +        if (serdeEnumStr.equals(HiveInputFormatFragmenter.PXF_HIVE_SERDES.ORC_SERDE.name())) {
    +            serdeType = HiveInputFormatFragmenter.PXF_HIVE_SERDES.ORC_SERDE;
    +        } else {
    +            throw new UnsupportedTypeException("Unsupported Hive Serde: " + serdeEnumStr);
    +        }
    +        partitionKeys = toks[HiveInputFormatFragmenter.TOK_KEYS];
    +        parseDelimiterChar(input);
    +        collectionDelim = input.getUserProperty("COLLECTION_DELIM") == null ? COLLECTION_DELIM
    +                : input.getUserProperty("COLLECTION_DELIM");
    +        mapkeyDelim = input.getUserProperty("MAPKEY_DELIM") == null ? MAPKEY_DELIM
    +                : input.getUserProperty("MAPKEY_DELIM");
    +    }
    +
    +    @Override
    +    void initPartitionFields() {
    +        parts = new StringBuilder();
    +        numberOfPartitions = initPartitionFields(parts);
    +    }
    +
    +    /**
    +     * getFields returns a singleton list of OneField item.
    +     * OneField item contains two fields: an integer representing the VARCHAR type and a Java
    +     * Object representing the field value.
    +     */
    +    @Override
    +    public List<OneField> getFields(OneRow onerow) throws Exception {
    +
    +        Object tuple = deserializer.deserialize((Writable) onerow.getData());
    +        // Each Hive record is a Struct
    +        StructObjectInspector soi = (StructObjectInspector) deserializer.getObjectInspector();
    +        List<OneField> record = traverseStruct(tuple, soi, false);
    +
    +        return record;
    +
    +    }
    +
    +    /*
    +     * Get and init the deserializer for the records of this Hive data fragment.
    +     * Suppress Warnings added because deserializer.initialize is an abstract function that is deprecated
    +     * but its implementations (ColumnarSerDe, LazyBinaryColumnarSerDe) still use the deprecated interface.
    +     */
    +    @SuppressWarnings("deprecation")
    +	@Override
    --- End diff --
    
    Identation


---
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 #895: HAWQ 971

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/895#discussion_r78231944
  
    --- Diff: pxf/pxf-hive/src/main/java/org/apache/hawq/pxf/plugins/hive/HiveORCSerdeResolver.java ---
    @@ -0,0 +1,150 @@
    +package org.apache.hawq.pxf.plugins.hive;
    +
    +/*
    + * 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.lang.CharUtils;
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.hive.common.type.HiveDecimal;
    +import org.apache.hadoop.hive.ql.io.orc.OrcSerde;
    +import org.apache.hadoop.hive.serde.serdeConstants;
    +import org.apache.hadoop.hive.serde2.objectinspector.*;
    +import org.apache.hadoop.hive.serde2.objectinspector.primitive.*;
    +import org.apache.hadoop.io.BytesWritable;
    +import org.apache.hadoop.io.Writable;
    +import org.apache.hadoop.mapred.JobConf;
    +import org.apache.hawq.pxf.api.BadRecordException;
    +import org.apache.hawq.pxf.api.OneField;
    +import org.apache.hawq.pxf.api.OneRow;
    +import org.apache.hawq.pxf.api.UnsupportedTypeException;
    +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.Utilities;
    +import org.apache.hawq.pxf.plugins.hdfs.utilities.HdfsUtilities;
    +import org.apache.hawq.pxf.plugins.hive.utilities.HiveUtilities;
    +
    +import java.io.IOException;
    +import java.math.BigDecimal;
    +import java.util.*;
    +
    +import static org.apache.hawq.pxf.api.io.DataType.*;
    +import static org.apache.hawq.pxf.api.io.DataType.DATE;
    +import static org.apache.hawq.pxf.api.io.DataType.SMALLINT;
    +
    +/**
    + * Specialized HiveResolver for a Hive table stored as RC file.
    + * Use together with HiveInputFormatFragmenter/HiveRCFileAccessor.
    + */
    +public class HiveORCSerdeResolver extends HiveResolver {
    +    private static final Log LOG = LogFactory.getLog(HiveORCSerdeResolver.class);
    +    private OrcSerde deserializer;
    +    private boolean firstColumn;
    +    private StringBuilder builder;
    --- End diff --
    
    Why do we need this?


---
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 #895: HAWQ 971

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

    https://github.com/apache/incubator-hawq/pull/895#discussion_r78460753
  
    --- Diff: pxf/pxf-hive/src/main/java/org/apache/hawq/pxf/plugins/hive/HiveORCAccessor.java ---
    @@ -0,0 +1,150 @@
    +package org.apache.hawq.pxf.plugins.hive;
    +
    +/*
    + * 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.hadoop.hive.ql.io.orc.OrcInputFormat;
    +import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
    +import org.apache.hadoop.hive.ql.io.sarg.SearchArgumentFactory;
    +import org.apache.hadoop.mapred.InputSplit;
    +import org.apache.hadoop.mapred.JobConf;
    +import org.apache.hadoop.mapred.Reporter;
    +import org.apache.hawq.pxf.api.FilterParser;
    +import org.apache.hawq.pxf.api.utilities.ColumnDescriptor;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +import org.apache.commons.lang.StringUtils;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import static org.apache.hawq.pxf.plugins.hive.HiveInputFormatFragmenter.PXF_HIVE_SERDES;
    +
    +/**
    + * Specialization of HiveAccessor for a Hive table that stores only ORC files.
    + * This class replaces the generic HiveAccessor for a case where a table is stored entirely as ORC files.
    + * Use together with {@link HiveInputFormatFragmenter}/{@link HiveColumnarSerdeResolver}
    + */
    +public class HiveORCAccessor extends HiveAccessor {
    +
    +    private final String READ_COLUMN_IDS_CONF_STR = "hive.io.file.readcolumn.ids";
    +    private final String READ_ALL_COLUMNS = "hive.io.file.read.all.columns";
    +    private final String READ_COLUMN_NAMES_CONF_STR = "hive.io.file.readcolumn.names";
    +    private final String SARG_PUSHDOWN = "sarg.pushdown";
    +
    +    /**
    +     * Constructs a HiveORCFileAccessor.
    +     *
    +     * @param input input containing user data
    +     * @throws Exception if user data was wrong
    +     */
    +    public HiveORCAccessor(InputData input) throws Exception {
    +        super(input, new OrcInputFormat());
    +        String[] toks = HiveInputFormatFragmenter.parseToks(input, PXF_HIVE_SERDES.ORC_SERDE.name());
    +        initPartitionFields(toks[HiveInputFormatFragmenter.TOK_KEYS]);
    +        filterInFragmenter = new Boolean(toks[HiveInputFormatFragmenter.TOK_FILTER_DONE]);
    +    }
    +
    +    @Override
    +    public boolean openForRead() throws Exception {
    +        addColumns();
    +        addFilters();
    +        return super.openForRead();
    +    }
    +
    +    /**
    +     * Adds the table tuple description to JobConf ojbect
    +     * so only these columns will be returned.
    +     */
    +    private void addColumns() throws Exception {
    +
    +        List<Integer> colIds = new ArrayList<Integer>();
    +        List<String> colNames = new ArrayList<String>();
    +        for(ColumnDescriptor col: inputData.getTupleDescription()) {
    +            if(col.isProjected()) {
    +                colIds.add(col.columnIndex());
    +                colNames.add(col.columnName());
    +            }
    +        }
    +        jobConf.set(READ_ALL_COLUMNS, "false");
    +        jobConf.set(READ_COLUMN_IDS_CONF_STR, StringUtils.join(colIds, ","));
    +        jobConf.set(READ_COLUMN_NAMES_CONF_STR, StringUtils.join(colNames, ","));
    +    }
    +
    +    /**
    +     * Uses {@link HiveFilterBuilder} to translate a filter string into a
    +     * Hive {@link SearchArgument} object. The result is added as a filter to
    +     * JobConf object
    +     */
    +    private void addFilters() throws Exception {
    +        if (!inputData.hasFilter()) {
    +            return;
    +        }
    +
    +        /* Predicate pushdown configuration */
    +        String filterStr = inputData.getFilterString();
    +        HiveFilterBuilder eval = new HiveFilterBuilder(inputData);
    +        Object filter = eval.getFilterObject(filterStr);
    +
    +        SearchArgument.Builder filterBuilder = SearchArgumentFactory.newBuilder();
    +        filterBuilder.startAnd();
    +        if (filter instanceof List) {
    +            for (Object f : (List<?>) filter) {
    +                buildArgument(filterBuilder, f);
    +            }
    +        } else {
    +            buildArgument(filterBuilder, filter);
    +        }
    +        filterBuilder.end();
    +        SearchArgument sarg = filterBuilder.build();
    +        jobConf.set(SARG_PUSHDOWN, sarg.toKryo());
    +    }
    +
    +    private void buildArgument(SearchArgument.Builder builder, Object filterObj) {
    +        /* The below functions will not be compatible and requires update  with Hive 2.0 APIs */
    +        FilterParser.BasicFilter filter = (FilterParser.BasicFilter) filterObj;
    +        int filterColumnIndex = filter.getColumn().index();
    +        Object filterValue = filter.getConstant().constant();
    +        ColumnDescriptor filterColumn = inputData.getColumn(filterColumnIndex);
    +        String filterColumnName = filterColumn.columnName();
    +
    +        switch(filter.getOperation()) {
    --- End diff --
    
    On the Java side we are yet to make the update to support both AND and OR logical operators. The cases within buildArgument are strictly only for individual operators.
    We can work on the update to support both AND/OR as a separate feature/pull request..


---
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 #895: HAWQ 971

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/895#discussion_r78432848
  
    --- Diff: pxf/pxf-hive/src/main/java/org/apache/hawq/pxf/plugins/hive/utilities/EnumHiveToHawqType.java ---
    @@ -152,6 +153,33 @@ public int compare(EnumHiveToHawqType a,
         }
     
         /**
    +     *
    +     * @return full type name including modifiers. eg: varchar(3)
    --- End diff --
    
    Hive or HAWQ type? I would mention that in JavaDoc and method name.


---
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 #895: HAWQ 971

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/895#discussion_r78231431
  
    --- Diff: pxf/pxf-hive/src/main/java/org/apache/hawq/pxf/plugins/hive/HiveORCAccessor.java ---
    @@ -0,0 +1,150 @@
    +package org.apache.hawq.pxf.plugins.hive;
    +
    +/*
    + * 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.hadoop.hive.ql.io.orc.OrcInputFormat;
    +import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
    +import org.apache.hadoop.hive.ql.io.sarg.SearchArgumentFactory;
    +import org.apache.hadoop.mapred.InputSplit;
    +import org.apache.hadoop.mapred.JobConf;
    +import org.apache.hadoop.mapred.Reporter;
    +import org.apache.hawq.pxf.api.FilterParser;
    +import org.apache.hawq.pxf.api.utilities.ColumnDescriptor;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +import org.apache.commons.lang.StringUtils;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import static org.apache.hawq.pxf.plugins.hive.HiveInputFormatFragmenter.PXF_HIVE_SERDES;
    +
    +/**
    + * Specialization of HiveAccessor for a Hive table that stores only ORC files.
    + * This class replaces the generic HiveAccessor for a case where a table is stored entirely as ORC files.
    + * Use together with {@link HiveInputFormatFragmenter}/{@link HiveColumnarSerdeResolver}
    + */
    +public class HiveORCAccessor extends HiveAccessor {
    +
    +    private final String READ_COLUMN_IDS_CONF_STR = "hive.io.file.readcolumn.ids";
    +    private final String READ_ALL_COLUMNS = "hive.io.file.read.all.columns";
    +    private final String READ_COLUMN_NAMES_CONF_STR = "hive.io.file.readcolumn.names";
    +    private final String SARG_PUSHDOWN = "sarg.pushdown";
    +
    +    /**
    +     * Constructs a HiveORCFileAccessor.
    +     *
    +     * @param input input containing user data
    +     * @throws Exception if user data was wrong
    +     */
    +    public HiveORCAccessor(InputData input) throws Exception {
    +        super(input, new OrcInputFormat());
    +        String[] toks = HiveInputFormatFragmenter.parseToks(input, PXF_HIVE_SERDES.ORC_SERDE.name());
    +        initPartitionFields(toks[HiveInputFormatFragmenter.TOK_KEYS]);
    +        filterInFragmenter = new Boolean(toks[HiveInputFormatFragmenter.TOK_FILTER_DONE]);
    +    }
    +
    +    @Override
    +    public boolean openForRead() throws Exception {
    +        addColumns();
    +        addFilters();
    +        return super.openForRead();
    +    }
    +
    +    /**
    +     * Adds the table tuple description to JobConf ojbect
    +     * so only these columns will be returned.
    +     */
    +    private void addColumns() throws Exception {
    +
    +        List<Integer> colIds = new ArrayList<Integer>();
    +        List<String> colNames = new ArrayList<String>();
    +        for(ColumnDescriptor col: inputData.getTupleDescription()) {
    +            if(col.isProjected()) {
    +                colIds.add(col.columnIndex());
    +                colNames.add(col.columnName());
    +            }
    +        }
    +        jobConf.set(READ_ALL_COLUMNS, "false");
    +        jobConf.set(READ_COLUMN_IDS_CONF_STR, StringUtils.join(colIds, ","));
    +        jobConf.set(READ_COLUMN_NAMES_CONF_STR, StringUtils.join(colNames, ","));
    +    }
    +
    +    /**
    +     * Uses {@link HiveFilterBuilder} to translate a filter string into a
    +     * Hive {@link SearchArgument} object. The result is added as a filter to
    +     * JobConf object
    +     */
    +    private void addFilters() throws Exception {
    +        if (!inputData.hasFilter()) {
    +            return;
    +        }
    +
    +        /* Predicate pushdown configuration */
    +        String filterStr = inputData.getFilterString();
    +        HiveFilterBuilder eval = new HiveFilterBuilder(inputData);
    +        Object filter = eval.getFilterObject(filterStr);
    +
    +        SearchArgument.Builder filterBuilder = SearchArgumentFactory.newBuilder();
    +        filterBuilder.startAnd();
    +        if (filter instanceof List) {
    +            for (Object f : (List<?>) filter) {
    +                buildArgument(filterBuilder, f);
    +            }
    +        } else {
    +            buildArgument(filterBuilder, filter);
    +        }
    +        filterBuilder.end();
    +        SearchArgument sarg = filterBuilder.build();
    +        jobConf.set(SARG_PUSHDOWN, sarg.toKryo());
    +    }
    +
    +    private void buildArgument(SearchArgument.Builder builder, Object filterObj) {
    +        /* The below functions will not be compatible and requires update  with Hive 2.0 APIs */
    +        FilterParser.BasicFilter filter = (FilterParser.BasicFilter) filterObj;
    +        int filterColumnIndex = filter.getColumn().index();
    +        Object filterValue = filter.getConstant().constant();
    +        ColumnDescriptor filterColumn = inputData.getColumn(filterColumnIndex);
    +        String filterColumnName = filterColumn.columnName();
    +
    +        switch(filter.getOperation()) {
    --- End diff --
    
    Should we also support HDOP_AND operation? Looks like Builder has method startAnd().


---
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 #895: HAWQ 971

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

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


---
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 #895: HAWQ 971

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/895#discussion_r78112412
  
    --- Diff: src/backend/access/external/pxfheaders.c ---
    @@ -165,6 +167,18 @@ static void add_tuple_desc_httpheader(CHURL_HEADERS headers, Relation rel)
             /* Add a key/value pair for attribute type name */
             resetStringInfo(&formatter);
             appendStringInfo(&formatter, "X-GP-ATTR-TYPENAME%u", i);
    +
    --- End diff --
    
    This might be not needed anymore since we merged HAWQ-997 and we are sending two new headers X-GP-ATTR-TYPEMODX-COUNT, X-GP-ATTR-TYPEMODX-Y along with all columns which might have modifiers(numeric, varchar, char).


---
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 #895: HAWQ 971

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/895#discussion_r78456276
  
    --- Diff: pxf/pxf-hive/src/main/java/org/apache/hawq/pxf/plugins/hive/HiveORCSerdeResolver.java ---
    @@ -0,0 +1,150 @@
    +package org.apache.hawq.pxf.plugins.hive;
    +
    +/*
    + * 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.lang.CharUtils;
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.hive.common.type.HiveDecimal;
    +import org.apache.hadoop.hive.ql.io.orc.OrcSerde;
    +import org.apache.hadoop.hive.serde.serdeConstants;
    +import org.apache.hadoop.hive.serde2.objectinspector.*;
    +import org.apache.hadoop.hive.serde2.objectinspector.primitive.*;
    +import org.apache.hadoop.io.BytesWritable;
    +import org.apache.hadoop.io.Writable;
    +import org.apache.hadoop.mapred.JobConf;
    +import org.apache.hawq.pxf.api.BadRecordException;
    +import org.apache.hawq.pxf.api.OneField;
    +import org.apache.hawq.pxf.api.OneRow;
    +import org.apache.hawq.pxf.api.UnsupportedTypeException;
    +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.Utilities;
    +import org.apache.hawq.pxf.plugins.hdfs.utilities.HdfsUtilities;
    +import org.apache.hawq.pxf.plugins.hive.utilities.HiveUtilities;
    +
    +import java.io.IOException;
    --- End diff --
    
    Not used.


---
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 #895: HAWQ 971

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/895#discussion_r78455769
  
    --- Diff: pxf/pxf-hive/src/main/java/org/apache/hawq/pxf/plugins/hive/HiveORCAccessor.java ---
    @@ -0,0 +1,150 @@
    +package org.apache.hawq.pxf.plugins.hive;
    +
    +/*
    + * 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.hadoop.hive.ql.io.orc.OrcInputFormat;
    +import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
    +import org.apache.hadoop.hive.ql.io.sarg.SearchArgumentFactory;
    +import org.apache.hadoop.mapred.InputSplit;
    +import org.apache.hadoop.mapred.JobConf;
    +import org.apache.hadoop.mapred.Reporter;
    +import org.apache.hawq.pxf.api.FilterParser;
    +import org.apache.hawq.pxf.api.utilities.ColumnDescriptor;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +import org.apache.commons.lang.StringUtils;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import static org.apache.hawq.pxf.plugins.hive.HiveInputFormatFragmenter.PXF_HIVE_SERDES;
    +
    +/**
    + * Specialization of HiveAccessor for a Hive table that stores only ORC files.
    + * This class replaces the generic HiveAccessor for a case where a table is stored entirely as ORC files.
    + * Use together with {@link HiveInputFormatFragmenter}/{@link HiveColumnarSerdeResolver}
    + */
    +public class HiveORCAccessor extends HiveAccessor {
    +
    +    private final String READ_COLUMN_IDS_CONF_STR = "hive.io.file.readcolumn.ids";
    +    private final String READ_ALL_COLUMNS = "hive.io.file.read.all.columns";
    +    private final String READ_COLUMN_NAMES_CONF_STR = "hive.io.file.readcolumn.names";
    +    private final String SARG_PUSHDOWN = "sarg.pushdown";
    +
    +    /**
    +     * Constructs a HiveORCFileAccessor.
    +     *
    +     * @param input input containing user data
    +     * @throws Exception if user data was wrong
    +     */
    +    public HiveORCAccessor(InputData input) throws Exception {
    +        super(input, new OrcInputFormat());
    +        String[] toks = HiveInputFormatFragmenter.parseToks(input, PXF_HIVE_SERDES.ORC_SERDE.name());
    +        initPartitionFields(toks[HiveInputFormatFragmenter.TOK_KEYS]);
    +        filterInFragmenter = new Boolean(toks[HiveInputFormatFragmenter.TOK_FILTER_DONE]);
    +    }
    +
    +    @Override
    +    public boolean openForRead() throws Exception {
    +        addColumns();
    +        addFilters();
    +        return super.openForRead();
    +    }
    +
    +    /**
    +     * Adds the table tuple description to JobConf ojbect
    +     * so only these columns will be returned.
    +     */
    +    private void addColumns() throws Exception {
    +
    +        List<Integer> colIds = new ArrayList<Integer>();
    +        List<String> colNames = new ArrayList<String>();
    +        for(ColumnDescriptor col: inputData.getTupleDescription()) {
    +            if(col.isProjected()) {
    +                colIds.add(col.columnIndex());
    +                colNames.add(col.columnName());
    +            }
    +        }
    +        jobConf.set(READ_ALL_COLUMNS, "false");
    +        jobConf.set(READ_COLUMN_IDS_CONF_STR, StringUtils.join(colIds, ","));
    +        jobConf.set(READ_COLUMN_NAMES_CONF_STR, StringUtils.join(colNames, ","));
    +    }
    +
    +    /**
    +     * Uses {@link HiveFilterBuilder} to translate a filter string into a
    +     * Hive {@link SearchArgument} object. The result is added as a filter to
    +     * JobConf object
    +     */
    +    private void addFilters() throws Exception {
    +        if (!inputData.hasFilter()) {
    +            return;
    +        }
    +
    +        /* Predicate pushdown configuration */
    +        String filterStr = inputData.getFilterString();
    +        HiveFilterBuilder eval = new HiveFilterBuilder(inputData);
    +        Object filter = eval.getFilterObject(filterStr);
    +
    +        SearchArgument.Builder filterBuilder = SearchArgumentFactory.newBuilder();
    +        filterBuilder.startAnd();
    +        if (filter instanceof List) {
    --- End diff --
    
    This logic might be moved down to buildArgument.


---
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 #895: HAWQ 971

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/895#discussion_r78454804
  
    --- Diff: pxf/pxf-hive/src/main/java/org/apache/hawq/pxf/plugins/hive/HiveORCAccessor.java ---
    @@ -0,0 +1,150 @@
    +package org.apache.hawq.pxf.plugins.hive;
    +
    +/*
    + * 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.hadoop.hive.ql.io.orc.OrcInputFormat;
    +import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
    +import org.apache.hadoop.hive.ql.io.sarg.SearchArgumentFactory;
    +import org.apache.hadoop.mapred.InputSplit;
    +import org.apache.hadoop.mapred.JobConf;
    +import org.apache.hadoop.mapred.Reporter;
    --- End diff --
    
    Not used import.


---
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 #895: HAWQ 971

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

    https://github.com/apache/incubator-hawq/pull/895#discussion_r78461557
  
    --- Diff: pxf/pxf-hive/src/main/java/org/apache/hawq/pxf/plugins/hive/HiveORCAccessor.java ---
    @@ -0,0 +1,150 @@
    +package org.apache.hawq.pxf.plugins.hive;
    +
    +/*
    + * 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.hadoop.hive.ql.io.orc.OrcInputFormat;
    +import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
    +import org.apache.hadoop.hive.ql.io.sarg.SearchArgumentFactory;
    +import org.apache.hadoop.mapred.InputSplit;
    +import org.apache.hadoop.mapred.JobConf;
    +import org.apache.hadoop.mapred.Reporter;
    +import org.apache.hawq.pxf.api.FilterParser;
    +import org.apache.hawq.pxf.api.utilities.ColumnDescriptor;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +import org.apache.commons.lang.StringUtils;
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +import static org.apache.hawq.pxf.plugins.hive.HiveInputFormatFragmenter.PXF_HIVE_SERDES;
    +
    +/**
    + * Specialization of HiveAccessor for a Hive table that stores only ORC files.
    + * This class replaces the generic HiveAccessor for a case where a table is stored entirely as ORC files.
    + * Use together with {@link HiveInputFormatFragmenter}/{@link HiveColumnarSerdeResolver}
    + */
    +public class HiveORCAccessor extends HiveAccessor {
    +
    +    private final String READ_COLUMN_IDS_CONF_STR = "hive.io.file.readcolumn.ids";
    +    private final String READ_ALL_COLUMNS = "hive.io.file.read.all.columns";
    +    private final String READ_COLUMN_NAMES_CONF_STR = "hive.io.file.readcolumn.names";
    +    private final String SARG_PUSHDOWN = "sarg.pushdown";
    +
    +    /**
    +     * Constructs a HiveORCFileAccessor.
    +     *
    +     * @param input input containing user data
    +     * @throws Exception if user data was wrong
    +     */
    +    public HiveORCAccessor(InputData input) throws Exception {
    +        super(input, new OrcInputFormat());
    +        String[] toks = HiveInputFormatFragmenter.parseToks(input, PXF_HIVE_SERDES.ORC_SERDE.name());
    +        initPartitionFields(toks[HiveInputFormatFragmenter.TOK_KEYS]);
    +        filterInFragmenter = new Boolean(toks[HiveInputFormatFragmenter.TOK_FILTER_DONE]);
    +    }
    +
    +    @Override
    +    public boolean openForRead() throws Exception {
    +        addColumns();
    +        addFilters();
    +        return super.openForRead();
    +    }
    +
    +    /**
    +     * Adds the table tuple description to JobConf ojbect
    +     * so only these columns will be returned.
    +     */
    +    private void addColumns() throws Exception {
    +
    +        List<Integer> colIds = new ArrayList<Integer>();
    +        List<String> colNames = new ArrayList<String>();
    +        for(ColumnDescriptor col: inputData.getTupleDescription()) {
    +            if(col.isProjected()) {
    +                colIds.add(col.columnIndex());
    +                colNames.add(col.columnName());
    +            }
    +        }
    +        jobConf.set(READ_ALL_COLUMNS, "false");
    +        jobConf.set(READ_COLUMN_IDS_CONF_STR, StringUtils.join(colIds, ","));
    +        jobConf.set(READ_COLUMN_NAMES_CONF_STR, StringUtils.join(colNames, ","));
    +    }
    +
    +    /**
    +     * Uses {@link HiveFilterBuilder} to translate a filter string into a
    +     * Hive {@link SearchArgument} object. The result is added as a filter to
    +     * JobConf object
    +     */
    +    private void addFilters() throws Exception {
    +        if (!inputData.hasFilter()) {
    +            return;
    +        }
    +
    +        /* Predicate pushdown configuration */
    +        String filterStr = inputData.getFilterString();
    +        HiveFilterBuilder eval = new HiveFilterBuilder(inputData);
    +        Object filter = eval.getFilterObject(filterStr);
    +
    +        SearchArgument.Builder filterBuilder = SearchArgumentFactory.newBuilder();
    +        filterBuilder.startAnd();
    +        if (filter instanceof List) {
    --- End diff --
    
    I wanted buildArgument to correspond to one search argument and didn't want to make buildArgument a recursive 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 #895: HAWQ 971

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/895#discussion_r78112853
  
    --- Diff: pxf/pxf-hive/src/main/java/org/apache/hawq/pxf/plugins/hive/HiveInputFormatFragmenter.java ---
    @@ -160,6 +166,89 @@ void verifySchema(Table tbl) throws Exception {
     
         }
     
    +    private void compareTypes(DataType type, String hiveType, String fieldName) {
    --- End diff --
    
    This was moved to HiveUtilities.validateTypeCompatible when HAWQ-992 has been merged.


---
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 #895: HAWQ 971

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/895#discussion_r78458628
  
    --- Diff: pxf/pxf-hive/src/test/java/org/apache/hawq/pxf/plugins/hive/utilities/HiveUtilitiesTest.java ---
    @@ -128,47 +128,47 @@ public void mapHiveTypeWithModifiers() throws Exception {
         @Test
         public void testCompatibleHiveType() {
     
    -        String compatibleTypeName = HiveUtilities.toCompatibleHiveType(DataType.BOOLEAN);
    +        String compatibleTypeName = HiveUtilities.toCompatibleHiveType(DataType.BOOLEAN, null);
             assertEquals(compatibleTypeName, EnumHiveToHawqType.BooleanType.getTypeName());
     
    -        compatibleTypeName = HiveUtilities.toCompatibleHiveType(DataType.BYTEA);
    +        compatibleTypeName = HiveUtilities.toCompatibleHiveType(DataType.BYTEA, null);
             assertEquals(compatibleTypeName, EnumHiveToHawqType.BinaryType.getTypeName());
     
    -        compatibleTypeName = HiveUtilities.toCompatibleHiveType(DataType.BPCHAR);
    +        compatibleTypeName = HiveUtilities.toCompatibleHiveType(DataType.BPCHAR, null);
             assertEquals(compatibleTypeName, EnumHiveToHawqType.CharType.getTypeName());
     
    -        compatibleTypeName = HiveUtilities.toCompatibleHiveType(DataType.BIGINT);
    +        compatibleTypeName = HiveUtilities.toCompatibleHiveType(DataType.BIGINT, null);
             assertEquals(compatibleTypeName, EnumHiveToHawqType.BigintType.getTypeName());
     
    -        compatibleTypeName = HiveUtilities.toCompatibleHiveType(DataType.SMALLINT);
    +        compatibleTypeName = HiveUtilities.toCompatibleHiveType(DataType.SMALLINT, null);
             assertEquals(compatibleTypeName, EnumHiveToHawqType.SmallintType.getTypeName());
     
    -        compatibleTypeName = HiveUtilities.toCompatibleHiveType(DataType.INTEGER);
    +        compatibleTypeName = HiveUtilities.toCompatibleHiveType(DataType.INTEGER, null);
             assertEquals(compatibleTypeName, EnumHiveToHawqType.IntType.getTypeName());
     
    -        compatibleTypeName = HiveUtilities.toCompatibleHiveType(DataType.TEXT);
    +        compatibleTypeName = HiveUtilities.toCompatibleHiveType(DataType.TEXT, null);
             assertEquals(compatibleTypeName, EnumHiveToHawqType.StringType.getTypeName());
     
    -        compatibleTypeName = HiveUtilities.toCompatibleHiveType(DataType.REAL);
    +        compatibleTypeName = HiveUtilities.toCompatibleHiveType(DataType.REAL, null);
             assertEquals(compatibleTypeName, EnumHiveToHawqType.FloatType.getTypeName());
     
    -        compatibleTypeName = HiveUtilities.toCompatibleHiveType(DataType.FLOAT8);
    +        compatibleTypeName = HiveUtilities.toCompatibleHiveType(DataType.FLOAT8, null);
             assertEquals(compatibleTypeName, EnumHiveToHawqType.DoubleType.getTypeName());
     
    -        compatibleTypeName = HiveUtilities.toCompatibleHiveType(DataType.VARCHAR);
    +        compatibleTypeName = HiveUtilities.toCompatibleHiveType(DataType.VARCHAR, null);
             assertEquals(compatibleTypeName, EnumHiveToHawqType.VarcharType.getTypeName());
     
    -        compatibleTypeName = HiveUtilities.toCompatibleHiveType(DataType.DATE);
    +        compatibleTypeName = HiveUtilities.toCompatibleHiveType(DataType.DATE, null);
             assertEquals(compatibleTypeName, EnumHiveToHawqType.DateType.getTypeName());
     
    -        compatibleTypeName = HiveUtilities.toCompatibleHiveType(DataType.TIMESTAMP);
    +        compatibleTypeName = HiveUtilities.toCompatibleHiveType(DataType.TIMESTAMP, null);
             assertEquals(compatibleTypeName, EnumHiveToHawqType.TimestampType.getTypeName());
     
    -        compatibleTypeName = HiveUtilities.toCompatibleHiveType(DataType.NUMERIC);
    +        compatibleTypeName = HiveUtilities.toCompatibleHiveType(DataType.NUMERIC, null);
    --- End diff --
    
    Could you please add tests with not-null modifiers to test getFullTypeName indirectly?


---
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 #895: HAWQ 971

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/895#discussion_r78458135
  
    --- Diff: pxf/pxf-hive/src/main/java/org/apache/hawq/pxf/plugins/hive/utilities/EnumHiveToHawqType.java ---
    @@ -152,6 +153,33 @@ public int compare(EnumHiveToHawqType a,
         }
     
         /**
    +     *
    +     * @return full type name including modifiers. eg: varchar(3)
    +     */
    +    public static String getFullTypeName(EnumHiveToHawqType hiveToHawqType, Integer[] modifiers) {
    +        hiveToHawqType.getTypeName();
    --- End diff --
    
    Why do we call 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 #895: HAWQ 971

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

    https://github.com/apache/incubator-hawq/pull/895#discussion_r78205210
  
    --- Diff: pxf/pxf-hive/src/main/java/org/apache/hawq/pxf/plugins/hive/HiveInputFormatFragmenter.java ---
    @@ -160,6 +166,89 @@ void verifySchema(Table tbl) throws Exception {
     
         }
     
    +    private void compareTypes(DataType type, String hiveType, String fieldName) {
    +        String convertedHive = toHiveType(type, fieldName, null);
    +        if (!convertedHive.equals(hiveType)
    +                && !(convertedHive.equals("smallint") && hiveType.equals("tinyint"))
    +                && !hiveType.startsWith(convertedHive)) {
    +            throw new UnsupportedTypeException(
    +                    "Schema mismatch definition: Field " + fieldName
    +                            + " (Hive type " + hiveType + ", HAWQ type "
    +                            + type.toString() + ")");
    +        }
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("Field " + fieldName + ": Hive type " + hiveType
    +                    + ", HAWQ type " + type.toString());
    +        }
    +    }
    +
    +    /**
    +     * Converts HAWQ type to hive type. The supported mappings are:<ul>
    +     * <li>{@code BOOLEAN -> boolean}</li>
    +     * <li>{@code SMALLINT -> smallint (tinyint is converted to smallint)}</li>
    +     * <li>{@code BIGINT -> bigint}</li>
    +     * <li>{@code TIMESTAMP, TIME -> timestamp}</li>
    +     * <li>{@code NUMERIC -> decimal}</li>
    +     * <li>{@code BYTEA -> binary}</li>
    +     * <li>{@code INTERGER -> int}</li>
    +     * <li>{@code TEXT -> string}</li>
    +     * <li>{@code REAL -> float}</li>
    +     * <li>{@code FLOAT8 -> double}</li>
    +     * </ul>
    +     * All other types (both in HAWQ and in HIVE) are not supported.
    +     * For varchar and char types, typeMod will have the length info along with type.
    +     *
    +     * @param type HAWQ data type
    +     * @param name field name
    +     * @return Hive type
    +     * @throws UnsupportedTypeException if type is not supported
    +     */
    +    public static String toHiveType(DataType type, String name, String typeMod) {
    --- End diff --
    
    Yes. Will remove this function as well. Can you make sure the mappings you have configured are the same as 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 #895: HAWQ 971

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/895#discussion_r78432538
  
    --- Diff: pxf/pxf-service/src/main/resources/pxf-profiles-default.xml ---
    @@ -28,7 +28,8 @@ under the License.
                 <plugins>
                     <plugin_A>...</plugin_A>
                     <plugin_B>...</plugin_B>
    -                ...
    +                ...:q
    --- End diff --
    
    Typo?


---
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 #895: HAWQ 971

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

    https://github.com/apache/incubator-hawq/pull/895#discussion_r78207962
  
    --- Diff: pxf/pxf-hive/src/test/java/org/apache/hawq/pxf/plugins/hive/HiveDataFragmenterTest.java ---
    @@ -52,55 +52,55 @@
         HiveMetaStoreClient hiveClient;
         HiveDataFragmenter fragmenter;
     
    -    @Test
    -    public void construction() throws Exception {
    -        prepareConstruction();
    -        fragmenter = new HiveDataFragmenter(inputData);
    -        PowerMockito.verifyNew(JobConf.class).withArguments(hadoopConfiguration, HiveDataFragmenter.class);
    -        PowerMockito.verifyNew(HiveMetaStoreClient.class).withArguments(hiveConfiguration);
    -    }
    -
    -    @Test
    -    public void constructorCantAccessMetaStore() throws Exception {
    -        prepareConstruction();
    -        PowerMockito.whenNew(HiveMetaStoreClient.class).withArguments(hiveConfiguration).thenThrow(new MetaException("which way to albuquerque"));
    -
    -        try {
    -            fragmenter = new HiveDataFragmenter(inputData);
    -            fail("Expected a RuntimeException");
    -        } catch (RuntimeException ex) {
    -            assertEquals(ex.getMessage(), "Failed connecting to Hive MetaStore service: which way to albuquerque");
    -        }
    -    }
    -
    -    @Test
    -    public void invalidTableName() throws Exception {
    -        prepareConstruction();
    -        fragmenter = new HiveDataFragmenter(inputData);
    -
    -        when(inputData.getDataSource()).thenReturn("t.r.o.u.b.l.e.m.a.k.e.r");
    -
    -        try {
    -            fragmenter.getFragments();
    -            fail("Expected an IllegalArgumentException");
    -        } catch (IllegalArgumentException ex) {
    -            assertEquals(ex.getMessage(), "\"t.r.o.u.b.l.e.m.a.k.e.r\" is not a valid Hive table name. Should be either <table_name> or <db_name.table_name>");
    -        }
    -    }
    -
    -    private void prepareConstruction() throws Exception {
    -        inputData = mock(InputData.class);
    -
    -        hadoopConfiguration = mock(Configuration.class);
    -        PowerMockito.whenNew(Configuration.class).withNoArguments().thenReturn(hadoopConfiguration);
    -
    -        jobConf = mock(JobConf.class);
    -        PowerMockito.whenNew(JobConf.class).withArguments(hadoopConfiguration, HiveDataFragmenter.class).thenReturn(jobConf);
    -
    -        hiveConfiguration = mock(HiveConf.class);
    -        PowerMockito.whenNew(HiveConf.class).withNoArguments().thenReturn(hiveConfiguration);
    -
    -        hiveClient = mock(HiveMetaStoreClient.class);
    -        PowerMockito.whenNew(HiveMetaStoreClient.class).withArguments(hiveConfiguration).thenReturn(hiveClient);
    -    }
    +//    @Test
    --- End diff --
    
    Can you just delete all this commented 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 #895: HAWQ 971

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/895#discussion_r78112664
  
    --- Diff: pxf/pxf-hive/src/main/java/org/apache/hawq/pxf/plugins/hive/HiveInputFormatFragmenter.java ---
    @@ -160,6 +166,89 @@ void verifySchema(Table tbl) throws Exception {
     
         }
     
    +    private void compareTypes(DataType type, String hiveType, String fieldName) {
    +        String convertedHive = toHiveType(type, fieldName, null);
    +        if (!convertedHive.equals(hiveType)
    +                && !(convertedHive.equals("smallint") && hiveType.equals("tinyint"))
    +                && !hiveType.startsWith(convertedHive)) {
    +            throw new UnsupportedTypeException(
    +                    "Schema mismatch definition: Field " + fieldName
    +                            + " (Hive type " + hiveType + ", HAWQ type "
    +                            + type.toString() + ")");
    +        }
    +        if (LOG.isDebugEnabled()) {
    +            LOG.debug("Field " + fieldName + ": Hive type " + hiveType
    +                    + ", HAWQ type " + type.toString());
    +        }
    +    }
    +
    +    /**
    +     * Converts HAWQ type to hive type. The supported mappings are:<ul>
    +     * <li>{@code BOOLEAN -> boolean}</li>
    +     * <li>{@code SMALLINT -> smallint (tinyint is converted to smallint)}</li>
    +     * <li>{@code BIGINT -> bigint}</li>
    +     * <li>{@code TIMESTAMP, TIME -> timestamp}</li>
    +     * <li>{@code NUMERIC -> decimal}</li>
    +     * <li>{@code BYTEA -> binary}</li>
    +     * <li>{@code INTERGER -> int}</li>
    +     * <li>{@code TEXT -> string}</li>
    +     * <li>{@code REAL -> float}</li>
    +     * <li>{@code FLOAT8 -> double}</li>
    +     * </ul>
    +     * All other types (both in HAWQ and in HIVE) are not supported.
    +     * For varchar and char types, typeMod will have the length info along with type.
    +     *
    +     * @param type HAWQ data type
    +     * @param name field name
    +     * @return Hive type
    +     * @throws UnsupportedTypeException if type is not supported
    +     */
    +    public static String toHiveType(DataType type, String name, String typeMod) {
    --- End diff --
    
    This was moved to HiveUtilities.toCompatibleHiveType when HAWQ-992 has been merged.


---
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 #895: HAWQ 971

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/895#discussion_r78456209
  
    --- Diff: pxf/pxf-hive/src/main/java/org/apache/hawq/pxf/plugins/hive/HiveORCSerdeResolver.java ---
    @@ -0,0 +1,150 @@
    +package org.apache.hawq.pxf.plugins.hive;
    +
    +/*
    + * 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.lang.CharUtils;
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.hive.common.type.HiveDecimal;
    +import org.apache.hadoop.hive.ql.io.orc.OrcSerde;
    +import org.apache.hadoop.hive.serde.serdeConstants;
    +import org.apache.hadoop.hive.serde2.objectinspector.*;
    +import org.apache.hadoop.hive.serde2.objectinspector.primitive.*;
    +import org.apache.hadoop.io.BytesWritable;
    +import org.apache.hadoop.io.Writable;
    +import org.apache.hadoop.mapred.JobConf;
    +import org.apache.hawq.pxf.api.BadRecordException;
    +import org.apache.hawq.pxf.api.OneField;
    +import org.apache.hawq.pxf.api.OneRow;
    +import org.apache.hawq.pxf.api.UnsupportedTypeException;
    +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.Utilities;
    +import org.apache.hawq.pxf.plugins.hdfs.utilities.HdfsUtilities;
    --- End diff --
    
    Not used.


---
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 #895: HAWQ 971

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

    https://github.com/apache/incubator-hawq/pull/895#discussion_r78467378
  
    --- Diff: pxf/pxf-hive/src/test/java/org/apache/hawq/pxf/plugins/hive/utilities/HiveUtilitiesTest.java ---
    @@ -128,47 +128,47 @@ public void mapHiveTypeWithModifiers() throws Exception {
         @Test
         public void testCompatibleHiveType() {
     
    -        String compatibleTypeName = HiveUtilities.toCompatibleHiveType(DataType.BOOLEAN);
    +        String compatibleTypeName = HiveUtilities.toCompatibleHiveType(DataType.BOOLEAN, null);
             assertEquals(compatibleTypeName, EnumHiveToHawqType.BooleanType.getTypeName());
     
    -        compatibleTypeName = HiveUtilities.toCompatibleHiveType(DataType.BYTEA);
    +        compatibleTypeName = HiveUtilities.toCompatibleHiveType(DataType.BYTEA, null);
             assertEquals(compatibleTypeName, EnumHiveToHawqType.BinaryType.getTypeName());
     
    -        compatibleTypeName = HiveUtilities.toCompatibleHiveType(DataType.BPCHAR);
    +        compatibleTypeName = HiveUtilities.toCompatibleHiveType(DataType.BPCHAR, null);
             assertEquals(compatibleTypeName, EnumHiveToHawqType.CharType.getTypeName());
     
    -        compatibleTypeName = HiveUtilities.toCompatibleHiveType(DataType.BIGINT);
    +        compatibleTypeName = HiveUtilities.toCompatibleHiveType(DataType.BIGINT, null);
             assertEquals(compatibleTypeName, EnumHiveToHawqType.BigintType.getTypeName());
     
    -        compatibleTypeName = HiveUtilities.toCompatibleHiveType(DataType.SMALLINT);
    +        compatibleTypeName = HiveUtilities.toCompatibleHiveType(DataType.SMALLINT, null);
             assertEquals(compatibleTypeName, EnumHiveToHawqType.SmallintType.getTypeName());
     
    -        compatibleTypeName = HiveUtilities.toCompatibleHiveType(DataType.INTEGER);
    +        compatibleTypeName = HiveUtilities.toCompatibleHiveType(DataType.INTEGER, null);
             assertEquals(compatibleTypeName, EnumHiveToHawqType.IntType.getTypeName());
     
    -        compatibleTypeName = HiveUtilities.toCompatibleHiveType(DataType.TEXT);
    +        compatibleTypeName = HiveUtilities.toCompatibleHiveType(DataType.TEXT, null);
             assertEquals(compatibleTypeName, EnumHiveToHawqType.StringType.getTypeName());
     
    -        compatibleTypeName = HiveUtilities.toCompatibleHiveType(DataType.REAL);
    +        compatibleTypeName = HiveUtilities.toCompatibleHiveType(DataType.REAL, null);
             assertEquals(compatibleTypeName, EnumHiveToHawqType.FloatType.getTypeName());
     
    -        compatibleTypeName = HiveUtilities.toCompatibleHiveType(DataType.FLOAT8);
    +        compatibleTypeName = HiveUtilities.toCompatibleHiveType(DataType.FLOAT8, null);
             assertEquals(compatibleTypeName, EnumHiveToHawqType.DoubleType.getTypeName());
     
    -        compatibleTypeName = HiveUtilities.toCompatibleHiveType(DataType.VARCHAR);
    +        compatibleTypeName = HiveUtilities.toCompatibleHiveType(DataType.VARCHAR, null);
             assertEquals(compatibleTypeName, EnumHiveToHawqType.VarcharType.getTypeName());
     
    -        compatibleTypeName = HiveUtilities.toCompatibleHiveType(DataType.DATE);
    +        compatibleTypeName = HiveUtilities.toCompatibleHiveType(DataType.DATE, null);
             assertEquals(compatibleTypeName, EnumHiveToHawqType.DateType.getTypeName());
     
    -        compatibleTypeName = HiveUtilities.toCompatibleHiveType(DataType.TIMESTAMP);
    +        compatibleTypeName = HiveUtilities.toCompatibleHiveType(DataType.TIMESTAMP, null);
             assertEquals(compatibleTypeName, EnumHiveToHawqType.TimestampType.getTypeName());
     
    -        compatibleTypeName = HiveUtilities.toCompatibleHiveType(DataType.NUMERIC);
    +        compatibleTypeName = HiveUtilities.toCompatibleHiveType(DataType.NUMERIC, null);
    --- End diff --
    
    Added


---
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 #895: HAWQ 971

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

    https://github.com/apache/incubator-hawq/pull/895
  
    Can you remove all references to `Vectorized` Hive classes since they are not yet being used


---
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 #895: HAWQ 971

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/895#discussion_r78231766
  
    --- Diff: pxf/pxf-hive/src/main/java/org/apache/hawq/pxf/plugins/hive/HiveORCSerdeResolver.java ---
    @@ -0,0 +1,150 @@
    +package org.apache.hawq.pxf.plugins.hive;
    +
    +/*
    + * 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.lang.CharUtils;
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.hive.common.type.HiveDecimal;
    +import org.apache.hadoop.hive.ql.io.orc.OrcSerde;
    +import org.apache.hadoop.hive.serde.serdeConstants;
    +import org.apache.hadoop.hive.serde2.objectinspector.*;
    +import org.apache.hadoop.hive.serde2.objectinspector.primitive.*;
    +import org.apache.hadoop.io.BytesWritable;
    +import org.apache.hadoop.io.Writable;
    +import org.apache.hadoop.mapred.JobConf;
    +import org.apache.hawq.pxf.api.BadRecordException;
    +import org.apache.hawq.pxf.api.OneField;
    +import org.apache.hawq.pxf.api.OneRow;
    +import org.apache.hawq.pxf.api.UnsupportedTypeException;
    +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.Utilities;
    +import org.apache.hawq.pxf.plugins.hdfs.utilities.HdfsUtilities;
    +import org.apache.hawq.pxf.plugins.hive.utilities.HiveUtilities;
    +
    +import java.io.IOException;
    +import java.math.BigDecimal;
    +import java.util.*;
    +
    +import static org.apache.hawq.pxf.api.io.DataType.*;
    +import static org.apache.hawq.pxf.api.io.DataType.DATE;
    +import static org.apache.hawq.pxf.api.io.DataType.SMALLINT;
    +
    +/**
    + * Specialized HiveResolver for a Hive table stored as RC file.
    + * Use together with HiveInputFormatFragmenter/HiveRCFileAccessor.
    + */
    +public class HiveORCSerdeResolver extends HiveResolver {
    +    private static final Log LOG = LogFactory.getLog(HiveORCSerdeResolver.class);
    +    private OrcSerde deserializer;
    +    private boolean firstColumn;
    --- End diff --
    
    Why do we need this variable?


---
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 #895: HAWQ 971

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/895#discussion_r78454983
  
    --- Diff: pxf/pxf-hive/src/main/java/org/apache/hawq/pxf/plugins/hive/HiveORCAccessor.java ---
    @@ -0,0 +1,150 @@
    +package org.apache.hawq.pxf.plugins.hive;
    +
    +/*
    + * 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.hadoop.hive.ql.io.orc.OrcInputFormat;
    +import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
    +import org.apache.hadoop.hive.ql.io.sarg.SearchArgumentFactory;
    +import org.apache.hadoop.mapred.InputSplit;
    +import org.apache.hadoop.mapred.JobConf;
    +import org.apache.hadoop.mapred.Reporter;
    +import org.apache.hawq.pxf.api.FilterParser;
    +import org.apache.hawq.pxf.api.utilities.ColumnDescriptor;
    +import org.apache.hawq.pxf.api.utilities.InputData;
    +import org.apache.commons.lang.StringUtils;
    +import java.io.IOException;
    --- End diff --
    
    Not used.


---
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 #895: HAWQ 971

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

    https://github.com/apache/incubator-hawq/pull/895#discussion_r78205055
  
    --- Diff: src/backend/access/external/pxfheaders.c ---
    @@ -165,6 +167,18 @@ static void add_tuple_desc_httpheader(CHURL_HEADERS headers, Relation rel)
             /* Add a key/value pair for attribute type name */
             resetStringInfo(&formatter);
             appendStringInfo(&formatter, "X-GP-ATTR-TYPENAME%u", i);
    +
    --- End diff --
    
    Will remove. I had this update prior to your change


---
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 #895: HAWQ 971

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/895#discussion_r78456061
  
    --- Diff: pxf/pxf-hive/src/main/java/org/apache/hawq/pxf/plugins/hive/HiveORCSerdeResolver.java ---
    @@ -0,0 +1,150 @@
    +package org.apache.hawq.pxf.plugins.hive;
    +
    +/*
    + * 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.lang.CharUtils;
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.hive.common.type.HiveDecimal;
    --- End diff --
    
    Not used.


---
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 #895: HAWQ 971

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/895#discussion_r78454682
  
    --- Diff: pxf/pxf-hive/src/main/java/org/apache/hawq/pxf/plugins/hive/HiveORCAccessor.java ---
    @@ -0,0 +1,150 @@
    +package org.apache.hawq.pxf.plugins.hive;
    +
    +/*
    + * 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.hadoop.hive.ql.io.orc.OrcInputFormat;
    +import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
    +import org.apache.hadoop.hive.ql.io.sarg.SearchArgumentFactory;
    +import org.apache.hadoop.mapred.InputSplit;
    --- End diff --
    
    Not used.


---
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 #895: HAWQ 971

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/895#discussion_r78458437
  
    --- Diff: pxf/pxf-hive/src/main/java/org/apache/hawq/pxf/plugins/hive/utilities/EnumHiveToHawqType.java ---
    @@ -152,6 +153,33 @@ public int compare(EnumHiveToHawqType a,
         }
     
         /**
    +     *
    +     * @return full type name including modifiers. eg: varchar(3)
    +     */
    +    public static String getFullTypeName(EnumHiveToHawqType hiveToHawqType, Integer[] modifiers) {
    +        hiveToHawqType.getTypeName();
    +        if(modifiers != null && modifiers.length > 0) {
    +            String modExpression = hiveToHawqType.getSplitExpression();
    +            StringBuilder fullType = new StringBuilder(hiveToHawqType.typeName);
    +            Character start = modExpression.charAt(1);
    +            Character separator = modExpression.charAt(2);
    +            Character end = modExpression.charAt(modExpression.length()-2);
    +            fullType.append(start);
    +            int index = 0;
    +            for (Integer modifier : modifiers) {
    +                if(index++ > 0) {
    +                    fullType.append(separator);
    +                }
    +                fullType.append(modifier);
    +            }
    +            fullType.append(end);
    +            return fullType.toString();
    +        } else {
    +            return hiveToHawqType.typeName;
    --- End diff --
    
    Call getter instead of field?


---
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 #895: HAWQ 971

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/895#discussion_r78112789
  
    --- Diff: pxf/pxf-hive/src/main/java/org/apache/hawq/pxf/plugins/hive/HiveColumnarSerdeResolver.java ---
    @@ -75,8 +76,6 @@ void parseUserData(InputData input) throws Exception {
             String serdeEnumStr = toks[HiveInputFormatFragmenter.TOK_SERDE];
             if (serdeEnumStr.equals(HiveInputFormatFragmenter.PXF_HIVE_SERDES.COLUMNAR_SERDE.name())) {
                 serdeType = HiveInputFormatFragmenter.PXF_HIVE_SERDES.COLUMNAR_SERDE;
    -        } else if (serdeEnumStr.equals(HiveInputFormatFragmenter.PXF_HIVE_SERDES.LAZY_BINARY_COLUMNAR_SERDE.name())) {
    --- End diff --
    
    Why are we removing support for this serde?


---
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 #895: HAWQ 971

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

    https://github.com/apache/incubator-hawq/pull/895#discussion_r78202986
  
    --- Diff: pxf/pxf-hive/src/main/java/org/apache/hawq/pxf/plugins/hive/HiveInputFormatFragmenter.java ---
    @@ -160,6 +166,89 @@ void verifySchema(Table tbl) throws Exception {
     
         }
     
    +    private void compareTypes(DataType type, String hiveType, String fieldName) {
    --- End diff --
    
    Should this function be removed or moved ?


---
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 #895: HAWQ 971

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/895#discussion_r78457207
  
    --- Diff: pxf/pxf-hdfs/src/main/java/org/apache/hawq/pxf/plugins/hdfs/utilities/HdfsUtilities.java ---
    @@ -236,6 +236,8 @@ public static Schema getAvroSchema(Configuration conf, String dataSource)
         public static String toString(List<OneField> complexRecord, String delimiter) {
             StringBuilder buff = new StringBuilder();
             String delim = ""; // first iteration has no delimiter
    +        if(complexRecord == null)
    +            return null;
    --- End diff --
    
    Is it safe to return null? Could be better to return empty string?


---
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 #895: HAWQ 971

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/895#discussion_r78456302
  
    --- Diff: pxf/pxf-hive/src/main/java/org/apache/hawq/pxf/plugins/hive/HiveORCSerdeResolver.java ---
    @@ -0,0 +1,150 @@
    +package org.apache.hawq.pxf.plugins.hive;
    +
    +/*
    + * 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.lang.CharUtils;
    +import org.apache.commons.logging.Log;
    +import org.apache.commons.logging.LogFactory;
    +import org.apache.hadoop.conf.Configuration;
    +import org.apache.hadoop.hive.common.type.HiveDecimal;
    +import org.apache.hadoop.hive.ql.io.orc.OrcSerde;
    +import org.apache.hadoop.hive.serde.serdeConstants;
    +import org.apache.hadoop.hive.serde2.objectinspector.*;
    +import org.apache.hadoop.hive.serde2.objectinspector.primitive.*;
    +import org.apache.hadoop.io.BytesWritable;
    +import org.apache.hadoop.io.Writable;
    +import org.apache.hadoop.mapred.JobConf;
    +import org.apache.hawq.pxf.api.BadRecordException;
    +import org.apache.hawq.pxf.api.OneField;
    +import org.apache.hawq.pxf.api.OneRow;
    +import org.apache.hawq.pxf.api.UnsupportedTypeException;
    +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.Utilities;
    +import org.apache.hawq.pxf.plugins.hdfs.utilities.HdfsUtilities;
    +import org.apache.hawq.pxf.plugins.hive.utilities.HiveUtilities;
    +
    +import java.io.IOException;
    +import java.math.BigDecimal;
    --- End diff --
    
    Not used.


---
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 #895: HAWQ 971

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/895#discussion_r78111816
  
    --- Diff: pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/utilities/ColumnDescriptor.java ---
    @@ -103,12 +112,21 @@ public boolean isKeyColumn() {
             return RECORD_KEY_NAME.equalsIgnoreCase(dbColumnName);
         }
     
    +    public boolean isProjected() {
    +        return isProjected;
    +    }
    +
    +    public void setProjected(boolean projected) {
    +        isProjected = projected;
    +    }
    +
         @Override
     	public String toString() {
     		return "ColumnDescriptor [dbColumnTypeCode=" + dbColumnTypeCode
     				+ ", dbColumnName=" + dbColumnName
     				+ ", dbColumnTypeName=" + dbColumnTypeName
     				+ ", dbColumnIndex=" + dbColumnIndex
    -				+ ", dbColumnTypeModifiers=" + dbColumnTypeModifiers + "]";
    +				+ ", dbColumnTypeModifiers=" + dbColumnTypeModifiers
    +                + ", isProjected=" + isProjected + "]";
    --- End diff --
    
    identation


---
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 #895: HAWQ 971

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/895#discussion_r78456013
  
    --- Diff: pxf/pxf-hive/src/main/java/org/apache/hawq/pxf/plugins/hive/HiveORCSerdeResolver.java ---
    @@ -0,0 +1,150 @@
    +package org.apache.hawq.pxf.plugins.hive;
    +
    +/*
    + * 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.lang.CharUtils;
    --- End diff --
    
    Not used.


---
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 #895: HAWQ 971

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

    https://github.com/apache/incubator-hawq/pull/895
  
    +1, good job!


---
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 #895: HAWQ 971

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

    https://github.com/apache/incubator-hawq/pull/895#discussion_r78203271
  
    --- Diff: pxf/pxf-hive/src/main/java/org/apache/hawq/pxf/plugins/hive/HiveColumnarSerdeResolver.java ---
    @@ -75,8 +76,6 @@ void parseUserData(InputData input) throws Exception {
             String serdeEnumStr = toks[HiveInputFormatFragmenter.TOK_SERDE];
             if (serdeEnumStr.equals(HiveInputFormatFragmenter.PXF_HIVE_SERDES.COLUMNAR_SERDE.name())) {
                 serdeType = HiveInputFormatFragmenter.PXF_HIVE_SERDES.COLUMNAR_SERDE;
    -        } else if (serdeEnumStr.equals(HiveInputFormatFragmenter.PXF_HIVE_SERDES.LAZY_BINARY_COLUMNAR_SERDE.name())) {
    --- End diff --
    
    Accidentally removed


---
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.
---