You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hawq.apache.org by sh...@apache.org on 2015/10/28 23:10:04 UTC

[14/15] incubator-hawq git commit: HAWQ-45. PXF Package Namespace change

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/1c7ab9eb/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/FilterParser.java
----------------------------------------------------------------------
diff --git a/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/FilterParser.java b/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/FilterParser.java
new file mode 100644
index 0000000..21cf7d7
--- /dev/null
+++ b/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/FilterParser.java
@@ -0,0 +1,377 @@
+package org.apache.hawq.pxf.api;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Stack;
+
+/**
+ * The parser code which goes over a filter string and pushes operands onto a stack.
+ * Once an operation is read, the evaluate function is called for the {@link FilterBuilder}
+ * interface with two pop-ed operands.
+ * <br>
+ * A string of filters looks like this:
+ * <code>a2c5o1a1c"abc"o2o7</code>
+ * which means {@code column#2 < 5 AND column#1 > "abc"}
+ * <br>
+ * It is a RPN serialized representation of a filters tree in GPDB where
+ * <ul>
+ * <li> a means an attribute (column)</li>
+ * <li>c means a constant (either string or numeric)</li>
+ * <li>o means operator</li>
+ * </ul>
+ *
+ * Assuming all operators are binary, RPN representation allows it to be read left to right easily.
+ * <br>
+ * FilterParser only knows about columns and constants. The rest is up to the {@link FilterBuilder} implementer.
+ * FilterParser makes sure column objects are always on the left of the expression (when relevant).
+ */
+public class FilterParser {
+    private int index;
+    private String filterString;
+    private Stack<Object> operandsStack;
+    private FilterBuilder filterBuilder;
+
+    private static Map<Integer, Operation> operatorTranslationMap = initOperatorTransMap();
+
+    /** Supported operations by the parser. */
+    public enum Operation {
+        HDOP_LT,
+        HDOP_GT,
+        HDOP_LE,
+        HDOP_GE,
+        HDOP_EQ,
+        HDOP_NE,
+        HDOP_AND
+    }
+
+    /**
+     * Interface a user of FilterParser should implement.
+     * This is used to let the user build filter expressions in the manner she sees fit.
+     * When an operator is parsed, this function is called to let the user decide what to do with its operands.
+     */
+    public interface FilterBuilder {
+        /**
+         * Builds the filter.
+         *
+         * @param operation the parse operation to perform
+         * @param left the left operand
+         * @param right the right operand
+         * @return the built filter
+         * @throws Exception if building the filter failed
+         */
+        public Object build(Operation operation, Object left, Object right) throws Exception;
+    }
+
+    /** Represents a column index. */
+    public class ColumnIndex {
+        private int index;
+
+        public ColumnIndex(int idx) {
+            index = idx;
+        }
+
+        public int index() {
+            return index;
+        }
+    }
+
+    /** Represents a constant object (String, Long, ...). */
+    public class Constant {
+        private Object constant;
+
+        public Constant(Object obj) {
+            constant = obj;
+        }
+
+        public Object constant() {
+            return constant;
+        }
+    }
+
+    /**
+     * Basic filter provided for cases where the target storage system does not provide it own filter
+     * For example: Hbase storage provides its own filter but for a Writable based record in a
+     * SequenceFile there is no filter provided and so we need to have a default
+     */
+    static public class BasicFilter {
+        private Operation oper;
+        private ColumnIndex column;
+        private Constant constant;
+
+        /**
+         * Constructs a BasicFilter.
+         *
+         * @param oper the parse operation to perform
+         * @param column the column index
+         * @param constant the constant object
+         */
+        public BasicFilter(Operation oper, ColumnIndex column, Constant constant) {
+            this.oper = oper;
+            this.column = column;
+            this.constant = constant;
+        }
+
+        public Operation getOperation() {
+            return oper;
+        }
+
+        public ColumnIndex getColumn() {
+            return column;
+        }
+
+        public Constant getConstant() {
+            return constant;
+        }
+    }
+
+    /**
+     * Thrown when a filter's parsing exception occurs.
+     */
+    @SuppressWarnings("serial")
+    class FilterStringSyntaxException extends Exception {
+        FilterStringSyntaxException(String desc) {
+            super(desc + " (filter string: '" + filterString + "')");
+        }
+    }
+
+    /**
+     * Constructs a FilterParser.
+     *
+     * @param eval the filter builder
+     */
+    public FilterParser(FilterBuilder eval) {
+        operandsStack = new Stack<Object>();
+        filterBuilder = eval;
+    }
+
+    /**
+     * Parses the string filter.
+     *
+     * @param filter the filter to parse
+     * @return the parsed filter
+     * @throws Exception if the filter string had wrong syntax
+     */
+    public Object parse(String filter) throws Exception {
+        index = 0;
+        filterString = filter;
+
+        if (filter == null) {
+            throw new FilterStringSyntaxException("filter parsing ended with no result");
+        }
+
+        while (index < filterString.length()) {
+            char op = filterString.charAt(index);
+            ++index; // skip op character
+            switch (op) {
+                case 'a':
+                    operandsStack.push(new ColumnIndex(safeToInt(parseNumber())));
+                    break;
+                case 'c':
+                    operandsStack.push(new Constant(parseParameter()));
+                    break;
+                case 'o':
+                    // Parse and translate opcode
+                    Operation operation = operatorTranslationMap.get(safeToInt(parseNumber()));
+                    if (operation == null) {
+                        throw new FilterStringSyntaxException("unknown op ending at " + index);
+                    }
+
+                    // Pop right operand
+                    if (operandsStack.empty()) {
+                        throw new FilterStringSyntaxException("missing operands for op " + operation + " at " + index);
+                    }
+                    Object rightOperand = operandsStack.pop();
+
+                    // Pop left operand
+                    if (operandsStack.empty()) {
+                        throw new FilterStringSyntaxException("missing operands for op " + operation + " at " + index);
+                    }
+                    Object leftOperand = operandsStack.pop();
+
+                    // Normalize order, evaluate
+                    // Column should be on the left
+                    Object result = (leftOperand instanceof Constant)
+                            // column on the right, reverse expression
+                            ? filterBuilder.build(reverseOp(operation), rightOperand, leftOperand)
+                            // no swap, column on the left
+                            : filterBuilder.build(operation, leftOperand, rightOperand);
+
+                    // Store result on stack
+                    operandsStack.push(result);
+                    break;
+                default:
+                    index--; // move index back to operand location
+                    throw new FilterStringSyntaxException("unknown opcode " + op +
+                            "(" + (int) op + ") at " + index);
+            }
+        }
+
+        if (operandsStack.empty()) {
+            throw new FilterStringSyntaxException("filter parsing ended with no result");
+        }
+
+        Object result = operandsStack.pop();
+
+        if (!operandsStack.empty()) {
+            throw new FilterStringSyntaxException("Stack not empty, missing operators?");
+        }
+
+        if ((result instanceof Constant) || (result instanceof ColumnIndex)) {
+            throw new FilterStringSyntaxException("filter parsing failed, missing operators?");
+        }
+
+        return result;
+    }
+
+    /**
+     * Safely converts a long value to an int.
+     *
+     * @param value the long value to convert
+     * @return the converted int value
+     * @throws FilterStringSyntaxException if the long value is not inside an int scope
+     */
+    int safeToInt(Long value) throws FilterStringSyntaxException {
+        if (value > Integer.MAX_VALUE || value < Integer.MIN_VALUE) {
+            throw new FilterStringSyntaxException("value " + value + " larger than intmax ending at " + index);
+        }
+
+        return value.intValue();
+    }
+
+    /**
+     * Parses either a number or a string.
+     */
+    private Object parseParameter() throws Exception {
+        if (index == filterString.length()) {
+            throw new FilterStringSyntaxException("argument should follow at " + index);
+        }
+
+        return senseString()
+                ? parseString()
+                : parseNumber();
+    }
+
+    private boolean senseString() {
+        return filterString.charAt(index) == '"';
+    }
+
+    private Long parseNumber() throws Exception {
+        if (index == filterString.length()) {
+            throw new FilterStringSyntaxException("numeric argument expected at " + index);
+        }
+
+        String digits = parseDigits();
+
+        try {
+            return Long.parseLong(digits);
+        } catch (NumberFormatException e) {
+            throw new FilterStringSyntaxException("invalid numeric argument " + digits);
+        }
+
+    }
+
+    /*
+     * Parses the longest sequence of digits into a number
+     * advances the index accordingly
+     */
+    private String parseDigits() throws Exception {
+        String result;
+        int i = index;
+        int filterLength = filterString.length();
+
+        // allow sign
+        if (filterLength > 0) {
+            int chr = filterString.charAt(i);
+            if (chr == '-' || chr == '+') {
+                ++i;
+            }
+        }
+        for (; i < filterLength; ++i) {
+            int chr = filterString.charAt(i);
+            if (chr < '0' || chr > '9') {
+                break;
+            }
+        }
+
+        if (i == index) {
+            throw new FilterStringSyntaxException("numeric argument expected at " + index);
+        }
+
+        result = filterString.substring(index, i);
+        index = i;
+        return result;
+    }
+
+    /*
+     * Parses a string after its beginning '"' until its ending '"'
+     * advances the index accordingly
+     *
+     * Currently the string cannot contain '"' itself
+     * TODO add support for '"' inside the string
+     */
+    private String parseString() throws Exception {
+        StringBuilder result = new StringBuilder();
+        boolean ended = false;
+        int i;
+
+        // starting from index + 1 to skip leading "
+        for (i = index + 1; i < filterString.length(); ++i) {
+            char chr = filterString.charAt(i);
+            if (chr == '"') {
+                ended = true;
+                break;
+            }
+            result.append(chr);
+        }
+
+        if (!ended) {
+            throw new FilterStringSyntaxException("string started at " + index + " not ended with \"");
+        }
+
+        index = i + 1; // +1 to skip ending "
+        return result.toString();
+    }
+
+    /*
+     * The function takes an operator and reverses it
+     * e.g. > turns into <
+     */
+    private Operation reverseOp(Operation operation) {
+        switch (operation) {
+            case HDOP_LT:
+                operation = Operation.HDOP_GT;
+                break;
+            case HDOP_GT:
+                operation = Operation.HDOP_LT;
+                break;
+            case HDOP_LE:
+                operation = Operation.HDOP_GE;
+                break;
+            case HDOP_GE:
+                operation = Operation.HDOP_LE;
+                break;
+            default:
+                // no change o/w
+        }
+
+        return operation;
+    }
+
+    /**
+     * Create a translation table of opcodes to their enum meaning.
+     *
+     * These codes correspond to the codes in GPDB C code
+     * see gphdfilters.h in pxf protocol.
+     */
+    static private Map<Integer, Operation> initOperatorTransMap() {
+        Map<Integer, Operation> operatorTranslationMap = new HashMap<Integer, Operation>();
+        operatorTranslationMap.put(1, Operation.HDOP_LT);
+        operatorTranslationMap.put(2, Operation.HDOP_GT);
+        operatorTranslationMap.put(3, Operation.HDOP_LE);
+        operatorTranslationMap.put(4, Operation.HDOP_GE);
+        operatorTranslationMap.put(5, Operation.HDOP_EQ);
+        operatorTranslationMap.put(6, Operation.HDOP_NE);
+        operatorTranslationMap.put(7, Operation.HDOP_AND);
+        return operatorTranslationMap;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/1c7ab9eb/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/Fragment.java
----------------------------------------------------------------------
diff --git a/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/Fragment.java b/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/Fragment.java
new file mode 100644
index 0000000..cd3c55a
--- /dev/null
+++ b/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/Fragment.java
@@ -0,0 +1,101 @@
+package org.apache.hawq.pxf.api;
+
+/**
+ * Fragment holds a data fragment' information.
+ * {@link Fragmenter#getFragments} returns a list of fragments.
+ */
+public class Fragment {
+    /**
+     * File path+name, table name, etc.
+     */
+    private String sourceName;
+
+    /**
+     * Fragment index (incremented per sourceName).
+     */
+    private int index;
+
+    /**
+     * Fragment replicas (1 or more).
+     */
+    private String[] replicas;
+
+    /**
+     * Fragment metadata information (starting point + length, region location, etc.).
+     */
+    private byte[] metadata;
+
+    /**
+     * ThirdParty data added to a fragment. Ignored if null.
+     */
+    private byte[] userData;
+
+    /**
+     * Constructs a Fragment.
+     *
+     * @param sourceName the resource uri (File path+name, table name, etc.)
+     * @param hosts the replicas
+     * @param metadata the meta data (Starting point + length, region location, etc.).
+     */
+    public Fragment(String sourceName,
+                    String[] hosts,
+                    byte[] metadata) {
+        this.sourceName = sourceName;
+        this.replicas = hosts;
+        this.metadata = metadata;
+    }
+
+    /**
+     * Constructs a Fragment.
+     *
+     * @param sourceName the resource uri (File path+name, table name, etc.)
+     * @param hosts the replicas
+     * @param metadata the meta data (Starting point + length, region location, etc.).
+     * @param userData third party data added to a fragment.
+     */
+    public Fragment(String sourceName,
+                    String[] hosts,
+                    byte[] metadata,
+                    byte[] userData) {
+        this.sourceName = sourceName;
+        this.replicas = hosts;
+        this.metadata = metadata;
+        this.userData = userData;
+    }
+
+    public String getSourceName() {
+        return sourceName;
+    }
+
+    public int getIndex() {
+        return index;
+    }
+
+    public void setIndex(int index) {
+        this.index = index;
+    }
+
+    public String[] getReplicas() {
+        return replicas;
+    }
+
+    public void setReplicas(String[] replicas) {
+        this.replicas = replicas;
+    }
+
+    public byte[] getMetadata() {
+        return metadata;
+    }
+
+    public void setMetadata(byte[] metadata) {
+        this.metadata = metadata;
+    }
+
+    public byte[] getUserData() {
+        return userData;
+    }
+
+    public void setUserData(byte[] userData) {
+        this.userData = userData;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/1c7ab9eb/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/Fragmenter.java
----------------------------------------------------------------------
diff --git a/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/Fragmenter.java b/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/Fragmenter.java
new file mode 100644
index 0000000..4ae057f
--- /dev/null
+++ b/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/Fragmenter.java
@@ -0,0 +1,33 @@
+package org.apache.hawq.pxf.api;
+
+import org.apache.hawq.pxf.api.utilities.InputData;
+import org.apache.hawq.pxf.api.utilities.Plugin;
+
+import java.util.LinkedList;
+import java.util.List;
+
+/**
+ * Abstract class that defines the splitting of a data resource into fragments that can be processed in parallel.
+ */
+public abstract class Fragmenter extends Plugin {
+    protected List<Fragment> fragments;
+
+    /**
+     * Constructs a Fragmenter.
+     *
+     * @param metaData the input data
+     */
+    public Fragmenter(InputData metaData) {
+        super(metaData);
+        fragments = new LinkedList<>();
+    }
+
+    /**
+     * Gets the fragments of a given path (source name and location of each fragment).
+     * Used to get fragments of data that could be read in parallel from the different segments.
+     *
+     * @return list of data fragments
+     * @throws Exception if fragment list could not be retrieved
+     */
+    public abstract List<Fragment> getFragments() throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/1c7ab9eb/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/Metadata.java
----------------------------------------------------------------------
diff --git a/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/Metadata.java b/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/Metadata.java
new file mode 100644
index 0000000..2171943
--- /dev/null
+++ b/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/Metadata.java
@@ -0,0 +1,129 @@
+package org.apache.hawq.pxf.api;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.lang.StringUtils;
+
+/**
+ * Metadata holds a table's metadata information.
+ * {@link MetadataFetcher#getTableMetadata} returns the table's metadata.
+ */
+public class Metadata {
+
+    /**
+     * Class representing table name - db (schema) name and table name.
+     */
+    public static class Table {
+        private String dbName;
+        private String tableName;
+
+        public Table(String dbName, String tableName) {
+
+            if (StringUtils.isBlank(dbName) || StringUtils.isBlank(tableName)) {
+                throw new IllegalArgumentException("Table name cannot be empty");
+            }
+
+            this.dbName = dbName;
+            this.tableName = tableName;
+        }
+
+        public String getDbName() {
+            return dbName;
+        }
+
+        public String getTableName() {
+            return tableName;
+        }
+
+        /**
+         * Returns full table name in the form db_name.table_name
+         */
+        @Override
+        public String toString() {
+            return dbName + "." + tableName;
+        }
+    }
+
+    /**
+     * Class representing table field - name and type.
+     */
+    public static class Field {
+        private String name;
+        private String type; // TODO: nhorn - 06-03-15 - change to enum
+        private String[] modifiers; // type modifiers, optional field
+
+        public Field(String name, String type) {
+
+            if (StringUtils.isBlank(name) || StringUtils.isBlank(type)) {
+                throw new IllegalArgumentException("Field name and type cannot be empty");
+            }
+
+            this.name = name;
+            this.type = type;
+        }
+
+        public Field(String name, String type, String[] modifiers) {
+            this(name, type);
+            this.modifiers = modifiers;
+        }
+
+        public String getName() {
+            return name;
+        }
+
+        public String getType() {
+            return type;
+        }
+
+        public String[] getModifiers() {
+            return modifiers;
+        }
+    }
+
+    /**
+     * Table name
+     */
+    private Metadata.Table table;
+
+    /**
+     * Table's fields
+     */
+    private List<Metadata.Field> fields;
+
+    /**
+     * Constructs a table's Metadata.
+     *
+     * @param tableName the table name
+     * @param fields the table's fields
+     */
+    public Metadata(Metadata.Table tableName,
+            List<Metadata.Field> fields) {
+        this.table = tableName;
+        this.fields = fields;
+    }
+
+    public Metadata(Metadata.Table tableName) {
+        this(tableName, new ArrayList<Metadata.Field>());
+    }
+
+    public Metadata.Table getTable() {
+        return table;
+    }
+
+    public List<Metadata.Field> getFields() {
+        return fields;
+    }
+
+    /**
+     * Adds a field to metadata fields.
+     *
+     * @param field field to add
+     */
+    public void addField(Metadata.Field field) {
+        if (fields == null) {
+            fields = new ArrayList<Metadata.Field>();
+        }
+        fields.add(field);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/1c7ab9eb/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/MetadataFetcher.java
----------------------------------------------------------------------
diff --git a/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/MetadataFetcher.java b/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/MetadataFetcher.java
new file mode 100644
index 0000000..c042884
--- /dev/null
+++ b/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/MetadataFetcher.java
@@ -0,0 +1,25 @@
+package org.apache.hawq.pxf.api;
+
+/**
+ * Abstract class that defines getting metadata of a table.
+ */
+public abstract class MetadataFetcher {
+    protected Metadata metadata;
+
+    /**
+     * Constructs a MetadataFetcher.
+     *
+     */
+    public MetadataFetcher() {
+
+    }
+
+    /**
+     * Gets a metadata of a given table
+     *
+     * @param tableName table name
+     * @return metadata of given table
+     * @throws Exception if metadata information could not be retrieved
+     */
+    public abstract Metadata getTableMetadata(String tableName) throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/1c7ab9eb/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/OneField.java
----------------------------------------------------------------------
diff --git a/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/OneField.java b/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/OneField.java
new file mode 100644
index 0000000..53144bd
--- /dev/null
+++ b/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/OneField.java
@@ -0,0 +1,26 @@
+package org.apache.hawq.pxf.api;
+
+/**
+ * Defines a one field in a deserialized record.
+ */
+public class OneField {
+    /** OID value recognized by GPDBWritable. */
+    public int type;
+
+    /** Field value. */
+    public Object val;
+
+    public OneField() {
+    }
+
+    /**
+     * Constructs a OneField object.
+     *
+     * @param type the OID value recognized by GPDBWritable
+     * @param val the field value
+     */
+    public OneField(int type, Object val) {
+        this.type = type;
+        this.val = val;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/1c7ab9eb/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/OneRow.java
----------------------------------------------------------------------
diff --git a/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/OneRow.java b/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/OneRow.java
new file mode 100755
index 0000000..3d36d83
--- /dev/null
+++ b/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/OneRow.java
@@ -0,0 +1,47 @@
+package org.apache.hawq.pxf.api;
+
+/**
+ * Represents one row in the external system data store.
+ * Supports the general case where one row contains both a record and a
+ * separate key like in the HDFS key/value model for MapReduce (Example: HDFS sequence file).
+ */
+public class OneRow {
+    private Object key;
+    private Object data;
+
+    public OneRow(){
+    }
+
+    /**
+     * Constructs a OneRow
+     *
+     * @param key the key for the record
+     * @param data the actual record
+     */
+    public OneRow(Object key, Object data) {
+        this.key = key;
+        this.data = data;
+    }
+
+    public void setKey(Object key) {
+        this.key = key;
+    }
+
+    public void setData(Object data) {
+        this.data = data;
+    }
+
+    public Object getKey() {
+        return key;
+    }
+
+    public Object getData() {
+        return data;
+    }
+
+    @Override
+    public String toString() {
+        return "OneRow:" + key + "->" + data;
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/1c7ab9eb/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/OutputFormat.java
----------------------------------------------------------------------
diff --git a/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/OutputFormat.java b/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/OutputFormat.java
new file mode 100644
index 0000000..4e96a23
--- /dev/null
+++ b/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/OutputFormat.java
@@ -0,0 +1,6 @@
+package org.apache.hawq.pxf.api;
+
+/**
+ * PXF supported output formats: {@link #TEXT} and {@link #BINARY}
+ */
+public enum OutputFormat {TEXT, BINARY}

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/1c7ab9eb/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/ReadAccessor.java
----------------------------------------------------------------------
diff --git a/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/ReadAccessor.java b/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/ReadAccessor.java
new file mode 100644
index 0000000..51634bc
--- /dev/null
+++ b/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/ReadAccessor.java
@@ -0,0 +1,29 @@
+package org.apache.hawq.pxf.api;
+
+/**
+ * Interface that defines access to the source data store (e.g, a file on HDFS, a region of an HBase table, etc).
+ */
+public interface ReadAccessor {
+    /**
+     * Opens the resource for reading.
+     *
+     * @return true if the resource is successfully opened
+     * @throws Exception if opening the resource failed
+     */
+    boolean openForRead() throws Exception;
+
+    /**
+     * Reads the next object.
+     *
+     * @return the object which was read
+     * @throws Exception if reading from the resource failed
+     */
+    OneRow readNextObject() throws Exception;
+
+    /**
+     * Closes the resource.
+     *
+     * @throws Exception if closing the resource failed
+     */
+    void closeForRead() throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/1c7ab9eb/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/ReadResolver.java
----------------------------------------------------------------------
diff --git a/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/ReadResolver.java b/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/ReadResolver.java
new file mode 100644
index 0000000..63628bb
--- /dev/null
+++ b/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/ReadResolver.java
@@ -0,0 +1,18 @@
+package org.apache.hawq.pxf.api;
+
+import java.util.List;
+
+/**
+ * Interface that defines the deserialization of one record brought from the {@link ReadAccessor}.
+ * All deserialization methods (e.g, Writable, Avro, ...) implement this interface.
+ */
+public interface ReadResolver {
+    /**
+     * Gets the {@link OneField} list of one row.
+     *
+     * @param row the row to get the fields from
+     * @return the {@link OneField} list of one row.
+     * @throws Exception if decomposing the row into fields failed
+     */
+    List<OneField> getFields(OneRow row) throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/1c7ab9eb/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/UnsupportedTypeException.java
----------------------------------------------------------------------
diff --git a/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/UnsupportedTypeException.java b/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/UnsupportedTypeException.java
new file mode 100644
index 0000000..d96e1ab
--- /dev/null
+++ b/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/UnsupportedTypeException.java
@@ -0,0 +1,25 @@
+package org.apache.hawq.pxf.api;
+
+/**
+ * Thrown when the resolver tries to serializes/deserializes an unsupported type.
+ */
+public class UnsupportedTypeException extends RuntimeException {
+
+    /**
+     * Constructs an UnsupportedTypeException
+     *
+     * @param cause cause of this exception
+     */
+    public UnsupportedTypeException(Throwable cause) {
+        super(cause);
+    }
+
+    /**
+     * Constructs an UnsupportedTypeException
+     *
+     * @param message cause of this exception
+     */
+    public UnsupportedTypeException(String message) {
+        super(message);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/1c7ab9eb/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/UserDataException.java
----------------------------------------------------------------------
diff --git a/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/UserDataException.java b/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/UserDataException.java
new file mode 100644
index 0000000..2c6a94f
--- /dev/null
+++ b/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/UserDataException.java
@@ -0,0 +1,25 @@
+package org.apache.hawq.pxf.api;
+
+/**
+ * Thrown when Accessor/Resolver failes to parse {@link org.apache.hawq.pxf.api.utilities.InputData#userData}.
+ */
+public class UserDataException extends Exception {
+
+    /**
+     * Constructs an UserDataException
+     *
+     * @param cause the cause of this exception
+     */
+    public UserDataException(Throwable cause) {
+        super(cause);
+    }
+
+    /**
+     * Constructs an UserDataException
+     *
+     * @param message the cause of this exception
+     */
+    public UserDataException(String message) {
+        super(message);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/1c7ab9eb/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/WriteAccessor.java
----------------------------------------------------------------------
diff --git a/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/WriteAccessor.java b/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/WriteAccessor.java
new file mode 100644
index 0000000..cf77c7a
--- /dev/null
+++ b/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/WriteAccessor.java
@@ -0,0 +1,30 @@
+package org.apache.hawq.pxf.api;
+
+/**
+ * Interface for writing data into a data store (e.g. a sequence file on HDFS).
+ */
+public interface WriteAccessor {
+    /**
+     * Opens the resource for write.
+     *
+     * @return true if the resource is successfully opened
+     * @throws Exception if opening the resource failed
+     */
+    boolean openForWrite() throws Exception;
+
+    /**
+     * Writes the next object.
+     *
+     * @param onerow the object to be written
+     * @return true if the write succeeded
+     * @throws Exception writing to the resource failed
+     */
+    boolean writeNextObject(OneRow onerow) throws Exception;
+
+    /**
+     * Closes the resource for write.
+     *
+     * @throws Exception if closing the resource failed
+     */
+    void closeForWrite() throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/1c7ab9eb/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/WriteResolver.java
----------------------------------------------------------------------
diff --git a/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/WriteResolver.java b/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/WriteResolver.java
new file mode 100644
index 0000000..0dd06c9
--- /dev/null
+++ b/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/WriteResolver.java
@@ -0,0 +1,19 @@
+package org.apache.hawq.pxf.api;
+
+import java.util.List;
+
+/**
+ * Interface that defines the serialization of data read from the DB
+ * into a OneRow object.
+ * This interface is implemented by all serialization methods (e.g, Writable, Avro, ...).
+ */
+public interface WriteResolver {
+    /**
+     * Constructs and sets the fields of a {@link OneRow}.
+     *
+     * @param record list of {@link OneField}
+     * @return the constructed {@link OneRow}
+     * @throws Exception if constructing a row from the fields failed
+     */
+    OneRow setFields(List<OneField> record) throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/1c7ab9eb/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/io/DataType.java
----------------------------------------------------------------------
diff --git a/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/io/DataType.java b/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/io/DataType.java
new file mode 100644
index 0000000..0e7f65e
--- /dev/null
+++ b/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/io/DataType.java
@@ -0,0 +1,59 @@
+package org.apache.hawq.pxf.api.io;
+
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Supported Data Types and OIDs (HAWQ Data Type identifiers).
+ * There's a one-to-one match between a Data Type and it's corresponding OID.
+ */
+public enum DataType {
+    BOOLEAN(16),
+    BYTEA(17),
+    CHAR(18),
+    BIGINT(20),
+    SMALLINT(21),
+    INTEGER(23),
+    TEXT(25),
+    REAL(700),
+    FLOAT8(701),
+    BPCHAR(1042),
+    VARCHAR(1043),
+    DATE(1082),
+    TIME(1083),
+    TIMESTAMP(1114),
+    NUMERIC(1700),
+    UNSUPPORTED_TYPE(-1);
+
+    private static final Map<Integer, DataType> lookup = new HashMap<>();
+
+    static {
+        for (DataType dt : EnumSet.allOf(DataType.class)) {
+            lookup.put(dt.getOID(), dt);
+        }
+    }
+
+    private final int OID;
+
+    DataType(int OID) {
+        this.OID = OID;
+    }
+
+    /**
+     * Utility method for converting an {@link #OID} to a {@link #DataType}.
+     *
+     * @param OID the oid to be converted
+     * @return the corresponding DataType if exists, else returns {@link #UNSUPPORTED_TYPE}
+     */
+    public static DataType get(int OID) {
+        DataType type = lookup.get(OID);
+        return type == null
+                ? UNSUPPORTED_TYPE
+                : type;
+    }
+
+    public int getOID() {
+        return OID;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/1c7ab9eb/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/utilities/ColumnDescriptor.java
----------------------------------------------------------------------
diff --git a/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/utilities/ColumnDescriptor.java b/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/utilities/ColumnDescriptor.java
new file mode 100644
index 0000000..b378ab2
--- /dev/null
+++ b/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/utilities/ColumnDescriptor.java
@@ -0,0 +1,79 @@
+package org.apache.hawq.pxf.api.utilities;
+
+/**
+ * ColumnDescriptor describes one column in hawq database.
+ * Currently it means a name, a type id (HAWQ/GPDB OID), a type name and column index.
+ */
+public class ColumnDescriptor {
+
+	int gpdbColumnTypeCode;
+    String gpdbColumnName;
+    String gpdbColumnTypeName;
+    int gpdbColumnIndex;
+
+    /**
+     * Reserved word for a table record key.
+     * A field with this name will be treated as record key.
+     */
+    public static final String RECORD_KEY_NAME = "recordkey";
+
+    /**
+     * Constructs a ColumnDescriptor.
+     *
+     * @param name column name
+     * @param typecode OID
+     * @param index column index
+     * @param typename type name
+     */
+    public ColumnDescriptor(String name, int typecode, int index, String typename) {
+        gpdbColumnTypeCode = typecode;
+        gpdbColumnTypeName = typename;
+        gpdbColumnName = name;
+        gpdbColumnIndex = index;
+    }
+
+    /**
+     * Constructs a copy of ColumnDescriptor.
+     *
+     * @param copy the ColumnDescriptor to copy
+     */
+    public ColumnDescriptor(ColumnDescriptor copy) {
+        this.gpdbColumnTypeCode = copy.gpdbColumnTypeCode;
+        this.gpdbColumnName = copy.gpdbColumnName;
+        this.gpdbColumnIndex = copy.gpdbColumnIndex;
+        this.gpdbColumnTypeName = copy.gpdbColumnTypeName;
+    }
+
+    public String columnName() {
+        return gpdbColumnName;
+    }
+
+    public int columnTypeCode() {
+        return gpdbColumnTypeCode;
+    }
+
+    public int columnIndex() {
+        return gpdbColumnIndex;
+    }
+
+    public String columnTypeName() {
+        return gpdbColumnTypeName;
+    }
+
+    /**
+     * Returns <tt>true</tt> if {@link #gpdbColumnName} is a {@link #RECORD_KEY_NAME}.
+     *
+     * @return whether column is a record key column
+     */
+    public boolean isKeyColumn() {
+        return RECORD_KEY_NAME.equalsIgnoreCase(gpdbColumnName);
+    }
+
+    @Override
+	public String toString() {
+		return "ColumnDescriptor [gpdbColumnTypeCode=" + gpdbColumnTypeCode
+				+ ", gpdbColumnName=" + gpdbColumnName
+				+ ", gpdbColumnTypeName=" + gpdbColumnTypeName
+				+ ", gpdbColumnIndex=" + gpdbColumnIndex + "]";
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/1c7ab9eb/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/utilities/InputData.java
----------------------------------------------------------------------
diff --git a/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/utilities/InputData.java b/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/utilities/InputData.java
new file mode 100644
index 0000000..bc1cbfc
--- /dev/null
+++ b/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/utilities/InputData.java
@@ -0,0 +1,306 @@
+package org.apache.hawq.pxf.api.utilities;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import java.util.*;
+
+/**
+ * Common configuration available to all PXF plugins. Represents input data
+ * coming from client applications, such as Hawq.
+ */
+public class InputData {
+
+    public static final int INVALID_SPLIT_IDX = -1;
+    private static final Log LOG = LogFactory.getLog(InputData.class);
+
+    protected Map<String, String> requestParametersMap;
+    protected ArrayList<ColumnDescriptor> tupleDescription;
+    protected int segmentId;
+    protected int totalSegments;
+    protected byte[] fragmentMetadata = null;
+    protected byte[] userData = null;
+    protected boolean filterStringValid;
+    protected String filterString;
+    protected String dataSource;
+    protected String accessor;
+    protected String resolver;
+    protected String analyzer;
+    protected String fragmenter;
+    protected String remoteLogin;
+    protected String remoteSecret;
+    protected int dataFragment; /* should be deprecated */
+
+    /**
+     * When false the bridge has to run in synchronized mode. default value -
+     * true.
+     */
+    protected boolean threadSafe;
+
+    /**
+     * The name of the recordkey column. It can appear in any location in the
+     * columns list. By specifying the recordkey column, the user declares that
+     * he is interested to receive for every record retrieved also the the
+     * recordkey in the database. The recordkey is present in HBase table (it is
+     * called rowkey), and in sequence files. When the HDFS storage element
+     * queried will not have a recordkey and the user will still specify it in
+     * the "create external table" statement, then the values for this field
+     * will be null. This field will always be the first field in the tuple
+     * returned.
+     */
+    protected ColumnDescriptor recordkeyColumn;
+
+    /**
+     * Constructs an empty InputData
+     */
+    public InputData() {
+    }
+
+    /**
+     * Constructs an InputData from a copy. Used to create from an extending
+     * class.
+     *
+     * @param copy the input data to copy
+     */
+    public InputData(InputData copy) {
+
+        this.requestParametersMap = copy.requestParametersMap;
+        this.segmentId = copy.segmentId;
+        this.totalSegments = copy.totalSegments;
+        this.fragmentMetadata = copy.fragmentMetadata;
+        this.userData = copy.userData;
+        this.tupleDescription = copy.tupleDescription;
+        this.recordkeyColumn = copy.recordkeyColumn;
+        this.filterStringValid = copy.filterStringValid;
+        this.filterString = copy.filterString;
+        this.dataSource = copy.dataSource;
+        this.accessor = copy.accessor;
+        this.resolver = copy.resolver;
+        this.fragmenter = copy.fragmenter;
+        this.analyzer = copy.analyzer;
+        this.remoteLogin = copy.remoteLogin;
+        this.remoteSecret = copy.remoteSecret;
+        this.threadSafe = copy.threadSafe;
+    }
+
+    /**
+     * Returns a user defined property.
+     *
+     * @param userProp the lookup user property
+     * @return property value as a String
+     */
+    public String getUserProperty(String userProp) {
+        return requestParametersMap.get("X-GP-" + userProp.toUpperCase());
+    }
+
+    /**
+     * Sets the byte serialization of a fragment meta data.
+     *
+     * @param location start, len, and location of the fragment
+     */
+    public void setFragmentMetadata(byte[] location) {
+        this.fragmentMetadata = location;
+    }
+
+    /**
+     * The byte serialization of a data fragment.
+     *
+     * @return serialized fragment metadata
+     */
+    public byte[] getFragmentMetadata() {
+        return fragmentMetadata;
+    }
+
+    /**
+     * Gets any custom user data that may have been passed from the fragmenter.
+     * Will mostly be used by the accessor or resolver.
+     *
+     * @return fragment user data
+     */
+    public byte[] getFragmentUserData() {
+        return userData;
+    }
+
+    /**
+     * Sets any custom user data that needs to be shared across plugins. Will
+     * mostly be set by the fragmenter.
+     *
+     * @param userData user data
+     */
+    public void setFragmentUserData(byte[] userData) {
+        this.userData = userData;
+    }
+
+    /**
+     * Returns the number of segments in HAWQ.
+     *
+     * @return number of segments
+     */
+    public int getTotalSegments() {
+        return totalSegments;
+    }
+
+    /**
+     * Returns the current segment ID in HAWQ.
+     *
+     * @return current segment ID
+     */
+    public int getSegmentId() {
+        return segmentId;
+    }
+
+    /**
+     * Returns true if there is a filter string to parse.
+     *
+     * @return whether there is a filter string
+     */
+    public boolean hasFilter() {
+        return filterStringValid;
+    }
+
+    /**
+     * Returns the filter string, <tt>null</tt> if #hasFilter is <tt>false</tt>.
+     *
+     * @return the filter string or null
+     */
+    public String getFilterString() {
+        return filterString;
+    }
+
+    /**
+     * Returns tuple description.
+     *
+     * @return tuple description
+     */
+    public ArrayList<ColumnDescriptor> getTupleDescription() {
+        return tupleDescription;
+    }
+
+    /**
+     * Returns the number of columns in tuple description.
+     *
+     * @return number of columns
+     */
+    public int getColumns() {
+        return tupleDescription.size();
+    }
+
+    /**
+     * Returns column index from tuple description.
+     *
+     * @param index index of column
+     * @return column by index
+     */
+    public ColumnDescriptor getColumn(int index) {
+        return tupleDescription.get(index);
+    }
+
+    /**
+     * Returns the column descriptor of the recordkey column. If the recordkey
+     * column was not specified by the user in the create table statement will
+     * return null.
+     *
+     * @return column of record key or null
+     */
+    public ColumnDescriptor getRecordkeyColumn() {
+        return recordkeyColumn;
+    }
+
+    /**
+     * Returns the data source of the required resource (i.e a file path or a
+     * table name).
+     *
+     * @return data source
+     */
+    public String getDataSource() {
+        return dataSource;
+    }
+
+    /**
+     * Sets the data source for the required resource.
+     *
+     * @param dataSource data source to be set
+     */
+    public void setDataSource(String dataSource) {
+        this.dataSource = dataSource;
+    }
+
+    /**
+     * Returns the ClassName for the java class that was defined as Accessor.
+     *
+     * @return class name for Accessor
+     */
+    public String getAccessor() {
+        return accessor;
+    }
+
+    /**
+     * Returns the ClassName for the java class that was defined as Resolver.
+     *
+     * @return class name for Resolver
+     */
+    public String getResolver() {
+        return resolver;
+    }
+
+    /**
+     * Returns the ClassName for the java class that was defined as Fragmenter
+     * or null if no fragmenter was defined.
+     *
+     * @return class name for Fragmenter or null
+     */
+    public String getFragmenter() {
+        return fragmenter;
+    }
+
+    /**
+     * Returns the ClassName for the java class that was defined as Analyzer or
+     * null if no analyzer was defined.
+     *
+     * @return class name for Analyzer or null
+     */
+    public String getAnalyzer() {
+        return analyzer;
+    }
+
+    /**
+     * Returns the contents of pxf_remote_service_login set in Hawq. Should the
+     * user set it to an empty string this function will return null.
+     *
+     * @return remote login details if set, null otherwise
+     */
+    public String getLogin() {
+        return remoteLogin;
+    }
+
+    /**
+     * Returns the contents of pxf_remote_service_secret set in Hawq. Should the
+     * user set it to an empty string this function will return null.
+     *
+     * @return remote password if set, null otherwise
+     */
+    public String getSecret() {
+        return remoteSecret;
+    }
+
+    /**
+     * Returns whether this request is thread safe.
+     * If it is not, request will be handled consequentially and not in parallel.
+     *
+     * @return whether the request is thread safe
+     */
+    public boolean isThreadSafe() {
+        return threadSafe;
+    }
+
+    /**
+     * Returns a data fragment index. plan to deprecate it in favor of using
+     * getFragmentMetadata().
+     *
+     * @return data fragment index
+     */
+    public int getDataFragment() {
+        return dataFragment;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/1c7ab9eb/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/utilities/Plugin.java
----------------------------------------------------------------------
diff --git a/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/utilities/Plugin.java b/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/utilities/Plugin.java
new file mode 100644
index 0000000..13f1daf
--- /dev/null
+++ b/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/utilities/Plugin.java
@@ -0,0 +1,28 @@
+package org.apache.hawq.pxf.api.utilities;
+
+
+/**
+ * Base class for all plugin types (Accessor, Resolver, Fragmenter, Analyzer, ...).
+ * Manages the meta data.
+ */
+public class Plugin {
+    protected InputData inputData;
+
+    /**
+     * Constructs a plugin.
+     *
+     * @param input the input data
+     */
+    public Plugin(InputData input) {
+        this.inputData = input;
+    }
+
+    /**
+     * Checks if the plugin is thread safe or not, based on inputData.
+     *
+     * @return true if plugin is thread safe
+     */
+    public boolean isThreadSafe() {
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/1c7ab9eb/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/utilities/ProfileConfException.java
----------------------------------------------------------------------
diff --git a/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/utilities/ProfileConfException.java b/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/utilities/ProfileConfException.java
new file mode 100644
index 0000000..a5defbc
--- /dev/null
+++ b/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/utilities/ProfileConfException.java
@@ -0,0 +1,42 @@
+package org.apache.hawq.pxf.api.utilities;
+
+/**
+ * Thrown when there is a configuration problem with pxf profiles definitions.
+ * {@link ProfileConfException.MessageFormat#PROFILES_FILE_NOT_FOUND} when pxf-profiles.xml is missing from the CLASSPATH.
+ * {@link ProfileConfException.MessageFormat#PROFILES_FILE_LOAD_ERR} when pxf-profiles.xml is not valid.
+ * {@link ProfileConfException.MessageFormat#NO_PROFILE_DEF} when a profile entry or attribute is missing.
+ */
+public class ProfileConfException extends RuntimeException {
+    public static enum MessageFormat {
+        PROFILES_FILE_NOT_FOUND("%s was not found on the CLASSPATH"),
+        PROFILES_FILE_LOAD_ERR("Profiles configuration %s could not be loaded: %s"),
+        NO_PROFILE_DEF("%s is not defined in %s");
+
+        String format;
+
+        MessageFormat(String format) {
+            this.format = format;
+        }
+
+        public String getFormat() {
+            return format;
+        }
+    }
+
+    private MessageFormat msgFormat;
+
+    /**
+     * Constructs a ProfileConfException.
+     *
+     * @param msgFormat the message format
+     * @param msgArgs the message arguments
+     */
+    public ProfileConfException(MessageFormat msgFormat, String... msgArgs) {
+        super(String.format(msgFormat.getFormat(), (Object[]) msgArgs));
+        this.msgFormat = msgFormat;
+    }
+
+    public MessageFormat getMsgFormat() {
+        return msgFormat;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/1c7ab9eb/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/utilities/ProfilesConf.java
----------------------------------------------------------------------
diff --git a/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/utilities/ProfilesConf.java b/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/utilities/ProfilesConf.java
new file mode 100644
index 0000000..5c85af6
--- /dev/null
+++ b/pxf/pxf-api/src/main/java/org/apache/hawq/pxf/api/utilities/ProfilesConf.java
@@ -0,0 +1,116 @@
+package org.apache.hawq.pxf.api.utilities;
+
+import org.apache.commons.collections.IteratorUtils;
+import org.apache.commons.configuration.Configuration;
+import org.apache.commons.configuration.ConfigurationException;
+import org.apache.commons.configuration.XMLConfiguration;
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import java.net.URL;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+
+import static org.apache.hawq.pxf.api.utilities.ProfileConfException.MessageFormat.*;
+
+/**
+ * This enum holds the profiles files: pxf-profiles.xml and pxf-profiles-default.xml.
+ * It exposes a public static method getProfilePluginsMap(String plugin) which returns the requested profile plugins
+ */
+public enum ProfilesConf {
+    INSTANCE;
+    private Log log = LogFactory.getLog(ProfilesConf.class);
+    private Map<String, Map<String, String>> profilesMap;
+    private final static String EXTERNAL_PROFILES = "pxf-profiles.xml";
+    private final static String INTERNAL_PROFILES = "pxf-profiles-default.xml";
+
+    /**
+     * Constructs the ProfilesConf enum singleton instance.
+     * <p/>
+     * External profiles take precedence over the internal ones and override them.
+     */
+    private ProfilesConf() {
+        profilesMap = new TreeMap<>(String.CASE_INSENSITIVE_ORDER);
+        loadConf(INTERNAL_PROFILES, true);
+        loadConf(EXTERNAL_PROFILES, false);
+        if (profilesMap.isEmpty()) {
+            throw new ProfileConfException(PROFILES_FILE_NOT_FOUND, EXTERNAL_PROFILES);
+        }
+        log.info("PXF profiles loaded: " + profilesMap.keySet());
+    }
+
+    /**
+     * Get requested profile plugins map.
+     * In case pxf-profiles.xml is not on the classpath, or it doesn't contains the requested profile,
+     * Fallback to pxf-profiles-default.xml occurs (@see useProfilesDefaults(String msgFormat))
+     *
+     * @param profile The requested profile
+     * @return Plugins map of the requested profile
+     */
+    public static Map<String, String> getProfilePluginsMap(String profile) {
+        Map<String, String> pluginsMap = INSTANCE.profilesMap.get(profile);
+        if (pluginsMap == null) {
+            throw new ProfileConfException(NO_PROFILE_DEF, profile, EXTERNAL_PROFILES);
+        }
+        return pluginsMap;
+    }
+
+    private ClassLoader getClassLoader() {
+        ClassLoader cL = Thread.currentThread().getContextClassLoader();
+        return (cL != null)
+                ? cL
+                : ProfilesConf.class.getClassLoader();
+    }
+
+    private void loadConf(String fileName, boolean isMandatory) {
+        URL url = getClassLoader().getResource(fileName);
+        if (url == null) {
+            log.warn(fileName + " not found in the classpath");
+            if (isMandatory) {
+                throw new ProfileConfException(PROFILES_FILE_NOT_FOUND, fileName);
+            }
+            return;
+        }
+        try {
+            XMLConfiguration conf = new XMLConfiguration(url);
+            loadMap(conf);
+        } catch (ConfigurationException e) {
+            throw new ProfileConfException(PROFILES_FILE_LOAD_ERR, url.getFile(), String.valueOf(e.getCause()));
+        }
+    }
+
+    private void loadMap(XMLConfiguration conf) {
+        String[] profileNames = conf.getStringArray("profile.name");
+        if (profileNames.length == 0) {
+            log.warn("Profile file: " + conf.getFileName() + " is empty");
+            return;
+        }
+        Map<String, Map<String, String>> profileMap = new TreeMap<>(String.CASE_INSENSITIVE_ORDER);
+        for (int profileIdx = 0; profileIdx < profileNames.length; profileIdx++) {
+            String profileName = profileNames[profileIdx];
+            if (profileMap.containsKey(profileName)) {
+                log.warn("Duplicate profile definition found in " + conf.getFileName() + " for: " + profileName);
+                continue;
+            }
+            Configuration profileSubset = conf.subset("profile(" + profileIdx + ").plugins");
+            profileMap.put(profileName, getProfilePluginMap(profileSubset));
+        }
+        profilesMap.putAll(profileMap);
+    }
+
+    private Map<String, String> getProfilePluginMap(Configuration profileSubset) {
+        @SuppressWarnings("unchecked") //IteratorUtils doesn't yet support generics.
+        List<String> plugins = IteratorUtils.toList(profileSubset.getKeys());
+        Map<String, String> pluginsMap = new HashMap<>();
+        for (String plugin : plugins) {
+            String pluginValue = profileSubset.getString(plugin);
+            if (!StringUtils.isEmpty(StringUtils.trim(pluginValue))) {
+                pluginsMap.put("X-GP-" + plugin.toUpperCase(), pluginValue);
+            }
+        }
+        return pluginsMap;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/1c7ab9eb/pxf/pxf-api/src/test/java/com/pivotal/pxf/api/FilterParserTest.java
----------------------------------------------------------------------
diff --git a/pxf/pxf-api/src/test/java/com/pivotal/pxf/api/FilterParserTest.java b/pxf/pxf-api/src/test/java/com/pivotal/pxf/api/FilterParserTest.java
deleted file mode 100644
index 1f54f0c..0000000
--- a/pxf/pxf-api/src/test/java/com/pivotal/pxf/api/FilterParserTest.java
+++ /dev/null
@@ -1,284 +0,0 @@
-package com.pivotal.pxf.api;
-
-import com.pivotal.pxf.api.FilterParser.FilterBuilder;
-import com.pivotal.pxf.api.FilterParser.Operation;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.powermock.core.classloader.annotations.PrepareForTest;
-import org.powermock.modules.junit4.PowerMockRunner;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.eq;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-@RunWith(PowerMockRunner.class)
-@PrepareForTest({FilterBuilder.class})
-public class FilterParserTest {
-
-    FilterBuilder filterBuilder;
-    FilterParser filterParser;
-    String filter, exception;
-
-    @Before
-    public void setUp() throws Exception {
-        filterBuilder = mock(FilterBuilder.class);
-        filterParser = new FilterParser(filterBuilder);
-    }
-
-    @Test
-    public void parseNegativeNull() {
-        filter = null;
-        runParseNegative("null string", null, "filter parsing ended with no result");
-    }
-
-    @Test
-    public void parseNegativeEmpty() {
-        filter = "";
-        runParseNegative("empty string", filter, "filter parsing ended with no result");
-    }
-
-    @Test
-    public void parseNegativeNotOperand() {
-        filter = "g is not an operand";
-        int index = 0;
-        char op = filter.charAt(index);
-
-        runParseNegative("illegal operand g", filter,
-                "unknown opcode " + op + "(" + (int) op + ") at " + index);
-    }
-
-    @Test
-    public void parseNegativeBadNumber() {
-
-        filter = "a";
-        int index = 1;
-        exception = "numeric argument expected at " + index;
-
-        runParseNegative("numeric operand with no number", filter, exception);
-
-        filter = "aa";
-        exception = "numeric argument expected at " + index;
-
-        runParseNegative("numeric operand with non-number value", filter, exception);
-
-        filter = "a12345678901234567890123456789";
-        exception = "invalid numeric argument 12345678901234567890123456789";
-
-        runParseNegative("numeric operand with too big number", filter, exception);
-
-        filter = "a-12345678901234567890";
-        exception = "invalid numeric argument -12345678901234567890";
-
-        runParseNegative("numeric operand with too big negative number", filter, exception);
-
-        filter = "a12345678901223456";
-        exception = "value 12345678901223456 larger than intmax ending at " + filter.length();
-
-        runParseNegative("numeric operand with long value", filter, exception);
-
-        filter = "a-12345678901223456";
-        exception = "value -12345678901223456 larger than intmax ending at " + filter.length();
-
-        runParseNegative("numeric operand with negative long value", filter, exception);
-    }
-
-    @Test
-    public void parseNegativeBadConst() {
-        filter = "c";
-        int index = 1;
-        exception = "argument should follow at " + index;
-        runParseNegative("const operand with no value", filter, exception);
-
-        filter = "cyan";
-        exception = "numeric argument expected at " + index;
-        runParseNegative("const operand with illegal value", filter, exception);
-
-        filter = "c\"and that's it";
-        exception = "string started at " + index + " not ended with \"";
-        runParseNegative("string without closing \"", filter, exception);
-    }
-
-    @Test
-    public void parseNegativeBadOperation() {
-        filter = "o";
-        int index = 1;
-        exception = "numeric argument expected at " + index;
-        runParseNegative("operation with no value", filter, exception);
-
-        filter = "ohno";
-        exception = "numeric argument expected at " + index;
-        runParseNegative("operation with no number", filter, exception);
-
-        filter = "o100";
-        index = 4;
-        exception = "unknown op ending at " + index;
-        runParseNegative("operation with out of bounds number", filter, exception);
-    }
-
-
-    @Test
-    public void parseNegativeNoOperator() {
-
-        filter = "a1234567890";
-        runParseNegative("filter with only column", filter, "filter parsing failed, missing operators?");
-
-        filter = "c1";
-        runParseNegative("filter with only numeric const", filter, "filter parsing failed, missing operators?");
-
-        filter = "c\"something in the way\"";
-        runParseNegative("filter with only string const", filter, "filter parsing failed, missing operators?");
-    }
-
-    @Test
-    public void parseNegativeTwoParams() {
-
-        filter = "c1c2";
-        exception = "Stack not empty, missing operators?";
-        runParseNegative("filter with two consts in a row", filter, exception);
-
-        filter = "c1a1";
-        exception = "Stack not empty, missing operators?";
-        runParseNegative("filter with const and attribute", filter, exception);
-
-        filter = "a1c80";
-        exception = "Stack not empty, missing operators?";
-        runParseNegative("filter with attribute and const", filter, exception);
-    }
-
-    @Test
-    public void parseNegativeOperationFirst() {
-
-        filter = "o1a3";
-        int index = 2;
-        FilterParser.Operation operation = FilterParser.Operation.HDOP_LT;
-        exception = "missing operands for op " + operation + " at " + index;
-        runParseNegative("filter with operation first", filter, exception);
-
-        filter = "a2o1";
-        index = 4;
-        exception = "missing operands for op " + operation + " at " + index;
-        runParseNegative("filter with only attribute before operation", filter, exception);
-    }
-
-    @Test
-    public void parseColumnOnLeft() throws Exception {
-
-        filter = "a1c2o1";
-        Operation op = Operation.HDOP_LT;
-
-        runParseOneOperation("this filter was build from HDOP_LT", filter, op);
-
-        filter = "a1c2o2";
-        op = Operation.HDOP_GT;
-        runParseOneOperation("this filter was build from HDOP_GT", filter, op);
-
-        filter = "a1c2o3";
-        op = Operation.HDOP_LE;
-        runParseOneOperation("this filter was build from HDOP_LE", filter, op);
-
-        filter = "a1c2o4";
-        op = Operation.HDOP_GE;
-        runParseOneOperation("this filter was build from HDOP_GE", filter, op);
-
-        filter = "a1c2o5";
-        op = Operation.HDOP_EQ;
-        runParseOneOperation("this filter was build from HDOP_EQ", filter, op);
-
-        filter = "a1c2o6";
-        op = Operation.HDOP_NE;
-        runParseOneOperation("this filter was build from HDOP_NE", filter, op);
-
-        filter = "a1c2o7";
-        op = Operation.HDOP_AND;
-        runParseOneOperation("this filter was build from HDOP_AND", filter, op);
-    }
-
-    @Test
-    public void parseColumnOnRight() throws Exception {
-
-        filter = "c2a1o1";
-        Operation op = Operation.HDOP_GT;
-        runParseOneOperation("this filter was build from HDOP_LT -> HDOP_GT using reverse!", filter, op);
-
-        filter = "c2a1o2";
-        op = Operation.HDOP_LT;
-        runParseOneOperation("this filter was build from HDOP_GT -> HDOP_LT using reverse!", filter, op);
-
-        filter = "c2a1o3";
-        op = Operation.HDOP_GE;
-        runParseOneOperation("this filter was build from HDOP_LE -> HDOP_GE using reverse!", filter, op);
-
-        filter = "c2a1o4";
-        op = Operation.HDOP_LE;
-        runParseOneOperation("this filter was build from HDOP_GE -> HDOP_LE using reverse!", filter, op);
-
-        filter = "c2a1o5";
-        op = Operation.HDOP_EQ;
-        runParseOneOperation("this filter was build from HDOP_EQ using reverse!", filter, op);
-
-        filter = "c2a1o6";
-        op = Operation.HDOP_NE;
-        runParseOneOperation("this filter was build from HDOP_NE using reverse!", filter, op);
-
-        filter = "c2a1o7";
-        op = Operation.HDOP_AND;
-        runParseOneOperation("this filter was build from HDOP_AND using reverse!", filter, op);
-    }
-
-    @Test
-    public void parseFilterWith2Operations() throws Exception {
-        filter = "a1c\"first\"o5a2c2o2o7";
-
-        Object firstOp = "first operation HDOP_EQ";
-        Object secondOp = "second operation HDOP_GT";
-        Object lastOp = "filter with 2 operations connected by AND";
-
-        when(filterBuilder.build(eq(Operation.HDOP_EQ),
-                any(),
-                any())).thenReturn(firstOp);
-
-        when(filterBuilder.build(eq(Operation.HDOP_GT),
-                any(),
-                any())).thenReturn(secondOp);
-
-        when(filterBuilder.build(eq(Operation.HDOP_AND),
-                eq(firstOp),
-                eq(secondOp))).thenReturn(lastOp);
-
-        Object result = filterParser.parse(filter);
-
-        assertEquals(lastOp, result);
-    }
-
-	/*
-     * Helper functions
-	 */
-    private void runParseNegative(String description, String filter, String exception) {
-        try {
-            filterParser.parse(filter);
-            fail(description + ": should have failed with FilterStringSyntaxException");
-        } catch (FilterParser.FilterStringSyntaxException e) {
-            assertEquals(description, exception + filterStringMsg(filter), e.getMessage());
-        } catch (Exception e) {
-            fail(description + ": should have failed with FilterStringSyntaxException and not " + e.getMessage());
-        }
-    }
-
-    private void runParseOneOperation(String description, String filter, Operation op) throws Exception {
-        when(filterBuilder.build(eq(op),
-                any(),
-                any())).thenReturn(description);
-
-        Object result = filterParser.parse(filter);
-
-        assertEquals(description, result);
-    }
-
-    private String filterStringMsg(String filter) {
-        return " (filter string: '" + filter + "')";
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/1c7ab9eb/pxf/pxf-api/src/test/java/com/pivotal/pxf/api/utilities/ProfilesConfTest.java
----------------------------------------------------------------------
diff --git a/pxf/pxf-api/src/test/java/com/pivotal/pxf/api/utilities/ProfilesConfTest.java b/pxf/pxf-api/src/test/java/com/pivotal/pxf/api/utilities/ProfilesConfTest.java
deleted file mode 100644
index 6891252..0000000
--- a/pxf/pxf-api/src/test/java/com/pivotal/pxf/api/utilities/ProfilesConfTest.java
+++ /dev/null
@@ -1,174 +0,0 @@
-package com.pivotal.pxf.api.utilities;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-import org.junit.runner.RunWith;
-import org.mockito.Mockito;
-import org.powermock.api.mockito.PowerMockito;
-import org.powermock.core.classloader.annotations.PrepareForTest;
-import org.powermock.modules.junit4.PowerMockRunner;
-
-import java.io.File;
-import java.io.IOException;
-import java.nio.file.Files;
-import java.util.Map;
-
-import static com.pivotal.pxf.api.utilities.ProfileConfException.MessageFormat.NO_PROFILE_DEF;
-import static com.pivotal.pxf.api.utilities.ProfileConfException.MessageFormat.PROFILES_FILE_NOT_FOUND;
-import static org.junit.Assert.*;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-/**
- * Base test class for all ProfilesConf tests.
- * Each test case is encapsulated inside its own inner class to force reloading of ProfilesConf enum singleton
- */
-@RunWith(PowerMockRunner.class)
-@PrepareForTest({ProfilesConf.class, Log.class, LogFactory.class, ClassLoader.class})
-public class ProfilesConfTest {
-    static ClassLoader classLoader;
-    static Log log;
-    String mandatoryFileName = "mandatory.xml";
-    String optionalFileName = "optional.xml";
-    File mandatoryFile;
-    File optionalFile;
-
-    @Rule
-    public TemporaryFolder testFolder = new TemporaryFolder();
-
-    @Before
-    public void setUp() throws Exception {
-        mandatoryFile = testFolder.newFile(mandatoryFileName);
-        optionalFile = testFolder.newFile(optionalFileName);
-        PowerMockito.mockStatic(LogFactory.class);
-        log = mock(Log.class);
-        when(LogFactory.getLog(ProfilesConf.class)).thenReturn(log);
-        classLoader = mock(ClassLoader.class);
-        PowerMockito.stub(PowerMockito.method(ProfilesConf.class, "getClassLoader")).toReturn(classLoader);
-    }
-
-    void writeFile(File file, String content) throws IOException {
-        Files.write(file.toPath(), content.getBytes());
-    }
-}
-
-class ProfilesConfTestDefinedProfile extends ProfilesConfTest {
-    @Test
-    public void definedProfile() throws Exception {
-        writeFile(mandatoryFile, "<profiles><profile><name>HBase</name><plugins><plugin1>X</plugin1><plugin2>XX</plugin2></plugins></profile></profiles>");
-        writeFile(optionalFile, "<profiles><profile><name>Hive</name><plugins><plugin1>Y</plugin1></plugins></profile></profiles>");
-        when(classLoader.getResource("pxf-profiles-default.xml")).thenReturn(mandatoryFile.toURI().toURL());
-        when(classLoader.getResource("pxf-profiles.xml")).thenReturn(optionalFile.toURI().toURL());
-
-        Map<String, String> hbaseProfile = ProfilesConf.getProfilePluginsMap("HBase");
-        assertEquals(2, hbaseProfile.keySet().size());
-        assertEquals(hbaseProfile.get("X-GP-PLUGIN1"), "X");
-        assertEquals(hbaseProfile.get("X-GP-PLUGIN2"), "XX");
-
-        Map<String, String> hiveProfile = ProfilesConf.getProfilePluginsMap("hIVe");// case insensitive profile name
-        assertEquals(1, hiveProfile.keySet().size());
-        assertEquals(hiveProfile.get("X-GP-PLUGIN1"), "Y");
-
-        Mockito.verify(log).info("PXF profiles loaded: [HBase, Hive]");
-    }
-}
-
-class ProfilesConfTestUndefinedProfile extends ProfilesConfTest {
-    @Test
-    public void undefinedProfile() throws Exception {
-        writeFile(mandatoryFile, "<profiles><profile><name>HBase</name><plugins><plugin1>X</plugin1></plugins></profile></profiles>");
-        writeFile(optionalFile, "<profiles><profile><name>Hive</name><plugins><plugin1>Y</plugin1></plugins></profile></profiles>");
-        when(classLoader.getResource("pxf-profiles-default.xml")).thenReturn(mandatoryFile.toURI().toURL());
-        when(classLoader.getResource("pxf-profiles.xml")).thenReturn(optionalFile.toURI().toURL());
-        try {
-            ProfilesConf.getProfilePluginsMap("UndefinedProfile");
-            fail("undefined profile should have thrown exception");
-        } catch (ProfileConfException pce) {
-            assertEquals(pce.getMessage(), String.format(NO_PROFILE_DEF.getFormat(), "UndefinedProfile", "pxf-profiles.xml"));
-        }
-    }
-}
-
-class ProfilesConfTestDuplicateProfileDefinition extends ProfilesConfTest {
-    @Test
-    public void duplicateProfileDefinition() throws Exception {
-        writeFile(mandatoryFile, "<profiles><profile><name>HBase</name><plugins><plugin1>Y</plugin1><plugin1>YY</plugin1></plugins></profile><profile><name>HBase</name><plugins><plugin1>Y</plugin1></plugins></profile></profiles>");
-        writeFile(optionalFile, "<profiles><profile><name>Hive</name><plugins><plugin1>Y</plugin1></plugins></profile></profiles>");
-        when(classLoader.getResource("pxf-profiles-default.xml")).thenReturn(mandatoryFile.toURI().toURL());
-        when(classLoader.getResource("pxf-profiles.xml")).thenReturn(optionalFile.toURI().toURL());
-        ProfilesConf.getProfilePluginsMap("HBase");
-        Mockito.verify(log).warn("Duplicate profile definition found in " + mandatoryFileName + " for: HBase");
-    }
-}
-
-class ProfilesConfTestOverrideProfile extends ProfilesConfTest {
-    @Test
-    public void overrideProfile() throws Exception {
-        writeFile(mandatoryFile, "<profiles><profile><name>HBase</name><plugins><plugin1>X</plugin1></plugins></profile></profiles>");
-        writeFile(optionalFile, "<profiles><profile><name>HBase</name><plugins><plugin1>Y</plugin1><plugin2>YY</plugin2></plugins></profile></profiles>");
-        when(classLoader.getResource("pxf-profiles-default.xml")).thenReturn(mandatoryFile.toURI().toURL());
-        when(classLoader.getResource("pxf-profiles.xml")).thenReturn(optionalFile.toURI().toURL());
-        Map profile = ProfilesConf.getProfilePluginsMap("HBase");
-        assertEquals(2, profile.keySet().size());
-        assertEquals(profile.get("X-GP-PLUGIN1"), "Y");
-        assertEquals(profile.get("X-GP-PLUGIN2"), "YY");
-    }
-}
-
-class ProfilesConfTestEmptyProfileFile extends ProfilesConfTest {
-    @Test
-    public void emptyProfileFile() throws Exception {
-        writeFile(mandatoryFile, "<profiles/>");
-        writeFile(optionalFile, "<profiles><profile><name>HBase</name><plugins><plugin1>Y</plugin1></plugins></profile></profiles>");
-        when(classLoader.getResource("pxf-profiles-default.xml")).thenReturn(mandatoryFile.toURI().toURL());
-        when(classLoader.getResource("pxf-profiles.xml")).thenReturn(optionalFile.toURI().toURL());
-        ProfilesConf.getProfilePluginsMap("HBase");
-        Mockito.verify(log).warn("Profile file: " + mandatoryFileName + " is empty");
-    }
-}
-
-class ProfilesConfTestMalformedProfileFile extends ProfilesConfTest {
-    @Test
-    public void malformedProfileFile() throws Exception {
-        writeFile(mandatoryFile, "I'm a malford x.m.l@#$#<%");
-        writeFile(optionalFile, "<profiles><profile><name>HBase</name><plugins><plugin1>Y</plugin1></plugins></profile></profiles>");
-        when(classLoader.getResource("pxf-profiles-default.xml")).thenReturn(mandatoryFile.toURI().toURL());
-        when(classLoader.getResource("pxf-profiles.xml")).thenReturn(optionalFile.toURI().toURL());
-        try {
-            ProfilesConf.getProfilePluginsMap("HBase");
-            fail("malformed profile file should have thrown exception");
-        } catch (ExceptionInInitializerError pce) {
-            assertTrue(pce.getCause().getMessage().contains(mandatoryFileName + " could not be loaded: org.xml.sax.SAXParseException"));
-        }
-    }
-}
-
-class ProfilesConfTestMissingMandatoryProfileFile extends ProfilesConfTest {
-    @Test
-    public void missingMandatoryProfileFile() throws Exception {
-        when(classLoader.getResource("pxf-profiles-default.xml")).thenReturn(null);
-        try {
-            ProfilesConf.getProfilePluginsMap("HBase");
-            fail("missing mandatory profile file should have thrown exception");
-        } catch (ExceptionInInitializerError pce) {
-            Mockito.verify(log).warn("pxf-profiles-default.xml not found in the classpath");
-            assertEquals(pce.getCause().getMessage(), String.format(PROFILES_FILE_NOT_FOUND.getFormat(), "pxf-profiles-default.xml"));
-        }
-    }
-}
-
-class ProfilesConfTestMissingOptionalProfileFile extends ProfilesConfTest {
-    @Test
-    public void missingOptionalProfileFile() throws Exception {
-        writeFile(mandatoryFile, "<profiles><profile><name>HBase</name><plugins><plugin1>Y</plugin1></plugins></profile></profiles>");
-        when(classLoader.getResource("pxf-profiles-default.xml")).thenReturn(mandatoryFile.toURI().toURL());
-        when(classLoader.getResource("pxf-profiles.xml")).thenReturn(null);
-        Map<String, String> hbaseProfile = ProfilesConf.getProfilePluginsMap("HBase");
-        assertEquals("Y", hbaseProfile.get("X-GP-PLUGIN1"));
-        Mockito.verify(log).warn("pxf-profiles.xml not found in the classpath");
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/1c7ab9eb/pxf/pxf-api/src/test/java/org/apache/hawq/pxf/api/FilterParserTest.java
----------------------------------------------------------------------
diff --git a/pxf/pxf-api/src/test/java/org/apache/hawq/pxf/api/FilterParserTest.java b/pxf/pxf-api/src/test/java/org/apache/hawq/pxf/api/FilterParserTest.java
new file mode 100644
index 0000000..5e777a7
--- /dev/null
+++ b/pxf/pxf-api/src/test/java/org/apache/hawq/pxf/api/FilterParserTest.java
@@ -0,0 +1,284 @@
+package org.apache.hawq.pxf.api;
+
+import org.apache.hawq.pxf.api.FilterParser.FilterBuilder;
+import org.apache.hawq.pxf.api.FilterParser.Operation;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.powermock.core.classloader.annotations.PrepareForTest;
+import org.powermock.modules.junit4.PowerMockRunner;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+@RunWith(PowerMockRunner.class)
+@PrepareForTest({FilterBuilder.class})
+public class FilterParserTest {
+
+    FilterBuilder filterBuilder;
+    FilterParser filterParser;
+    String filter, exception;
+
+    @Before
+    public void setUp() throws Exception {
+        filterBuilder = mock(FilterBuilder.class);
+        filterParser = new FilterParser(filterBuilder);
+    }
+
+    @Test
+    public void parseNegativeNull() {
+        filter = null;
+        runParseNegative("null string", null, "filter parsing ended with no result");
+    }
+
+    @Test
+    public void parseNegativeEmpty() {
+        filter = "";
+        runParseNegative("empty string", filter, "filter parsing ended with no result");
+    }
+
+    @Test
+    public void parseNegativeNotOperand() {
+        filter = "g is not an operand";
+        int index = 0;
+        char op = filter.charAt(index);
+
+        runParseNegative("illegal operand g", filter,
+                "unknown opcode " + op + "(" + (int) op + ") at " + index);
+    }
+
+    @Test
+    public void parseNegativeBadNumber() {
+
+        filter = "a";
+        int index = 1;
+        exception = "numeric argument expected at " + index;
+
+        runParseNegative("numeric operand with no number", filter, exception);
+
+        filter = "aa";
+        exception = "numeric argument expected at " + index;
+
+        runParseNegative("numeric operand with non-number value", filter, exception);
+
+        filter = "a12345678901234567890123456789";
+        exception = "invalid numeric argument 12345678901234567890123456789";
+
+        runParseNegative("numeric operand with too big number", filter, exception);
+
+        filter = "a-12345678901234567890";
+        exception = "invalid numeric argument -12345678901234567890";
+
+        runParseNegative("numeric operand with too big negative number", filter, exception);
+
+        filter = "a12345678901223456";
+        exception = "value 12345678901223456 larger than intmax ending at " + filter.length();
+
+        runParseNegative("numeric operand with long value", filter, exception);
+
+        filter = "a-12345678901223456";
+        exception = "value -12345678901223456 larger than intmax ending at " + filter.length();
+
+        runParseNegative("numeric operand with negative long value", filter, exception);
+    }
+
+    @Test
+    public void parseNegativeBadConst() {
+        filter = "c";
+        int index = 1;
+        exception = "argument should follow at " + index;
+        runParseNegative("const operand with no value", filter, exception);
+
+        filter = "cyan";
+        exception = "numeric argument expected at " + index;
+        runParseNegative("const operand with illegal value", filter, exception);
+
+        filter = "c\"and that's it";
+        exception = "string started at " + index + " not ended with \"";
+        runParseNegative("string without closing \"", filter, exception);
+    }
+
+    @Test
+    public void parseNegativeBadOperation() {
+        filter = "o";
+        int index = 1;
+        exception = "numeric argument expected at " + index;
+        runParseNegative("operation with no value", filter, exception);
+
+        filter = "ohno";
+        exception = "numeric argument expected at " + index;
+        runParseNegative("operation with no number", filter, exception);
+
+        filter = "o100";
+        index = 4;
+        exception = "unknown op ending at " + index;
+        runParseNegative("operation with out of bounds number", filter, exception);
+    }
+
+
+    @Test
+    public void parseNegativeNoOperator() {
+
+        filter = "a1234567890";
+        runParseNegative("filter with only column", filter, "filter parsing failed, missing operators?");
+
+        filter = "c1";
+        runParseNegative("filter with only numeric const", filter, "filter parsing failed, missing operators?");
+
+        filter = "c\"something in the way\"";
+        runParseNegative("filter with only string const", filter, "filter parsing failed, missing operators?");
+    }
+
+    @Test
+    public void parseNegativeTwoParams() {
+
+        filter = "c1c2";
+        exception = "Stack not empty, missing operators?";
+        runParseNegative("filter with two consts in a row", filter, exception);
+
+        filter = "c1a1";
+        exception = "Stack not empty, missing operators?";
+        runParseNegative("filter with const and attribute", filter, exception);
+
+        filter = "a1c80";
+        exception = "Stack not empty, missing operators?";
+        runParseNegative("filter with attribute and const", filter, exception);
+    }
+
+    @Test
+    public void parseNegativeOperationFirst() {
+
+        filter = "o1a3";
+        int index = 2;
+        FilterParser.Operation operation = FilterParser.Operation.HDOP_LT;
+        exception = "missing operands for op " + operation + " at " + index;
+        runParseNegative("filter with operation first", filter, exception);
+
+        filter = "a2o1";
+        index = 4;
+        exception = "missing operands for op " + operation + " at " + index;
+        runParseNegative("filter with only attribute before operation", filter, exception);
+    }
+
+    @Test
+    public void parseColumnOnLeft() throws Exception {
+
+        filter = "a1c2o1";
+        Operation op = Operation.HDOP_LT;
+
+        runParseOneOperation("this filter was build from HDOP_LT", filter, op);
+
+        filter = "a1c2o2";
+        op = Operation.HDOP_GT;
+        runParseOneOperation("this filter was build from HDOP_GT", filter, op);
+
+        filter = "a1c2o3";
+        op = Operation.HDOP_LE;
+        runParseOneOperation("this filter was build from HDOP_LE", filter, op);
+
+        filter = "a1c2o4";
+        op = Operation.HDOP_GE;
+        runParseOneOperation("this filter was build from HDOP_GE", filter, op);
+
+        filter = "a1c2o5";
+        op = Operation.HDOP_EQ;
+        runParseOneOperation("this filter was build from HDOP_EQ", filter, op);
+
+        filter = "a1c2o6";
+        op = Operation.HDOP_NE;
+        runParseOneOperation("this filter was build from HDOP_NE", filter, op);
+
+        filter = "a1c2o7";
+        op = Operation.HDOP_AND;
+        runParseOneOperation("this filter was build from HDOP_AND", filter, op);
+    }
+
+    @Test
+    public void parseColumnOnRight() throws Exception {
+
+        filter = "c2a1o1";
+        Operation op = Operation.HDOP_GT;
+        runParseOneOperation("this filter was build from HDOP_LT -> HDOP_GT using reverse!", filter, op);
+
+        filter = "c2a1o2";
+        op = Operation.HDOP_LT;
+        runParseOneOperation("this filter was build from HDOP_GT -> HDOP_LT using reverse!", filter, op);
+
+        filter = "c2a1o3";
+        op = Operation.HDOP_GE;
+        runParseOneOperation("this filter was build from HDOP_LE -> HDOP_GE using reverse!", filter, op);
+
+        filter = "c2a1o4";
+        op = Operation.HDOP_LE;
+        runParseOneOperation("this filter was build from HDOP_GE -> HDOP_LE using reverse!", filter, op);
+
+        filter = "c2a1o5";
+        op = Operation.HDOP_EQ;
+        runParseOneOperation("this filter was build from HDOP_EQ using reverse!", filter, op);
+
+        filter = "c2a1o6";
+        op = Operation.HDOP_NE;
+        runParseOneOperation("this filter was build from HDOP_NE using reverse!", filter, op);
+
+        filter = "c2a1o7";
+        op = Operation.HDOP_AND;
+        runParseOneOperation("this filter was build from HDOP_AND using reverse!", filter, op);
+    }
+
+    @Test
+    public void parseFilterWith2Operations() throws Exception {
+        filter = "a1c\"first\"o5a2c2o2o7";
+
+        Object firstOp = "first operation HDOP_EQ";
+        Object secondOp = "second operation HDOP_GT";
+        Object lastOp = "filter with 2 operations connected by AND";
+
+        when(filterBuilder.build(eq(Operation.HDOP_EQ),
+                any(),
+                any())).thenReturn(firstOp);
+
+        when(filterBuilder.build(eq(Operation.HDOP_GT),
+                any(),
+                any())).thenReturn(secondOp);
+
+        when(filterBuilder.build(eq(Operation.HDOP_AND),
+                eq(firstOp),
+                eq(secondOp))).thenReturn(lastOp);
+
+        Object result = filterParser.parse(filter);
+
+        assertEquals(lastOp, result);
+    }
+
+	/*
+     * Helper functions
+	 */
+    private void runParseNegative(String description, String filter, String exception) {
+        try {
+            filterParser.parse(filter);
+            fail(description + ": should have failed with FilterStringSyntaxException");
+        } catch (FilterParser.FilterStringSyntaxException e) {
+            assertEquals(description, exception + filterStringMsg(filter), e.getMessage());
+        } catch (Exception e) {
+            fail(description + ": should have failed with FilterStringSyntaxException and not " + e.getMessage());
+        }
+    }
+
+    private void runParseOneOperation(String description, String filter, Operation op) throws Exception {
+        when(filterBuilder.build(eq(op),
+                any(),
+                any())).thenReturn(description);
+
+        Object result = filterParser.parse(filter);
+
+        assertEquals(description, result);
+    }
+
+    private String filterStringMsg(String filter) {
+        return " (filter string: '" + filter + "')";
+    }
+}