You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hawq.apache.org by yo...@apache.org on 2017/02/08 22:51:49 UTC

incubator-hawq-docs git commit: HAWQ-1304 - multiple doc changes for PXF and Hive Plugin (closes #94)

Repository: incubator-hawq-docs
Updated Branches:
  refs/heads/develop 1102fdc04 -> dcfe1a47a


HAWQ-1304 - multiple doc changes for PXF and Hive Plugin (closes #94)


Project: http://git-wip-us.apache.org/repos/asf/incubator-hawq-docs/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-hawq-docs/commit/dcfe1a47
Tree: http://git-wip-us.apache.org/repos/asf/incubator-hawq-docs/tree/dcfe1a47
Diff: http://git-wip-us.apache.org/repos/asf/incubator-hawq-docs/diff/dcfe1a47

Branch: refs/heads/develop
Commit: dcfe1a47a71dc7eb501d0a74aa5605cf2deba3e5
Parents: 1102fdc
Author: Lisa Owen <lo...@pivotal.io>
Authored: Wed Feb 8 14:51:42 2017 -0800
Committer: David Yozie <yo...@apache.org>
Committed: Wed Feb 8 14:51:42 2017 -0800

----------------------------------------------------------------------
 markdown/pxf/HivePXF.html.md.erb                | 301 +++++++++----------
 .../PXFExternalTableandAPIReference.html.md.erb | 206 ++++++++-----
 markdown/pxf/ReadWritePXF.html.md.erb           |  24 +-
 markdown/pxf/TroubleshootingPXF.html.md.erb     |   4 +
 4 files changed, 301 insertions(+), 234 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-hawq-docs/blob/dcfe1a47/markdown/pxf/HivePXF.html.md.erb
----------------------------------------------------------------------
diff --git a/markdown/pxf/HivePXF.html.md.erb b/markdown/pxf/HivePXF.html.md.erb
index e0f6c79..51b317c 100644
--- a/markdown/pxf/HivePXF.html.md.erb
+++ b/markdown/pxf/HivePXF.html.md.erb
@@ -25,15 +25,17 @@ Apache Hive is a distributed data warehousing infrastructure.  Hive facilitates
 
 This section describes how to use PXF to access Hive data. Options for querying data stored in Hive include:
 
--  Creating an external table in PXF and querying that table
 -  Querying Hive tables via PXF's integration with HCatalog
+-  Creating an external table in PXF and querying that table
 
 ## <a id="installingthepxfhiveplugin"></a>Prerequisites
 
 Before accessing Hive data with HAWQ and PXF, ensure that:
 
--   The PXF HDFS plug-in is installed on all cluster nodes. See [Installing PXF Plug-ins](InstallPXFPlugins.html) for PXF plug-in installation information.
--   The PXF Hive plug-in is installed on all cluster nodes.
+-   The PXF HDFS plug-in is installed on all HAWQ and HDFS cluster nodes (master, segment, NameNode, DataNode). See [Installing PXF Plug-ins](InstallPXFPlugins.html) for PXF plug-in installation information.
+-   The PXF Hive plug-in is installed on all HAWQ and HDFS cluster nodes.
+-   If you configured Hadoop with high availability, PXF must also be installed on all HDFS nodes running NameNode services.
+-   The Hive client is installed on all PXF nodes.
 -   The Hive JAR files and conf directory�are installed on all cluster nodes.
 -   You have tested PXF on HDFS.
 -   You are running the Hive Metastore service on a machine in your cluster.�
@@ -80,7 +82,7 @@ The following table summarizes external mapping rules for Hive primitive types.
 
 Hive supports complex data types including array, struct, map, and union. PXF maps each of these complex types to `text`.  While HAWQ does not natively support these types, you can create HAWQ functions or application code to extract subcomponents of these complex data types.
 
-An example using complex data types is provided later in this topic.
+An example using complex data types with the `Hive` profile is provided later in this topic.
 
 
 ## <a id="hive_sampledataset"></a>Sample Data Set
@@ -162,9 +164,143 @@ Create a Hive table to expose our sample data set.
 
 In examples later in this section, you will access the `sales_info` Hive table directly via PXF. You will also insert `sales_info` data into tables of other Hive file format types, and use PXF to access those directly as well.
 
+
+## <a id="hcatalog"></a>Using PXF and HCatalog to Query Hive
+
+You can query Hive tables directly through HCatalog integration with HAWQ and PXF, regardless of the underlying file storage format. This integration allows HAWQ to directly use table metadata stored in HCatalog.
+
+HCatalog is built on top of the Hive metastore and incorporates Hive's DDL. This provides several advantages:
+
+-   You do not need to know the table schema of your Hive tables
+-   You do not need to manually enter information about Hive table location or format
+-   If Hive table metadata changes, HCatalog provides updated metadata. This is in contrast to the use of static external PXF tables to define Hive table metadata for HAWQ.
+
+The following diagram depicts how HAWQ integrates with HCatalog to query Hive tables:
+
+<img src="../images/hawq_hcatalog.png" id="hcatalog__image_ukw_h2v_c5" class="image" width="672" />
+
+1.  HAWQ retrieves table metadata from HCatalog using PXF.
+2.  HAWQ creates in-memory catalog tables from the retrieved metadata. If a table is referenced multiple times in a transaction, HAWQ uses its in-memory metadata to reduce external calls to HCatalog.
+3.  PXF queries Hive using table metadata that is stored in the HAWQ in-memory catalog tables. Table metadata is dropped at the end of the transaction.
+
+
+### <a id="topic_j1l_enabling"></a>Enabling HCatalog Integration
+
+To enable HCatalog query integration in HAWQ, perform the following steps:
+
+1.  Make sure your deployment meets the requirements listed in [Prerequisites](#installingthepxfhiveplugin).
+2.  If necessary, set the `pxf_service_address` global configuration property to the hostname or IP address and port where you have installed the PXF Hive plug-in. By default, the value is set to `localhost:51200`.
+
+    ``` sql
+    postgres=# SET pxf_service_address TO <hivenode>:51200
+    ```
+
+3.  HCatalog internally uses the `pxf` protocol to query.  Grant this protocol privilege to all roles requiring access:
+
+    ``` sql
+    postgres=# GRANT ALL ON PROTOCOL pxf TO <role>;
+    ```
+
+4. It is not recommended to create a HAWQ table using the `WITH (OIDS)` clause. If any user tables were created using the `WITH (OIDS)` clause, additional operations are required to enable HCatalog integration. To access a Hive table via HCatalog when user tables were created using `WITH (OIDS)`, HAWQ users must have `SELECT` permission to query every user table within the same schema that was created using the `WITH (OIDS)` clause.
+
+    1. Determine which user tables were created using the `WITH (OIDS)` clause:
+
+        ``` sql
+        postgres=# SELECT oid, relname FROM pg_class
+                     WHERE relhasoids = true
+                       AND relnamespace <> (SELECT oid FROM pg_namespace WHERE nspname = 'pg_catalog');
+        ```
+
+    2. Grant `SELECT` privilege on all returned tables to all roles to which you chose to provide HCatalog query access. For example:
+
+        ``` sql
+        postgres=# GRANT SELECT ON <table-created-WITH-OIDS> TO <role>
+        ```
+
+### <a id="topic_j1l_y55_c5"></a>Usage
+
+To query a Hive table with HCatalog integration, query HCatalog directly from HAWQ. The query syntax is:
+
+``` sql
+postgres=# SELECT * FROM hcatalog.hive-db-name.hive-table-name;
+```
+
+For example:
+
+``` sql
+postgres=# SELECT * FROM hcatalog.default.sales_info;
+```
+
+To obtain a description of a Hive table with HCatalog integration, you can use the `psql` client interface.
+
+-   Within HAWQ, use either the `\d                                         hcatalog.hive-db-name.hive-table-name` or `\d+                                         hcatalog.hive-db-name.hive-table-name` commands to describe a single table.  `\d` displays only HAWQ's interpretation of the underlying source (Hive in this case) data type, while `\d+` displays both the HAWQ interpreted and Hive source data types. For example, from the `psql` client interface:
+
+    ``` shell
+    $ psql -d postgres
+    ```
+
+    ``` sql
+    postgres=# \d+ hcatalog.default.sales_info_rcfile;
+    ```
+
+    ``` shell
+       PXF Hive Table "default.sales_info"
+          Column      |  Type  | Source type 
+    ------------------+--------+-------------
+     location         | text   | string
+     month            | text   | string
+     number_of_orders | int4   | int
+     total_sales      | float8 | double
+    ```
+-   Use `\d hcatalog.hive-db-name.*` to describe the whole database schema, i.e. all tables in `hive-db-name`.
+-   Use `\d hcatalog.*.*` to describe the whole schema, i.e. all databases and tables.
+
+When using `\d` or `\d+` commands in the `psql` HAWQ client, `hcatalog` will not be listed as a database. If you use other `psql` compatible clients, `hcatalog` will be listed as a database with a size value of `-1` since `hcatalog` is not a real database in HAWQ.
+
+Alternatively, you can use the `pxf_get_item_fields` user-defined function (UDF) to obtain Hive table descriptions from other client interfaces or third-party applications. The UDF takes a PXF profile and a table pattern string as its input parameters.  **Note:** The only supported input profile at this time is `'Hive'`.
+
+- The following statement returns a description of a specific table. The description includes path, itemname (table), fieldname, fieldtype (HAWQ type), and sourcefieldtype (Hive type).
+
+    ``` sql
+    postgres=# SELECT * FROM pxf_get_item_fields('Hive','default.sales_info');
+    ```
+
+    ``` pre
+      path   |  itemname  |    fieldname     | fieldtype | sourcefieldtype 
+    ---------+------------+------------------+-----------+-----------------
+     default | sales_info | location         | text      | string
+     default | sales_info | month            | text      | string
+     default | sales_info | number_of_orders | int4      | int
+     default | sales_info | total_sales      | float8    | double
+    ```
+
+- The following statement returns table descriptions from the default database.
+
+    ``` sql
+    postgres=# SELECT * FROM pxf_get_item_fields('Hive','default.*');
+    ```
+
+- The following statement returns a description of the entire schema.
+
+    ``` sql
+    postgres=# SELECT * FROM pxf_get_item_fields('Hive', '*.*');
+    ```
+
+### <a id="topic_r5k_pst_25"></a>Limitations
+
+HCatalog integration has the following limitations:
+
+-   HCatalog queries on Hive tables with complex type fields return those fields serialized as text.
+-   Even for primitive types, HCatalog metadata descriptions produced by `\d` are HAWQ's interpretation of the underlying Hive data types. For example, the Hive type `tinyint` is converted to HAWQ type `int2`. (See [Data Type Mapping](#hive_primdatatypes).)
+-   HAWQ reserves the database name `hcatalog` for system use. You cannot connect to or alter the system `hcatalog` database.
+
+
+
 ## <a id="topic_p2s_lvl_28"></a>Querying External Hive Data
 
-The PXF Hive plug-in supports several Hive-related profiles. These include `Hive`, `HiveText`, and `HiveRC`.
+In the previous section, you used HCatalog integration to query a Hive table. You can also create a PXF/HAWQ external table to access Hive table data. This Hive table access mechanism requires that you identify an appropriate Hive profile.
+
+The PXF Hive plug-in supports several Hive-related profiles. These include `Hive`, `HiveText`, and `HiveRC`. The `HiveText` and `HiveRC` profiles are specifically optimized for text and RC file formats, respectively. The `Hive` profile is optimized for all file storage types; use the `Hive` profile when the underlying Hive table is composed of multiple partitions with differing file formats.
 
 Use the following syntax to create a HAWQ external table representing Hive data:
 
@@ -191,10 +327,7 @@ Hive-plug-in-specific keywords and values used in the [CREATE EXTERNAL TABLE](..
 
 ## <a id="profile_hive"></a>Hive Profile
 
-The `Hive` profile works with any Hive file format. It can access heterogenous format data in a single table where each partition may be stored as a different file format.
-
-While you can use the `Hive` profile to access any file format, the more specific profiles perform better for those single file format types.
-
+Use the `Hive` profile with any Hive file storage format. With the `Hive` profile, you can also access heterogenous format data in a single table where each partition may be stored in a different file format. In both cases, the `Hive` profile will use the optimal `Hive*` profile for the underlying file format type.
 
 ### <a id="profile_hive_using"></a>Example: Using the Hive Profile
 
@@ -226,7 +359,7 @@ Use the `Hive` profile to create a queryable HAWQ external table from the Hive `
 
 ## <a id="profile_hivetext"></a>HiveText Profile
 
-Use the `HiveText` profile to query text format files. The `HiveText` profile is more performant than the `Hive` profile.
+Use the `HiveText` profile to query text format files.
 
 **Note**: When using the `HiveText` profile, you *must* specify a delimiter option in *both* the `LOCATION` and `FORMAT` clauses.
 
@@ -266,9 +399,11 @@ Use the PXF `HiveText` profile to create a queryable HAWQ external table from th
 
 The RCFile Hive format is used for row columnar formatted data. The `HiveRC` profile provides access to RCFile data.
 
+**Note**: When using the `HiveRC` profile, you *must* specify a delimiter option in *both* the `LOCATION` and `FORMAT` clauses.
+
 ### <a id="profile_hiverc_rcfiletbl_using"></a>Example: Using the HiveRC Profile
 
-Use the `HiveRC` profile to query RCFile-formatted data in Hive tables. The `HiveRC` profile is more performant than the `Hive` profile for this file format type.
+Use the `HiveRC` profile to query RCFile-formatted data in Hive tables.
 
 1. Create a Hive table with RCFile format:
 
@@ -347,14 +482,9 @@ postgres=# SELECT fname,lname FROM pxf_parquet_table;
 ```
 
 
-## <a id="profileperf"></a>Profile Performance Considerations
-
-The `HiveRC` and `HiveText` profiles are faster than the generic `Hive` profile.
+## <a id="complex_dt_example"></a>Hive Profile Complex Data Type Example
 
-
-## <a id="complex_dt_example"></a>Complex Data Type Example
-
-This example will employ the array and map complex types, specifically an array of integers and a string key/value pair map.
+This example employs the `Hive` profile and the array and map complex types, specifically an array of integers and a string key/value pair map.
 
 The data schema for this example includes fields with the following names and data types:
 
@@ -458,138 +588,7 @@ When specifying an array field in a Hive table, you must identify the terminator
     (10 rows)
     ```
 
-    `intarray` and `propmap` are each text strings.
-
-## <a id="hcatalog"></a>Using PXF and HCatalog to Query Hive
-
-Hive tables can be queried directly through HCatalog integration with HAWQ and PXF, regardless of the underlying file storage format.
-
-In previous sections, you created an external table in PXF that described the target table's Hive metadata. Another option for querying Hive tables is to take advantage of HAWQ's integration with HCatalog. This integration allows HAWQ to directly use table metadata stored in HCatalog.
-
-HCatalog is built on top of the Hive metastore and incorporates Hive's DDL. This provides several advantages:
-
--   You do not need to know the table schema of your Hive tables
--   You do not need to manually enter information about Hive table location or format
--   If Hive table metadata changes, HCatalog provides updated metadata. This is in contrast to the use of static external PXF tables to define Hive table metadata for HAWQ.
-
-The following diagram depicts how HAWQ integrates with HCatalog to query Hive tables:
-
-<img src="../images/hawq_hcatalog.png" id="hcatalog__image_ukw_h2v_c5" class="image" width="672" />
-
-1.  HAWQ retrieves table metadata from HCatalog using PXF.
-2.  HAWQ creates in-memory catalog tables from the retrieved metadata. If a table is referenced multiple times in a transaction, HAWQ uses its in-memory metadata to reduce external calls to HCatalog.
-3.  PXF queries Hive using table metadata that is stored in the HAWQ in-memory catalog tables. Table metadata is dropped at the end of the transaction.
-
-
-### <a id="topic_j1l_enabling"></a>Enabling HCatalog Integration
-
-To enable HCatalog query integration in HAWQ, perform the following steps:
-
-1.  Make sure your deployment meets the requirements listed in [Prerequisites](#installingthepxfhiveplugin).
-2.  If necessary, set the `pxf_service_address` global configuration property to the hostname or IP address and port where you have installed the PXF Hive plug-in. By default, the value is set to `localhost:51200`.
-
-    ``` sql
-    postgres=# SET pxf_service_address TO <hivenode>:51200
-    ```
-
-3.  HCatalog internally uses the `pxf` protocol to query.  Grant this protocol privilege to all roles requiring access:
-
-    ``` sql
-    postgres=# GRANT ALL ON PROTOCOL pxf TO <role>;
-    ```
-
-4. It is not recommended to create a HAWQ table using the `WITH (OIDS)` clause. If any user tables were created using the `WITH (OIDS)` clause, additional operations are required to enable HCatalog integration. To access a Hive table via HCatalog when user tables were created using `WITH (OIDS)`, HAWQ users must have `SELECT` permission to query every user table within the same schema that was created using the `WITH (OIDS)` clause. 
-
-    1. Determine which user tables were created using the `WITH (OIDS)` clause:
-
-        ``` sql
-        postgres=# SELECT oid, relname FROM pg_class 
-                     WHERE relhasoids = true 
-                       AND relnamespace <> (SELECT oid FROM pg_namespace WHERE nspname = 'pg_catalog');
-        ```
-
-    2. Grant `SELECT` privilege on all returned tables to all roles to which you chose to provide HCatalog query access. For example:
-
-        ``` sql
-        postgres=# GRANT SELECT ON <table-created-WITH-OIDS> TO <role>
-        ``` 
-
-### <a id="topic_j1l_y55_c5"></a>Usage    
-
-To query a Hive table with HCatalog integration, query HCatalog directly from HAWQ. The query syntax is:
-
-``` sql
-postgres=# SELECT * FROM hcatalog.hive-db-name.hive-table-name;
-```
-
-For example:
-
-``` sql
-postgres=# SELECT * FROM hcatalog.default.sales_info;
-```
-
-To obtain a description of a Hive table with HCatalog integration, you can use the `psql` client interface.
-
--   Within HAWQ, use either the `\d                                         hcatalog.hive-db-name.hive-table-name` or `\d+                                         hcatalog.hive-db-name.hive-table-name` commands to describe a single table.  `\d` displays only HAWQ's interpretation of the underlying source (Hive in this case) data type, while `\d+` displays both the HAWQ interpreted and Hive source data types. For example, from the `psql` client interface:
-
-    ``` shell
-    $ psql -d postgres
-    ```
-
-    ``` sql
-    postgres=# \d+ hcatalog.default.sales_info_rcfile;
-    ```
-
-    ``` shell
-    PXF Hive Table "default.sales_info_rcfile"
-          Column      |  Type  | Source type 
-    ------------------+--------+-------------
-     location         | text   | string
-     month            | text   | string
-     number_of_orders | int4   | int
-     total_sales      | float8 | double
-    ```
--   Use `\d hcatalog.hive-db-name.*` to describe the whole database schema, i.e. all tables in `hive-db-name`.
--   Use `\d hcatalog.*.*` to describe the whole schema, i.e. all databases and tables.
-
-When using `\d` or `\d+` commands in the `psql` HAWQ client, `hcatalog` will not be listed as a database. If you use other `psql` compatible clients, `hcatalog` will be listed as a database with a size value of `-1` since `hcatalog` is not a real database in HAWQ.
-
-Alternatively, you can use the `pxf_get_item_fields` user-defined function (UDF) to obtain Hive table descriptions from other client interfaces or third-party applications. The UDF takes a PXF profile and a table pattern string as its input parameters.  **Note:** The only supported input profile at this time is `'Hive'`.
-
-- The following statement returns a description of a specific table. The description includes path, itemname (table), fieldname, and fieldtype.
-
-    ``` sql
-    postgres=# SELECT * FROM pxf_get_item_fields('Hive','default.sales_info_rcfile');
-    ```
-
-    ``` pre
-      path   |     itemname      |    fieldname     | fieldtype
-    ---------+-------------------+------------------+-----------
-     default | sales_info_rcfile | location         | text
-     default | sales_info_rcfile | month            | text
-     default | sales_info_rcfile | number_of_orders | int4
-     default | sales_info_rcfile | total_sales      | float8
-    ```
-
-- The following statement returns table descriptions from the default database.
-
-    ``` sql
-    postgres=# SELECT * FROM pxf_get_item_fields('Hive','default.*');
-    ```
-
-- The following statement returns a description of the entire schema.
-
-    ``` sql
-    postgres=# SELECT * FROM pxf_get_item_fields('Hive', '*.*');
-    ```
-
-### <a id="topic_r5k_pst_25"></a>Limitations
-
-HCatalog integration has the following limitations:
-
--   HCatalog integration queries and describe commands do not support complex types; only primitive types are supported. Use PXF external tables to query complex types in Hive. (See [Complex Types Example](#complex_dt_example).)
--   Even for primitive types, HCatalog metadata descriptions produced by `\d` are HAWQ's interpretation of the underlying Hive data types. For example, the Hive type `tinyint` is converted to HAWQ type `int2`. (See [Data Type Mapping](#hive_primdatatypes).)
--   HAWQ reserves the database name `hcatalog` for system use. You cannot connect to or alter the system `hcatalog` database.
+    `intarray` and `propmap` are each serialized as text strings.
 
 ## <a id="partitionfiltering"></a>Partition Filtering
 
@@ -710,7 +709,7 @@ The insertion of row 4 creates a Hive default partition, because the partition c
 In Hive, any query that filters on the partition column omits data in the default partition. For example, the following query returns no rows:
 
 ``` sql
-hive> SELECT * FROM sales WHERE xdate is null;
+hive> SELECT * FROM sales WHERE xdate IS null;
 ```
 
 However, if you map this table as a PXF external table in HAWQ, all default partition values are translated into actual NULL values. In HAWQ, executing the same query against the PXF table returns row 4 as the result, because the filter matches the NULL value.

http://git-wip-us.apache.org/repos/asf/incubator-hawq-docs/blob/dcfe1a47/markdown/pxf/PXFExternalTableandAPIReference.html.md.erb
----------------------------------------------------------------------
diff --git a/markdown/pxf/PXFExternalTableandAPIReference.html.md.erb b/markdown/pxf/PXFExternalTableandAPIReference.html.md.erb
index c947775..8a29d1d 100644
--- a/markdown/pxf/PXFExternalTableandAPIReference.html.md.erb
+++ b/markdown/pxf/PXFExternalTableandAPIReference.html.md.erb
@@ -23,56 +23,73 @@ under the License.
 
 You can use the PXF API to create�your own connectors to access any other type of parallel data store or processing engine.
 
-The PXF Java API lets you extend PXF functionality and add new services and formats without changing HAWQ. The API includes three classes that are extended to allow HAWQ to access an external data source: Fragmenter, Accessor, and Resolver.
+The PXF Java API lets you extend PXF functionality and add new services and formats without changing HAWQ. The API includes three classes that are extended to allow HAWQ to access an external data source: `Fragmenter`, `Accessor`, and `Resolver`.
 
-The Fragmenter produces a list of data fragments that can be read in parallel from the data source. The Accessor produces a list of records from a single fragment, and the Resolver both deserializes and serializes records.
+The `Fragmenter` produces a list of data fragments that can be read in parallel from the data source. The `Accessor` produces a list of records from a single fragment, and the `Resolver` both deserializes and serializes records.
 
-Together, the Fragmenter, Accessor, and Resolver classes implement a connector. PXF includes plug-ins for tables in HDFS, HBase, and Hive.
+Together, the `Fragmenter`, `Accessor`, and `Resolver` classes implement a connector. PXF includes plug-ins for HDFS and JSON files and tables in HBase and Hive.
 
 ## <a id="creatinganexternaltable"></a>Creating an External Table
 
-The syntax for a readable `EXTERNAL TABLE` that uses the PXF protocol is as follows:
+The syntax for an `EXTERNAL TABLE` that uses the PXF protocol is as follows:
 
 ``` sql
-CREATE [READABLE|WRITABLE] EXTERNAL TABLE table_name
-        ( column_name data_type [, ...] | LIKE other_table )
-LOCATION('pxf://host[:port]/path-to-data<pxf parameters>[&custom-option=value...]')
+CREATE [READABLE|WRITABLE] EXTERNAL TABLE <table_name>
+        ( <column_name> <data_type> [, ...] | LIKE <other_table> )
+LOCATION('pxf://<host>[:<port>]/<path-to-data>?<pxf-parameters>[&<custom-option>=<value>[...]]')
 FORMAT 'custom' (formatter='pxfwritable_import|pxfwritable_export');
 ```
 
-�where *&lt;pxf parameters&gt;* is:
+�where \<pxf\-parameters\> is:
 
 ``` pre
-   ?FRAGMENTER=fragmenter_class&ACCESSOR=accessor_class&RESOLVER=resolver_class]
- | ?PROFILE=profile-name
+    [FRAGMENTER=<fragmenter_class>&ACCESSOR=<accessor_class>
+         &RESOLVER=<resolver_class>] | ?PROFILE=profile-name
 ```
+
 <caption><span class="tablecap">Table 1. Parameter values and description</span></caption>
 
 <a id="creatinganexternaltable__table_pfy_htz_4p"></a>
 
 | Parameter               | Value and description                                                                                                                                                                                                                                                          |
 |-------------------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
-| host                    | The current host of the PXF service.                                                                                                                                                                                                                                           |
-| port�                   | Connection port for the PXF service. If the port is omitted, PXF assumes that High Availability (HA) is enabled and connects to the HA name service port, 51200 by default. The HA name service port can be changed by setting the `pxf_service_port` configuration parameter. |
-| *path\_to\_data*        | A directory, file name, wildcard pattern, table name, etc.                                                                                                                                                                                                                     |
-| FRAGMENTER              | The plug-in (Java class) to use for fragmenting data. Used for READABLE external tables only.                                                                                                                                                                                   |
-| ACCESSOR                | The plug-in (Java class) to use for accessing the data. Used for READABLE and WRITABLE tables.                                                                                                                                                                                  |
-| RESOLVER                | The�plug-in (Java class) to use for serializing and deserializing the data. Used for READABLE and WRITABLE tables.                                                                                                                                                              |
-| *custom-option*=*value* | Additional values to pass to the plug-in class. The parameters are passed at runtime to the plug-ins indicated above. The plug-ins can lookup custom options with `org.apache.hawq.pxf.api.utilities.InputData`.�                                                                 |
+| host                    | The HDFS NameNode.                                                                                                                                                                                                                                           |
+| port�                   | Connection port for the PXF service. If the port is omitted, PXF assumes that High Availability (HA) is enabled and connects to the HA name service port, 51200, by default. The HA name service port can be changed by setting the `pxf_service_port` configuration parameter. |
+| \<path\-to\-data\>        | A directory, file name, wildcard pattern, table name, etc.                                                                                                                                                                                                                     |
+| PROFILE              | The profile PXF uses to access the data. PXF supports multiple plug-ins that currently expose profiles named `HBase`, `Hive`, `HiveRC`, `HiveText`, `HiveORC`,  `HdfsTextSimple`, `HdfsTextMulti`, `Avro`, `SequenceWritable`, and `Json`.                                                                                                                                                                                   |
+| FRAGMENTER              | The Java class the plug-in uses for fragmenting data. Used for READABLE external tables only.                                                                                                                                                                                   |
+| ACCESSOR                | The Java class the plug-in uses for accessing the data. Used for READABLE and WRITABLE tables.                                                                                                                                                                                  |
+| RESOLVER                | The�Java class the plug-in uses for serializing and deserializing the data. Used for READABLE and WRITABLE tables.                                                                                                                                                                                                                                                                                                                     |
+| \<custom-option\> | Additional values to pass to the plug-in at runtime. A plug-in can parse custom options with the PXF helper class  `org.apache.hawq.pxf.api.utilities.InputData`.�                                                                 |
 
 **Note:** When creating PXF external tables, you cannot use the `HEADER` option in your `FORMAT` specification.
 
-For more information about this example, see [About the Java Class Services and Formats](#aboutthejavaclassservicesandformats).
 
 ## <a id="aboutthejavaclassservicesandformats"></a>About the Java Class Services and Formats
 
-The `LOCATION` string in a PXF `CREATE EXTERNAL TABLE` statement is a URI that specifies the host and port of an external data source and the path to the data in the external data source. The query portion of the URI, introduced by the question mark (?), must include the required parameters `FRAGMENTER` (readable tables only), `ACCESSOR`, and `RESOLVER`, which specify Java class names that extend the base PXF API plug-in classes. Alternatively, the required parameters can be replaced with a `PROFILE` parameter with the name of a profile defined in the `/etc/conf/pxf-profiles.xml` that defines the required classes.
+The `LOCATION` string in a PXF `CREATE EXTERNAL TABLE` statement is a URI that specifies the host and port of an external data source and the path to the data in the external data source. The query portion of the URI, introduced by the question mark (?), must include the PXF profile name or the plug-in's  `FRAGMENTER` (readable tables only), `ACCESSOR`, and `RESOLVER` class names.
+
+PXF profiles are defined in the `/etc/pxf/conf/pxf-profiles.xml` file. Profile definitions include plug-in class names. For example, the `HdfsTextSimple` profile definition is:
+
+``` xml
+<profile>
+    <name>HdfsTextSimple</name>
+    <description> This profile is suitable for use when reading delimited
+      single line records from plain text files on HDFS.
+    </description>
+    <plugins>
+        <fragmenter>org.apache.hawq.pxf.plugins.hdfs.HdfsDataFragmenter</fragmenter>
+        <accessor>org.apache.hawq.pxf.plugins.hdfs.LineBreakAccessor</accessor>
+        <resolver>org.apache.hawq.pxf.plugins.hdfs.StringPassResolver</resolver>
+    </plugins>
+</profile>
+```
 
-The parameters in the PXF URI are passed from HAWQ as headers to the PXF Java service. You can pass custom information to user-implemented PXF plug-ins by adding optional parameters to the LOCATION string.
+The parameters in the PXF URI are passed from HAWQ as headers to the PXF Java service. You can pass custom information to user-implemented PXF plug-ins by adding optional parameters to the `LOCATION` string.
 
 The Java PXF service�retrieves the source data from the external data source and converts it to a HAWQ-readable table format.
 
-The Accessor, Resolver, and Fragmenter Java classes extend the `org.apache.hawq.pxf.api.utilities.Plugin` class:
+The `Accessor`, `Resolver`, and `Fragmenter` Java classes extend the `org.apache.hawq.pxf.api.utilities.Plugin` class:
 
 ``` java
 package org.apache.hawq.pxf.api.utilities;
@@ -101,7 +118,7 @@ public class Plugin {
 }
 ```
 
-The parameters in the `LOCATION` string are available to the plug-ins through methods in the `org.apache.hawq.pxf.api.utilities.InputData` class. Custom parameters added to the location string can be looked up with the `getUserProperty()` method.
+The parameters in the `LOCATION` string are available to the plug-ins through methods in the `org.apache.hawq.pxf.api.utilities.InputData` class. Plug-ins can look up the custom parameters added to the location string with the `getUserProperty()` method.
 
 ``` java
 /**
@@ -232,25 +249,32 @@ public class InputData {
 
 ### <a id="fragmenter"></a>Fragmenter
 
-**Note:** The Fragmenter Plugin reads data into HAWQ readable external tables. The Fragmenter Plugin cannot write data out of HAWQ into writable external tables.
+**Note:** You use the `Fragmenter` class to read data into HAWQ. You cannot use this class to write data out of HAWQ.
 
-The Fragmenter is responsible for passing datasource metadata back to HAWQ. It also returns a list of data fragments to the Accessor or Resolver. Each data fragment describes some part of the requested data set. It contains the datasource name, such as the file or table name, including the hostname where it is located. For example, if the source is a HDFS file, the Fragmenter returns a list of data fragments containing a HDFS file block.�Each fragment includes the location of the block. If the source data is an HBase table, the Fragmenter returns information about table regions, including their locations.
+The `Fragmenter` is responsible for passing datasource metadata back to HAWQ. It also returns a list of data fragments to the `Accessor` or `Resolver`. Each data fragment describes some part of the requested data set. It contains the datasource name, such as the file or table name, including the hostname where it is located. For example, if the source is an HDFS file, the `Fragmenter` returns a list of data fragments containing an HDFS file block.�Each fragment includes the location of the block. If the source data is an HBase table, the `Fragmenter` returns information about table regions, including their locations.
 
 The `ANALYZE` command now retrieves advanced statistics for PXF readable tables by estimating the number of tuples in a table, creating a sample table from the external table, and running advanced statistics queries on the sample table in the same way statistics are collected for native HAWQ tables.
 
 The configuration parameter `pxf_enable_stat_collection` controls collection of advanced statistics. If `pxf_enable_stat_collection` is set to false, no analysis is performed on PXF tables. An additional parameter, `pxf_stat_max_fragments`, controls the number of fragments sampled to build a sample table. By default `pxf_stat_max_fragments` is set to 100, which means that even if there are more than 100 fragments, only this number of fragments will be used in `ANALYZE` to sample the data. Increasing this number will result in better sampling, but can also impact performance.
 
-When a PXF table is analyzed and `pxf_enable_stat_collection` is set to off, or an error occurs because the table is not defined correctly, the PXF service is down, or `getFragmentsStats` is not implemented, a warning message is shown and no statistics are gathered for that table. If `ANALYZE` is running over all tables in the database, the next table will be processed \u2013 a failure processing one table does not stop the command.
+When a PXF table is analyzed, any of the following conditions might result in a warning message with no statistics gathered for the table:
+
+- `pxf_enable_stat_collection` is set to off,
+- an error occurs because the table is not defined correctly,
+- the PXF service is down, or
+- `getFragmentsStats()` is not implemented 
 
-For a�detailed explanation�about HAWQ statistical data gathering, see `ANALYZE` in the SQL Commands Reference.
+If `ANALYZE` is running over all tables in the database, the next table will be processed \u2013 a failure processing one table does not stop the command.
+
+For a�detailed explanation�about HAWQ statistical data gathering, refer to the [`ANALYZE`](../reference/sql/ANALYZE.html) SQL command reference.
 
 **Note:**
 
--   Depending on external table size, the time required to complete an ANALYZE operation can be lengthy. The boolean parameter `pxf_enable_stat_collection` enables statistics collection for PXF. The default value is `on`. Turning this parameter off (disabling PXF statistics collection) can help decrease the time needed for the ANALYZE operation.
--   You can also use *pxf\_stat\_max\_fragments* to limit the number of fragments to be sampled by decreasing it from the default (100). However, if the number is too low, the sample might not be uniform and the statistics might be skewed.
--   You can also implement getFragmentsStats to return an error. This will cause ANALYZE on a table with this Fragmenter to fail immediately, and default statistics values will be used for that table.
+-   Depending on external table size, the time required to complete an `ANALYZE` operation can be lengthy. The boolean parameter `pxf_enable_stat_collection` enables statistics collection for PXF. The default value is `on`. Turning this parameter off (disabling PXF statistics collection) can help decrease the time needed for the `ANALYZE` operation.
+-   You can also use `pxf_stat_max_fragments` to limit the number of fragments to be sampled by decreasing it from the default (100). However, if the number is too low, the sample might not be uniform and the statistics might be skewed.
+-   You can also implement `getFragmentsStats()` to return an error. This will cause `ANALYZE` on a table with this `Fragmenter` to fail immediately, and default statistics values will be used for that table.
 
-The following table lists the Fragmenter plug-in implementations included with the PXF API.
+The following table lists the `Fragmenter` plug-in implementations included with the PXF API.
 
 <a id="fragmenter__table_cgs_svp_3s"></a>
 
@@ -262,31 +286,31 @@ The following table lists the Fragmenter plug-in implementations included with t
 </colgroup>
 <thead>
 <tr class="header">
-<th><p><code class="ph codeph">Fragmenter class</code></p></th>
-<th><p><code class="ph codeph">Description</code></p></th>
+<th><p>Fragmenter class</p></th>
+<th><p>Description</p></th>
 </tr>
 </thead>
 <tbody>
 <tr class="odd">
 <td>org.apache.hawq.pxf.plugins.hdfs.HdfsDataFragmenter</td>
-<td>Fragmenter for Hdfs files</td>
+<td>Fragmenter for HDFS, JSON files</td>
 </tr>
 <tr class="even">
-<td>org.apache.hawq.pxf.plugins.hbase.HBaseAtomicDataAccessor</td>
+<td>org.apache.hawq.pxf.plugins.hbase.HBaseDataFragmenter</td>
 <td>Fragmenter for HBase tables</td>
 </tr>
 <tr class="odd">
-<td>org.apache.hawq.pxf.plugins.hive.HiveDataFragmenter</td>
+<td>org.apache.hawq.pxf.plugins.hive.HiveDataFragmenter</li>
 <td>Fragmenter for Hive tables�</td>
 </tr>
 <tr class="even">
 <td>org.apache.hawq.pxf.plugins.hdfs.HiveInputFormatFragmenter</td>
-<td>Fragmenter for Hive tables with RC or text files�</td>
+<td>Fragmenter for Hive tables with RC, ORC, or text file formats�</td>
 </tr>
 </tbody>
 </table>
 
-A�Fragmenter class extends `org.apache.hawq.pxf.api.Fragmenter`:
+A�`Fragmenter` class extends `org.apache.hawq.pxf.api.Fragmenter`:
 
 #### <a id="com.pivotal.pxf.api.fragmenter"></a>org.apache.hawq.pxf.api.Fragmenter
 
@@ -339,7 +363,7 @@ public abstract class Fragmenter extends Plugin {
 
 #### <a id="classdescription"></a>Class Description
 
-The Fragmenter.getFragments()�method returns a�List&lt;Fragment&gt;;:
+The `Fragmenter.getFragments()`�method returns a�`List<Fragment>`:
 
 ``` java
 package org.apache.hawq.pxf.api;
@@ -416,7 +440,7 @@ public class FragmentsStats {
 
 ### <a id="accessor"></a>Accessor
 
-The Accessor retrieves specific fragments and passes records back to the Resolver.�For example, the HDFS plug-ins create a `org.apache.hadoop.mapred.FileInputFormat` and a `org.apache.hadoop.mapred.RecordReader` for an HDFS file and sends this to the Resolver.�In the case of HBase or Hive files, the Accessor returns single rows from an HBase or Hive table. PXF 1.x or higher contains the following Accessor implementations:
+The `Accessor` retrieves specific fragments and passes records back to the Resolver.�For example, the HDFS plug-ins create a `org.apache.hadoop.mapred.FileInputFormat` and a `org.apache.hadoop.mapred.RecordReader` for an HDFS file and sends this to the `Resolver`.�In the case of HBase or Hive files, the `Accessor` returns single rows from an HBase or Hive table. PXF includes the following `Accessor` implementations:
 
 <a id="accessor__table_ewm_ttz_4p"></a>
 
@@ -428,8 +452,8 @@ The Accessor retrieves specific fragments and passes records back to the Resolve
 </colgroup>
 <thead>
 <tr class="header">
-<th><p><code class="ph codeph">Accessor class</code></p></th>
-<th><p><code class="ph codeph">Description</code></p></th>
+<th><p>Accessor class</p></th>
+<th><p>Description</p></th>
 </tr>
 </thead>
 <tbody>
@@ -467,16 +491,26 @@ The Accessor retrieves specific fragments and passes records back to the Resolve
 </tr>
 <tr class="odd">
 <td>org.apache.hawq.pxf.plugins.hive.HiveLineBreakAccessor</td>
-<td>Accessor for Hive tables with text files</td>
+<td>Accessor for Hive tables stored as text file format</td>
 </tr>
 <tr class="even">
 <td>org.apache.hawq.pxf.plugins.hive.HiveRCFileAccessor</td>
-<td>Accessor for Hive tables with RC files</td>
+<td>Accessor for Hive tables stored as RC file format</td>
+</tr>
+</tr>
+<tr class="odd">
+<td>org.apache.hawq.pxf.plugins.hive.HiveORCAccessor</td>
+<td>Accessor for Hive tables stored as ORC format </td>
+</tr>
+</tr>
+<tr class="odd">
+<td>org.apache.hawq.pxf.plugins.json.JsonAccessor</td>
+<td>Accessor for JSON files</td>
 </tr>
 </tbody>
 </table>
 
-The class must extend the `org.apache.hawq.pxf.Plugin`� class, and�implement one or both interfaces:
+The class must extend the `org.apache.hawq.pxf.Plugin`� class, and�implement one or both of the interfaces:
 
 -   `org.apache.hawq.pxf.api.ReadAccessor`
 -   `org.apache.hawq.pxf.api.WriteAccessor`
@@ -511,12 +545,12 @@ public interface WriteAccessor {
 }
 ```
 
-The Accessor calls `openForRead()` to read existing data. After reading the data, it calls `closeForRead()`. `readNextObject()` returns one of the�following:
+The `Accessor` calls `openForRead()` to read existing data. After reading the data, it calls `closeForRead()`. `readNextObject()` returns one of the�following:
 
--   a single record, encapsulated in a OneRow object
+-   a single record, encapsulated in a `OneRow` object
 -   null if it reaches `EOF`
 
-The Accessor calls `openForWrite()` to write data out. After writing the data, it�writes a `OneRow` object with `writeNextObject()`, and when done calls `closeForWrite()`. `OneRow` represents a key-value item.
+The `Accessor` calls `openForWrite()` to write data out. After writing the data, it�writes a `OneRow` object with `writeNextObject()`, and when done calls `closeForWrite()`. `OneRow` represents a key-value item.
 
 #### <a id="com.pivotal.pxf.api.onerow"></a>org.apache.hawq.pxf.api.OneRow
 
@@ -568,7 +602,7 @@ public class OneRow {
 
 ### <a id="resolver"></a>Resolver
 
-The Resolver deserializes records in the `OneRow` format and serializes them to a list of `OneField` objects. PXF converts a `OneField` object to a HAWQ-readable�`GPDBWritable` format.�PXF 1.x or higher contains the following implementations:
+The `Resolver` deserializes records in the `OneRow` format and serializes them to a list of `OneField` objects. PXF converts a `OneField` object to a HAWQ-readable�`GPDBWritable` format.�PXF 1.x or higher contains the following implementations:
 
 <a id="resolver__table_nbd_d5z_4p"></a>
 
@@ -580,18 +614,18 @@ The Resolver deserializes records in the `OneRow` format and serializes them to
 </colgroup>
 <thead>
 <tr class="header">
-<th><p><code class="ph codeph">Resolver class</code></p></th>
-<th><p><code class="ph codeph">Description</code></p></th>
+<th><p>Resolver class</p></th>
+<th><p>Description</p></th>
 </tr>
 </thead>
 <tbody>
 <tr class="odd">
-<td><p><code class="ph codeph">org.apache.hawq.pxf.plugins.hdfs.StringPassResolver</code></p></td>
+<td><p>org.apache.hawq.pxf.plugins.hdfs.StringPassResolver</p></td>
 <td><p><code class="ph codeph">StringPassResolver</code> replaced the deprecated <code class="ph codeph">TextResolver</code>. It passes whole records (composed of any data types) as strings without parsing them</p></td>
 </tr>
 <tr class="even">
-<td><p><code class="ph codeph">org.apache.hawq.pxf.plugins.hdfs.WritableResolver</code></p></td>
-<td><p>Resolver for custom Hadoop Writable implementations. Custom class can be specified with the schema in DATA-SCHEMA. Supports the following types:</p>
+<td><p>org.apache.hawq.pxf.plugins.hdfs.WritableResolver</p></td>
+<td><p>Resolver for custom Hadoop Writable implementations. Custom class can be specified with the schema in `DATA-SCHEMA`. Supports the following types:</p>
 <pre class="pre codeblock"><code>DataType.BOOLEAN
 DataType.INTEGER
 DataType.BIGINT
@@ -601,11 +635,11 @@ DataType.VARCHAR
 DataType.BYTEA</code></pre></td>
 </tr>
 <tr class="odd">
-<td><p><code class="ph codeph">org.apache.hawq.pxf.plugins.hdfs.AvroResolver</code></p></td>
+<td><p>org.apache.hawq.pxf.plugins.hdfs.AvroResolver</p></td>
 <td><p>Supports the same field objects as <code class="ph codeph">WritableResolver</code>.�</p></td>
 </tr>
 <tr class="even">
-<td><p><code class="ph codeph">org.apache.hawq.pxf.plugins.hbase.HBaseResolver</code></p></td>
+<td><p>org.apache.hawq.pxf.plugins.hbase.HBaseResolver</p></td>
 <td><p>Supports the same field objects as <code class="ph codeph">WritableResolver</code> and also supports the following:</p>
 <pre class="pre codeblock"><code>DataType.SMALLINT
 DataType.NUMERIC
@@ -614,20 +648,24 @@ DataType.BPCHAR
 DataType.TIMESTAMP</code></pre></td>
 </tr>
 <tr class="odd">
-<td><p><code class="ph codeph">org.apache.hawq.pxf.plugins.hive.HiveResolver</code></p></td>
+<td><p>org.apache.hawq.pxf.plugins.hive.HiveResolver</p></td>
 <td><p>Supports the same field objects as <code class="ph codeph">WritableResolver</code> and also supports the following:</p>
 <pre class="pre codeblock"><code>DataType.SMALLINT
 DataType.TEXT
 DataType.TIMESTAMP</code></pre></td>
 </tr>
 <tr class="even">
-<td><p><code class="ph codeph">org.apache.hawq.pxf.plugins.hive.HiveStringPassResolver</code></p></td>
+<td><p>org.apache.hawq.pxf.plugins.hive.HiveStringPassResolver</p></td>
 <td>Specialized <code class="ph codeph">HiveResolver</code> for a Hive table stored as Text files. Should be used together with <code class="ph codeph">HiveInputFormatFragmenter</code>/<code class="ph codeph">HiveLineBreakAccessor</code>.</td>
 </tr>
 <tr class="odd">
-<td><code class="ph codeph">org.apache.hawq.pxf.plugins.hive.HiveColumnarSerdeResolver</code></td>
+<td>org.apache.hawq.pxf.plugins.hive.HiveColumnarSerdeResolver</td>
 <td>Specialized <code class="ph codeph">HiveResolver</code> for a Hive table stored as RC file. Should be used together with <code class="ph codeph">HiveInputFormatFragmenter</code>/<code class="ph codeph">HiveRCFileAccessor</code>.</td>
 </tr>
+<tr class="odd">
+<td>org.apache.hawq.pxf.plugins.hive.HiveORCSerdeResolver</td>
+<td>Specialized <code class="ph codeph">HiveResolver</code> for a Hive table stored in ORC format. Should be used together with <code class="ph codeph">HiveInputFormatFragmenter</code>/<code class="ph codeph">HiveORCAccessor</code>.</td>
+</tr>
 </tbody>
 </table>
 
@@ -663,8 +701,8 @@ public interface WriteResolver {
 
 **Note:**
 
--   getFields should return a List&lt;OneField&gt;, each OneField representing a single field.
--   `setFields�`should return a single�`OneRow�`object, given a List&lt;OneField&gt;.
+-   `getFields()` should return a `List<OneField>`, with each `OneField` representing a single field.
+-   `setFields()` should return a single�`OneRow` object, given a `List<OneField>`.
 
 #### <a id="com.pivotal.pxf.api.onefield"></a>org.apache.hawq.pxf.api.OneField
 
@@ -687,7 +725,7 @@ public class OneField {
 }
 ```
 
-The value of `type` should follow the org.apache.hawq.pxf.api.io.DataType�`enums`. `val` is the appropriate Java class. Supported types are as follows:
+The value of `type` should follow the `org.apache.hawq.pxf.api.io.DataType`�`enums`. `val` is the appropriate Java class. Supported types are:
 
 <a id="com.pivotal.pxf.api.onefield__table_f4x_35z_4p"></a>
 
@@ -759,19 +797,13 @@ The value of `type` should follow the org.apache.hawq.pxf.api.io.DataType�`enum
 </tbody>
 </table>
 
-### <a id="analyzer"></a>Analyzer
-
-The Analyzer has been deprecated. A new function in the Fragmenter API (Fragmenter.getFragmentsStats) is used to gather initial statistics for the data source, and provides PXF statistical data for the HAWQ query optimizer. For a�detailed explanation�about HAWQ statistical data gathering, see `ANALYZE` in the SQL Command Reference.
-
-Using the Analyzer API will result in an error message. Use the Fragmenter and getFragmentsStats to gather advanced statistics.
-
 ## <a id="aboutcustomprofiles"></a>About Custom Profiles
 
-Administrators can add new profiles or edit the built-in profiles in�`/etc/conf/pxf-profiles.xml` file. See [Using Profiles to Read and Write Data](ReadWritePXF.html#readingandwritingdatawithpxf) for information on how to add custom profiles.
+Administrators can add new profiles or edit the built-in profiles in�`/etc/pxf/conf/pxf-profiles.xml`. See [Using Profiles to Read and Write Data](ReadWritePXF.html#readingandwritingdatawithpxf) for information on how to add custom profiles.
 
 ## <a id="aboutqueryfilterpush-down"></a>About Query Filter Push-Down
 
-If a query includes a number of WHERE clause filters, �HAWQ may push all or some queries to PXF. If pushed to PXF, the Accessor can use the filtering information when accessing the data source to fetch tuples. These filters�only return records that pass filter evaluation conditions.�This reduces data processing�and�reduces network traffic from the SQL engine.
+If a query includes a number of `WHERE` clause filters, �HAWQ may push all or some queries to PXF. If pushed to PXF, the `Accessor` can use the filtering information when accessing the data source to fetch tuples. These filters�only return records that pass filter evaluation conditions.�This reduces data processing�and�reduces network traffic from the SQL engine.
 
 This topic includes the following information:
 
@@ -788,17 +820,17 @@ PXF�allows push-down filtering if the following rules are met:
 -   Uses only�single expressions or a group of AND'ed expressions - no OR'ed expressions.
 -   Uses only expressions of supported data types and operators.
 
-FilterParser�scans the pushed down filter list and uses the user's build() implementation to build the filter.
+`FilterParser`�scans the pushed down filter list and uses the user's `build()` implementation to build the filter.
 
--   For simple expressions (e.g, a &gt;= 5), FilterParser places column objects on the left of the�expression and constants on the right.
--   For compound expressions (e.g &lt;expression&gt; AND &lt;expression&gt;) it handles three cases in the build() function:
+-   For simple expressions (e.g, a &gt;= 5), `FilterParser` places column objects on the left of the�expression and constants on the right.
+-   For compound expressions (e.g &lt;expression&gt; AND &lt;expression&gt;) it handles three cases in the `build()` function:
     1.  Simple Expression: &lt;Column Index&gt; &lt;Operation&gt; &lt;Constant&gt;
     2.  Compound Expression: &lt;Filter Object&gt; AND &lt;Filter Object&gt;
     3.  Compound Expression: &lt;List of Filter Objects&gt; AND &lt;Filter Object&gt;
 
 ### <a id="creatingafilterbuilderclass"></a>Creating a Filter Builder Class
 
-To check�if a filter queried PXF, call the `InputData                   hasFilter()` function:
+To check�if a filter queried PXF, call the `InputData.hasFilter()` function:
 
 ``` java
 /*
@@ -810,7 +842,7 @@ public boolean hasFilter()
 }
 ```
 
-If `hasFilter()` returns `false`, there is no filter information. If it returns `true`,�PXF parses the serialized filter string into a meaningful filter object to use later. To do so, create a filter builder�class that implements the�`FilterParser.FilterBuilder�` interface:
+If `hasFilter()` returns `false`, there is no filter information. If it returns `true`,�PXF parses the serialized filter string into a meaningful filter object to use later. To do so, create a filter builder�class that implements the�`FilterParser.FilterBuilder` interface:
 
 ``` java
 package org.apache.hawq.pxf.api;
@@ -827,7 +859,7 @@ interface FilterBuilder {
 }
 ```
 
-While PXF parses the serialized filter string from the incoming HAWQ query, it calls the `build() interface` function. PXF�calls this function for each condition or filter pushed down to PXF. Implementing this function returns some Filter object or representation that the Fragmenter, Accessor, or Resolver uses in runtime to filter out records. The `build()` function accepts an Operation as input, and�left and right operands.
+While PXF parses the serialized filter string from the incoming HAWQ query, it calls the `build()` function. PXF�calls this function for each condition or filter pushed down to PXF. Implementing this function returns some Filter object or representation that the `Fragmenter`, `Accessor`, or `Resolver` uses in runtime to filter out records. The `build()` function accepts an Operation as input, and�left and right operands.
 
 ### <a id="filteroperations"></a>Filter Operations
 
@@ -843,8 +875,20 @@ public enum Operation
     HDOP_GE, //greater than or equal
     HDOP_EQ, //equal
     HDOP_NE, //not equal
-    HDOP_AND //AND'ed conditions
+    HDOP_LIKE,
+    HDOP_IS_NULL,
+    HDOP_IS_NOT_NULL,
+    HDOP_IN
 };
+
+/**
+ * Logical operators
+ */
+public enum LogicalOperation {
+    HDOP_AND,
+    HDOP_OR,
+    HDOP_NOT
+}
 ```
 
 #### <a id="filteroperands"></a>Filter Operands
@@ -885,7 +929,7 @@ public class Constant
 
 #### <a id="filterobject"></a>Filter Object
 
-Filter Objects can be internal, such as those you define; or external, those that the remote system uses. For example, for HBase, you define the HBase�`Filter` class (`org.apache.hadoop.hbase.filter.Filter`), while�for Hive, you use an internal default representation created by the PXF framework, called�`BasicFilter`. You can decide�the filter object to use, including writing a new one. `BasicFilter` is the most common:
+Filter Objects can be internal - such as those you define - or external, those that the remote system uses. For example, for HBase you define the HBase�`Filter` class (`org.apache.hadoop.hbase.filter.Filter`), while�for Hive you use an internal default representation created by the PXF framework, called�`BasicFilter`. You can choose�the filter object to use, including writing a new one. `BasicFilter` is the most common:
 
 ``` java
 /*
@@ -919,7 +963,7 @@ static public class BasicFilter
 
 ### <a id="sampleimplementation"></a>Sample Implementation
 
-Let's look at the following sample implementation of the filter builder class and its `build()` function that handles all 3 cases. Let's assume that BasicFilter was used to hold our filter operations.
+Let's look at the following sample implementation of the filter builder class and its `build()` function that handles all 3 cases. Let's assume that `BasicFilter` was used to hold our filter operations.
 
 ``` java
 import java.util.LinkedList;
@@ -1003,7 +1047,7 @@ public class MyDemoFilterBuilder implements FilterParser.FilterBuilder
 }
 ```
 
-Here is an example of creating a filter-builder class to implement the Filter interface, implement the `build()` function, and generate�the Filter object. To do this, use either the Accessor, Resolver, or both to�call the `getFilterObject` function:
+Here is an example of creating a filter-builder class to implement the Filter interface, implement the `build()` function, and generate�the Filter object. To do this, use either the `Accessor`, `Resolver`, or both to�call the `getFilterObject()` function:
 
 ``` java
 if (inputData.hasFilter())
@@ -1034,7 +1078,7 @@ else
 }
 ```
 
-Example of evaluating a single filter:
+Example showing evaluation of a single filter:
 
 ``` java
 //Get our BasicFilter Object
@@ -1134,7 +1178,7 @@ FORMAT 'custom' (formatter='pxfwritable_export');
 
 ### <a id="pluginexamples"></a>Plug-in Examples
 
-This section contains sample dummy implementations of all three plug-ins. It also contains a usage example.
+This section contains sample dummy implementations of all three plug-ins. It also includes a usage example.
 
 #### <a id="dummyfragmenter"></a>Dummy Fragmenter
 

http://git-wip-us.apache.org/repos/asf/incubator-hawq-docs/blob/dcfe1a47/markdown/pxf/ReadWritePXF.html.md.erb
----------------------------------------------------------------------
diff --git a/markdown/pxf/ReadWritePXF.html.md.erb b/markdown/pxf/ReadWritePXF.html.md.erb
index f424afd..5c29ae8 100644
--- a/markdown/pxf/ReadWritePXF.html.md.erb
+++ b/markdown/pxf/ReadWritePXF.html.md.erb
@@ -34,7 +34,7 @@ PXF comes with a number of built-in profiles that group together�a collection o
 -   HBase (Read only)
 -   JSON (Read only)
 
-You can specify a built-in profile when you want to read data that exists inside HDFS files, Hive tables, HBase tables, and JSON files and for writing data into HDFS files.
+You can specify a built-in profile when you want to read data that exists inside HDFS files, Hive tables, HBase tables, or JSON files, and when you want to write data into HDFS files.
 
 <table>
 <colgroup>
@@ -46,7 +46,7 @@ You can specify a built-in profile when you want to read data that exists inside
 <tr class="header">
 <th>Profile</th>
 <th>Description</th>
-<th>Fragmenter/Accessor/Resolver</th>
+<th>Fragmenter/Accessor/Resolver/Metadata/OutputFormat</th>
 </tr>
 </thead>
 <tbody>
@@ -75,6 +75,8 @@ You can specify a built-in profile when you want to read data that exists inside
 <li>org.apache.hawq.pxf.plugins.hive.HiveDataFragmenter</li>
 <li>org.apache.hawq.pxf.plugins.hive.HiveAccessor</li>
 <li>org.apache.hawq.pxf.plugins.hive.HiveResolver</li>
+<li>org.apache.hawq.pxf.plugins.hive.HiveMetadataFetcher</li>
+<li>org.apache.hawq.pxf.service.io.GPDBWritable</li>
 </ul></td>
 </tr>
 <tr class="even">
@@ -87,6 +89,20 @@ Note: The <code class="ph codeph">DELIMITER</code> parameter is mandatory.
 <li>org.apache.hawq.pxf.plugins.hive.HiveInputFormatFragmenter</li>
 <li>org.apache.hawq.pxf.plugins.hive.HiveRCFileAccessor</li>
 <li>org.apache.hawq.pxf.plugins.hive.HiveColumnarSerdeResolver</li>
+<li>org.apache.hawq.pxf.plugins.hive.HiveMetadataFetcher</li>
+<li>org.apache.hawq.pxf.service.io.Text</li>
+</ul></td>
+</tr>
+<tr class="odd">
+<td>HiveORC</td>
+<td>Optimized read of a Hive table where each partition is stored as an ORC file.
+</td>
+<td><ul>
+<li>org.apache.hawq.pxf.plugins.hive.HiveInputFormatFragmenter</li>
+<li>org.apache.hawq.pxf.plugins.hive.HiveORCAccessor</li>
+<li>org.apache.hawq.pxf.plugins.hive.HiveORCSerdeResolver</li>
+<li>org.apache.hawq.pxf.plugins.hive.HiveMetadataFetcher</li>
+<li>org.apache.hawq.pxf.service.io.GPDBWritable</li>
 </ul></td>
 </tr>
 <tr class="odd">
@@ -99,6 +115,8 @@ Note: The <code class="ph codeph">DELIMITER</code> parameter is mandatory.
 <li>org.apache.hawq.pxf.plugins.hive.HiveInputFormatFragmenter</li>
 <li>org.apache.hawq.pxf.plugins.hive.HiveLineBreakAccessor</li>
 <li>org.apache.hawq.pxf.plugins.hive.HiveStringPassResolver</li>
+<li>org.apache.hawq.pxf.plugins.hive.HiveMetadataFetcher</li>
+<li>org.apache.hawq.pxf.service.io.Text</li>
 </ul></td>
 </tr>
 <tr class="even">
@@ -131,6 +149,8 @@ Note: The <code class="ph codeph">DELIMITER</code> parameter is mandatory.
 </tbody>
 </table>
 
+**Notes**: Metadata identifies the Java class that provides field definitions in the relation. OutputFormat identifies the output serialization format (text or binary) for which a specific profile is optimized. While the built-in `Hive*` profiles provide Metadata and OutputFormat classes, other profiles may have no need to implement or specify these classes.
+
 ## <a id="addingandupdatingprofiles"></a>Adding and Updating Profiles
 
 Each profile has a mandatory unique�name and an optional�description. In addition, each profile contains a set of plug-ins�that�are an�extensible set of metadata attributes.  Administrators can add new profiles or edit the built-in profiles defined in�`/etc/pxf/conf/pxf-profiles.xml`. 

http://git-wip-us.apache.org/repos/asf/incubator-hawq-docs/blob/dcfe1a47/markdown/pxf/TroubleshootingPXF.html.md.erb
----------------------------------------------------------------------
diff --git a/markdown/pxf/TroubleshootingPXF.html.md.erb b/markdown/pxf/TroubleshootingPXF.html.md.erb
index c1bd43a..57fe9d5 100644
--- a/markdown/pxf/TroubleshootingPXF.html.md.erb
+++ b/markdown/pxf/TroubleshootingPXF.html.md.erb
@@ -195,6 +195,8 @@ Examine/collect the log messages from `pxf-service.log`.
 
 ### <a id="pxfdblogmsg"></a>Database-Level Logging
 
+Database-level logging may provide insight into internal PXF service operations. Additionally, when you access Hive tables using `hcatalog` or the `Hive*` profiles, log messages identify the underlying `Hive*` profile(s) employed to access the data.
+
 Enable HAWQ and PXF debug message logging during operations on PXF external tables by setting the `client_min_messages` server configuration parameter to `DEBUG2` in your `psql` session.
 
 ``` shell
@@ -211,6 +213,8 @@ DEBUG2:  churl http header: cell #21: X-GP-DATA-DIR: pxf_hive1
 DEBUG2:  churl http header: cell #22: X-GP-profile: Hive
 DEBUG2:  churl http header: cell #23: X-GP-URI: pxf://namenode:51200/pxf_hive1?profile=Hive
 ...
+DEBUG2:  pxf: set_current_fragment_headers: using profile: Hive
+...
 ```
 
 Examine/collect the log messages from `stdout`.