http://git-wip-us.apache.org/repos/asf/incubator-hawq-docs/blob/7514e193/pxf/HBasePXF.html.md.erb
----------------------------------------------------------------------
diff --git a/pxf/HBasePXF.html.md.erb b/pxf/HBasePXF.html.md.erb
new file mode 100644
index 0000000..8b89730
--- /dev/null
+++ b/pxf/HBasePXF.html.md.erb
@@ -0,0 +1,105 @@
+---
+title: Accessing HBase Data
+---
+
+## <a id="installingthepxfhbaseplugin"></a>Prerequisites
+
+Before trying to access HBase data with PXF, verify the following:
+
+-   The `/etc/hbase/conf/hbase-env.sh` configuration file must reference the 
`pxf-hbase.jar`. For example, `/etc/hbase/conf/hbase-env.sh` should include the 
line:
+
+    ``` bash
+    export HBASE_CLASSPATH=${HBASE_CLASSPATH}:/usr/lib/pxf/pxf-hbase.jar
+    ```
+
+    **Note:** You must restart HBase after making any changes to the HBase 
configuration.
+
+-   PXF HBase plug-in is installed on all cluster nodes.
+-   HBase and ZooKeeper jars are installed on all cluster nodes.
+
+## <a id="syntax3"></a>Syntax
+
+To create an external HBase table, use the following syntax:
+
+``` sql
+CREATE [READABLE|WRITABLE] EXTERNAL TABLE table_name 
+    ( column_name data_type [, ...] | LIKE other_table )
+LOCATION ('pxf://namenode[:port]/hbase-table-name?Profile=HBase')
+FORMAT 'CUSTOM' (Formatter='pxfwritable_import');
+```
+
+The HBase profile is equivalent to the following PXF parameters:
+
+-   Fragmenter=org.apache.hawq.pxf.plugins.hbase.HBaseDataFragmenter
+-   Accessor=org.apache.hawq.pxf.plugins.hbase.HBaseAccessor
+-   Resolver=org.apache.hawq.pxf.plugins.hbase.HBaseResolver
+
+## <a id="columnmapping"></a>Column Mapping
+
+Most HAWQ external tables (PXF or others) require that the HAWQ table 
attributes match the source data record layout, and include all the available 
attributes. With HAWQ, however, you use the PXF HBase plug-in to specify the 
subset of HBase qualifiers that define the HAWQ PXF table. To set up a clear 
mapping between each attribute in the PXF table and a specific qualifier in the 
HBase table, you can use either direct mapping or indirect mapping. In 
addition, the HBase row key is handled in a special way.
+
+### <a id="rowkey"></a>Row Key
+
+You can use the HBase table row key in several ways. For example, you can see 
them using query results, or you can run a WHERE clause filter on a range of 
row key values. To use the row key in the HAWQ query, define the HAWQ table 
with the reserved PXF attribute `recordkey.` This attribute name tells PXF to 
return the record key in any key-value based system and in HBase.
+
+**Note:** Because HBase is byte and not character-based, you should define the 
recordkey as type bytea. This may result in better ability to filter data and 
increase performance.
+
+``` sql
+CREATE EXTERNAL TABLE <tname> (recordkey bytea, ... ) LOCATION ('pxf:// ...')
+```
+
+### <a id="directmapping"></a>Direct Mapping
+
+Use Direct Mapping to map HAWQ table attributes to HBase qualifiers. You can 
specify the HBase qualifier names of interest, with column family names 
included, as quoted values. 
+
+For example, you have defined an HBase table called `hbase_sales` with 
multiple column families and many qualifiers. To create a HAWQ table with these 
attributes:
+
+-   `rowkey`
+-   qualifier `saleid` in the column family `cf1`
+-   qualifier `comments` in the column family `cf8` 
+
+use the following `CREATE EXTERNAL TABLE` syntax:
+
+``` sql
+CREATE EXTERNAL TABLE hbase_sales (
+  recordkey bytea,
+  "cf1:saleid" int,
+  "cf8:comments" varchar
+) ...
+```
+
+The PXF HBase plug-in uses these attribute names as-is and returns the values 
of these HBase qualifiers.
+
+### <a id="indirectmappingvialookuptable"></a>Indirect Mapping (via Lookup 
Table)
+
+The direct mapping method is fast and intuitive, but using indirect mapping 
helps to reconcile HBase qualifier names with HAWQ behavior:
+
+-   HBase qualifier names may be longer than 32 characters. HAWQ has a 
32-character limit on attribute name size.
+-   HBase qualifier names can be binary or non-printable. HAWQ attribute 
names are character based.
+
+In either case, Indirect Mapping uses a lookup table on HBase. You can create 
the lookup table to store all necessary lookup information. This works as a 
template for any future queries. The name of the lookup table must be 
`pxflookup` and must include the column family named `mapping`.
+
+Using the sales example in Direct Mapping, if our `rowkey` represents the 
HBase table name and the `mapping` column family includes the actual attribute 
mapping in the key value form of`<hawq attr name>=<hbase                        
     cf:qualifier>`.
+
+#### <a id="example5"></a>Example
+
+This example maps the `saleid` qualifier in the `cf1` column family to the 
HAWQ `id` column and the `comments` qualifier in the `cf8` family to the HAWQ 
`cmts` column.
+
+| (row key) | mapping           |
+|-----------|-------------------|
+| sales     | id=cf1:saleid     |
+| sales     | cmts=cf8:comments |
+
+The mapping assigned new names for each qualifier. You can use these names in 
your HAWQ table definition:
+
+``` sql
+CREATE EXTERNAL TABLE hbase_sales (
+  recordkey bytea
+  id int,
+  cmts varchar
+) ...
+```
+
+PXF automatically matches HAWQ to HBase column names when a `pxflookup` table 
exists in HBase.
+
+

http://git-wip-us.apache.org/repos/asf/incubator-hawq-docs/blob/7514e193/pxf/HDFSFileDataPXF.html.md.erb
----------------------------------------------------------------------
diff --git a/pxf/HDFSFileDataPXF.html.md.erb b/pxf/HDFSFileDataPXF.html.md.erb
new file mode 100644
index 0000000..99c27ba
--- /dev/null
+++ b/pxf/HDFSFileDataPXF.html.md.erb
@@ -0,0 +1,507 @@
+---
+title: Accessing HDFS File Data
+---
+
+## <a id="installingthepxfhdfsplugin"></a>Prerequisites
+
+Before working with HDFS file data using HAWQ and PXF, you should perform the 
following operations:
+
+-   Test PXF on HDFS before connecting to Hive or HBase.
+-   Ensure that all HDFS users have read permissions to HDFS services and 
that write permissions have been limited to specific users.
+
+## <a id="syntax1"></a>Syntax
+
+The syntax for creating an external HDFS file 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...]')
+      FORMAT '[TEXT | CSV | CUSTOM]' (<formatting_properties>);
+```
+
+where `<pxf parameters>` is:
+
+``` pre
+   FRAGMENTER=fragmenter_class&ACCESSOR=accessor_class&RESOLVER=resolver_class]
+ | PROFILE=profile-name
+```
+
+**Note:** Omit the `FRAGMENTER` parameter for `READABLE` external tables.
+
+Use an SQL `SELECT` statement to read from an HDFS READABLE table:
+
+``` sql
+SELECT ... FROM table_name;
+```
+
+Use an SQL `INSERT` statement to add data to an HDFS WRITABLE table:
+
+``` sql
+INSERT INTO table_name ...;
+```
+
+To read the data in the files or to write based on the existing format, use 
`FORMAT`, `PROFILE`, or one of the classes.
+
+This topic describes the following:
+
+-   FORMAT clause
+-   Profile
+-   Accessor
+-   Resolver
+-   Avro
+
+**Note:** For more details about the API and classes, see [PXF External Tables 
and API](PXFExternalTableandAPIReference.html#pxfexternaltableandapireference).
+
+### <a id="formatclause"></a>FORMAT clause
+
+Use one of the following formats to read data with any PXF connector:
+
+-   `FORMAT 'TEXT'`: Use with plain delimited text files on HDFS.
+-   `FORMAT 'CSV'`: Use with comma-separated value files on HDFS.
+-   `FORMAT 'CUSTOM'`: Use with all other files, including Avro format and 
binary formats. Must always be used with the built-in formatter 
'`pxfwritable_import`' (for read) or '`pxfwritable_export`' (for write).
+
+**Note:** When creating PXF external tables, you cannot use the `HEADER` 
option in your `FORMAT` specification.
+
+### <a id="topic_ab2_sxy_bv"></a>Profile
+
+For plain or comma-separated text files in HDFS use either the 
`HdfsTextSimple` or `HdfsTextMulti` Profile, or the classname 
org.apache.hawq.pxf.plugins.hdfs.*HdfsDataFragmenter*. Use the `Avro` profile 
for Avro files. See [Using Profiles to Read and Write 
Data](ReadWritePXF.html#readingandwritingdatawithpxf) for more information.
+
+**Note:** For read tables, you must include a Profile or a Fragmenter in the 
table definition.
+
+### <a id="accessor"></a>Accessor
+
+The choice of an Accessor depends on the HDFS data file type. 
+
+**Note:** You must include either a Profile or an Accessor in the table 
definition.
+
+<table>
+<colgroup>
+<col width="25%" />
+<col width="25%" />
+<col width="25%" />
+<col width="25%" />
+</colgroup>
+<thead>
+<tr class="header">
+<th>File Type</th>
+<th>Accessor</th>
+<th>FORMAT clause</th>
+<th>Comments</th>
+</tr>
+</thead>
+<tbody>
+<tr class="odd">
+<td>Plain Text delimited</td>
+<td>org.apache.hawq.pxf.plugins. hdfs.LineBreakAccessor</td>
+<td>FORMAT 'TEXT' (<em>format param list</em>)</td>
+<td> Read + Write
+<p>You cannot use the <code class="ph codeph">HEADER</code> option.</p></td>
+</tr>
+<tr class="even">
+<td>Plain Text CSV </td>
+<td>org.apache.hawq.pxf.plugins. hdfs.LineBreakAccessor</td>
+<td>FORMAT 'CSV' (<em>format param list</em>) </td>
+<td><p>LineBreakAccessor is parallel and faster.</p>
+<p>Use if each logical data row is a physical data line.</p>
+<p>Read + Write </p>
+<p>You cannot use the <code class="ph codeph">HEADER</code> option.</p></td>
+</tr>
+<tr class="odd">
+<td>Plain Text CSV </td>
+<td>org.apache.hawq.pxf.plugins. hdfs.QuotedLineBreakAccessor</td>
+<td>FORMAT 'CSV' (<em>format param list</em>) </td>
+<td><p>QuotedLineBreakAccessor is slower and non-parallel.</p>
+<p>Use if the data includes embedded (quoted) linefeed characters.</p>
+<p>Read Only </p>
+<p>You cannot use the <code class="ph codeph">HEADER</code> option.</p></td>
+</tr>
+<tr class="even">
+<td>SequenceFile</td>
+<td>org.apache.hawq.pxf.plugins. hdfs.SequenceFileAccessor</td>
+<td>FORMAT 'CUSTOM' (formatter='pxfwritable_import')</td>
+<td> Read + Write (use formatter='pxfwritable_export' for write)</td>
+</tr>
+<tr class="odd">
+<td>AvroFile</td>
+<td>org.apache.hawq.pxf.plugins. hdfs.AvroFileAccessor</td>
+<td>FORMAT 'CUSTOM' (formatter='pxfwritable_import')</td>
+<td> Read Only</td>
+</tr>
+</tbody>
+</table>
+
+### <a id="resolver"></a>Resolver
+
+Choose the Resolver format if data records are serialized in the HDFS file. 
+
+**Note:** You must include a Profile or a Resolver in the table definition.
+
+<table>
+<colgroup>
+<col width="33%" />
+<col width="33%" />
+<col width="33%" />
+</colgroup>
+<thead>
+<tr class="header">
+<th>Record Serialization</th>
+<th>Resolver</th>
+<th>Comments</th>
+</tr>
+</thead>
+<tbody>
+<tr class="odd">
+<td>Avro</td>
+<td>org.apache.hawq.pxf.plugins. hdfs.AvroResolver</td>
+<td><ul>
+<li>Avro files include the record schema, Avro serialization can be used in 
other file types (e.g, Sequence File). </li>
+<li>For Avro serialized records outside of an Avro file, include a schema file 
name (.avsc) in the url under the optional <code class="ph 
codeph">Schema-Data </code>option.</li>
+<li>Deserialize Only (Read) .</li>
+</ul></td>
+</tr>
+<tr class="even">
+<td>Java Writable</td>
+<td>org.apache.hawq.pxf.plugins. hdfs.WritableResolver</td>
+<td><ul>
+<li>Include the name of the Java class that uses Writable serialization in 
the URL under the optional <code class="ph codeph">Schema-Data.</code></li>
+<li>The class file must exist in the public stage directory (or in Hadoop's 
class path).</li>
+<li>Deserialize and Serialize (Read + Write). </li>
+<li>See <a href="#customizedwritableschemafileguidelines">Customized Writable 
Schema File Guidelines</a>.</li>
+</ul></td>
+</tr>
+<tr class="odd">
+<td>None (plain text)</td>
+<td>org.apache.hawq.pxf.plugins. hdfs.StringPassResolver</td>
+<td><ul>
+<li>Does not serialize plain text records. The database parses plain records. 
Passes records as they are.</li>
+<li>Deserialize and Serialize (Read + Write).</li>
+</ul></td>
+</tr>
+</tbody>
+</table>
+
+#### <a id="customizedwritableschemafileguidelines"></a>Schema File Guidelines 
for WritableResolver
+
+When using a WritableResolver, a schema file needs to be defined. The file 
needs to be a Java class file and must be on the class path of PXF.
+
+The class file must follow the following requirements:
+
+1.  Must implement org.apache.hadoop.io.Writable interface.
+2.  WritableResolver uses reflection to recreate the schema and populate its 
fields (for both read and write). Then it uses the Writable interface functions 
to read/write. Therefore, fields must be public, to enable access to them. 
Private fields will be ignored.
+3.  Fields are accessed and populated in the order in which they are declared 
in the class file.
+4.  Supported field types:
+    -   boolean
+    -   byte array
+    -   double
+    -   float
+    -   int
+    -   long
+    -   short
+    -   string
+
+    Arrays of any of the above types are supported, but the constructor must 
define the array size so the reflection will work.
+
+### <a id="additionaloptions"></a>Additional Options
+
+<a id="additionaloptions__table_skq_kpz_4p"></a>
+
+<table>
+<caption><span class="tablecap">Table 1. Additional PXF 
Options</span></caption>
+<colgroup>
+<col width="50%" />
+<col width="50%" />
+</colgroup>
+<thead>
+<tr class="header">
+<th>Option Name</th>
+<th>Description</th>
+</tr>
+</thead>
+<tbody>
+<tr class="odd">
+<td>COLLECTION_DELIM</td>
+<td>(Avro or Hive profiles only.) The delimiter character(s) to place between 
entries in a top-level array, map, or record field when PXF maps a Hive or Avro 
complex data type to a text column. The default is a &quot;,&quot; 
character.</td>
+</tr>
+<tr class="even">
+<td>COMPRESSION_CODEC</td>
+<td><ul>
+<li>Useful for WRITABLE PXF tables.</li>
+<li>Specifies the compression codec class name for compressing the written 
data. The class must implement the 
org.apache.hadoop.io.compress.CompressionCodec interface.</li>
+<li> Some valid values are org.apache.hadoop.io.compress.DefaultCodec 
org.apache.hadoop.io.compress.GzipCodec 
org.apache.hadoop.io.compress.BZip2Codec.</li>
+<li>Note: org.apache.hadoop.io.compress.BZip2Codec runs in a single thread and 
can be slow.</li>
+<li>This option has no default value. </li>
+<li>When the option is not defined, no compression will be done.</li>
+</ul></td>
+</tr>
+<tr class="odd">
+<td>COMPRESSION_TYPE</td>
+<td><ul>
+<li>Useful WRITABLE PXF tables with SequenceFileAccessor.</li>
+<li>Ignored when COMPRESSION_CODEC is not defined.</li>
+<li>Specifies the compression type for sequence file.</li>
+<li>Valid options are: 
+<ul>
+<li>RECORD - only the value part of each row is compressed.</li>
+<li>BLOCK - both keys and values are collected in 'blocks' separately and 
compressed.</li>
+</ul></li>
+<li>Default value: RECORD.</li>
+</ul></td>
+</tr>
+<tr class="even">
+<td>MAPKEY_DELIM</td>
+<td>(Avro or Hive profiles only.) The delimiter character(s) to place between 
the key and value of a map entry when PXF maps a Hive or Avro complex data type 
to a text colum. The default is a &quot;:&quot; character.</td>
+</tr>
+<tr class="odd">
+<td>RECORDKEY_DELIM</td>
+<td>(Avro profile only.) The delimiter character(s) to place between the field 
name and value of a record entry when PXF maps an Avro complex data type to a 
text colum. The default is a &quot;:&quot; character.</td>
+</tr>
+<tr class="even">
+<td>SCHEMA-DATA</td>
+<td>The data schema file used to create and read the HDFS file. For example, 
you could create an avsc (for Avro), or a Java class (for Writable 
Serialization) file. Make sure that you have added any JAR files containing the 
schema to <code class="ph codeph">pxf-public.classpath</code>.
+<p>This option has no default value.</p></td>
+</tr>
+<tr class="odd">
+<td>THREAD-SAFE</td>
+<td>Determines if the table query can run in multithread mode or not. When set 
to FALSE, requests will be handled in a single thread.
+<p>Should be set when a plug-in or other elements that are not thread safe are 
used (e.g. compression codec).</p>
+<p>Allowed values: TRUE, FALSE. Default value is TRUE - requests can run in 
multithread mode.</p></td>
+</tr>
+<tr class="even">
+<td> &lt;custom&gt;</td>
+<td>Any option added to the pxf URI string will be accepted and passed, along 
with its value, to the Fragmenter, Accessor, and Resolver implementations.</td>
+</tr>
+</tbody>
+</table>
+
+## <a id="accessingdataonahighavailabilityhdfscluster"></a>Accessing Data on a 
High Availability HDFS Cluster
+
+To access data on a High Availability HDFS cluster, change the authority in 
the URI in the LOCATION. Use *HA\_nameservice* instead of 
*name\_node\_host:51200*.
+
+``` sql
+CREATE [READABLE|WRITABLE] EXTERNAL TABLE <tbl name> (<attr list>)
+LOCATION ('pxf://<HA nameservice>/<path to file or 
directory>?Profile=profile[&<additional options>=<value>]')
+FORMAT '[TEXT | CSV | CUSTOM]' (<formatting properties>);
+```
+
+The opposite is true when a highly available HDFS cluster is reverted to a 
single namenode configuration. In that case, any table definition that has the 
nameservice specified should use the &lt;NN host&gt;:&lt;NN rest port&gt; 
syntax. 
+
+## <a id="recordkeyinkey-valuefileformats"></a>Using a Record Key with 
Key-Value File Formats
+
+For sequence file and other file formats that store rows in a key-value 
format, the key value can be accessed through HAWQ by using the saved keyword 
'`recordkey`' as a field name.
+
+The field type must correspond to the key type, much as the other fields must 
match the HDFS data. 
+
+WritableResolver supports read and write of recordkey, which can be of the 
following Writable Hadoop types:
+
+-   BooleanWritable
+-   ByteWritable
+-   DoubleWritable
+-   FloatWritable
+-   IntWritable
+-   LongWritable
+-   Text
+
+If the `recordkey` field is not defined, the key is ignored in read, and a 
default value (segment id as LongWritable) is written in write.
+
+### <a id="example1"></a>Example
+
+A data schema `Babies.class` contains three fields: (name text, birthday text, 
weight float). An external table must include these three fields, and can 
either include or ignore the recordkey.
+
+``` sql
+-- writable table with recordkey
+CREATE WRITABLE EXTERNAL TABLE babies_registry (recordkey int, name text, 
birthday text, weight float)
+  LOCATION ('pxf://namenode_host:51200/babies_1940s'
+            '?ACCESSOR=org.apache.hawq.pxf.plugins.hdfs.SequenceFileAccessor'
+            '&RESOLVER=org.apache.hawq.pxf.plugins.hdfs.WritableResolver'
+            '&DATA-SCHEMA=Babies')
+  FORMAT 'CUSTOM' (formatter='pxfwritable_export');
+INSERT INTO babies_registry VALUES (123456, "James Paul McCartney", "June 18, 
1942", 3.800);
+
+-- writable table without recordkey
+CREATE WRITABLE EXTERNAL TABLE babies_registry2 (name text, birthday text, 
weight float)
+  LOCATION ('pxf://namenode_host:51200/babies_1940s'
+            '?ACCESSOR=org.apache.hawq.pxf.plugins.SequenceFileAccessor'
+            '&RESOLVER=org.apache.hawq.pxf.plugins.WritableResolver'
+            '&DATA-SCHEMA=Babies')
+  FORMAT 'CUSTOM' (formatter='pxfwritable_export');
+
+-- this record's key will have some default value
+INSERT INTO babies_registry VALUES ("Richard Starkey", "July 7, 1940", 4.0);
+```
+
+The same goes for reading data from an existing file with a key-value format, 
e.g. a Sequence file.
+
+``` sql
+-- readable table with recordkey
+CREATE EXTERNAL TABLE babies_1940 (recordkey int, name text, birthday text, 
weight float)
+  LOCATION ('pxf://namenode_host:51200/babies_1940s'
+            '?FRAGMENTER=org.apache.hawq.pxf.plugins.hdfs.HdfsDataFragmenter'
+            '&ACCESSOR=org.apache.hawq.pxf.plugins.hdfs.SequenceFileAccessor'
+            '&RESOLVER=org.apache.hawq.pxf.plugins.hdfs.WritableResolver'
+            '&DATA-SCHEMA=Babies')
+  FORMAT 'CUSTOM' (formatter='pxfwritable_import');
+-- retrieve each record's key
+SELECT * FROM babies_1940;
+
+-- readable table without recordkey
+CREATE EXTERNAL TABLE babies_1940_2 (name text, birthday text, weight float)
+  LOCATION ('pxf://namenode_host:51200/babies_1940s'
+            '?FRAGMENTER=org.apache.hawq.pxf.plugins.hdfs.HdfsDataFragmenter'
+            '&ACCESSOR=org.apache.hawq.pxf.plugins.hdfs.SequenceFileAccessor'
+            '&RESOLVER=org.apache.hawq.pxf.plugins.hdfs.WritableResolver'
+            '&DATA-SCHEMA=Babies') 
+  FORMAT 'CUSTOM' (formatter='pxfwritable_import');
+-- ignores the records' key
+SELECT * FROM babies_1940_2; 
+```
+
+## <a id="topic_oy3_qwm_ss"></a>Working with Avro Files
+
+Avro files combine their data with a schema, and can support complex data 
types such as arrays, maps, records, enumerations, and fixed types. When you 
create a PXF external table to represent Avro data, map top-level fields in the 
schema that use a primitive data type to HAWQ columns of the same type. Map 
top-level fields that use a complex data type to a TEXT column in the external 
table. The PXF Avro profile automatically separates components of a complex 
type by inserting delimiters in the text column. You can then use functions or 
application code to further process components of the complex data.
+
+The following table summarizes external table mapping rules for Avro data.
+
+<caption><span class="tablecap">Table 2. Avro Data Type 
Mapping</span></caption>
+
+<a id="topic_oy3_qwm_ss__table_j4s_h1n_ss"></a>
+
+| Avro Data Type                                                    | PXF Type 
                                                                                
                                                                                
                           |
+|-------------------------------------------------------------------|-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
+| Primitive type (int, double, float, long, string, bytes, boolean) | 
Corresponding HAWQ data type. See [Data 
Types](../reference/HAWQDataTypes.html). |
+| Complex type: Array, Map, Record, or Enum                         | TEXT, 
with default delimiters inserted between collection items, mapped key-value 
pairs, and record data.                                                         
                                  |
+| Complex type: Fixed                                               | BYTEA    
                                                                                
                                                                                
                           |
+| Union                                                             | Follows 
the above conventions for primitive or complex data types, depending on the 
union. Null values are supported in Unions.                                     
                                |
+
+For complex types, the PXF Avro profile inserts default delimiters between 
collection items and values. You can use non-default delimiter characters by 
including the `COLLECTION_DELIM`, `MAPKEY_DELIM`, and/or `RECORDKEY_DELIM` 
optional parameters on the Avro profile. See [Additional PXF 
Options](#additionaloptions__table_skq_kpz_4p) for a description of the 
parameters.
+
+### <a id="topic_tr3_dpg_ts"></a>Example
+
+The following example uses the Avro schema shown in [Sample Avro 
Schema](#topic_tr3_dpg_ts__section_m2p_ztg_ts) and the associated data file 
shown in [Sample Avro Data (JSON)](#topic_tr3_dpg_ts__section_spk_15g_ts).
+
+#### <a id="topic_tr3_dpg_ts__section_m2p_ztg_ts"></a>Sample Avro Schema
+
+``` json
+{
+  "type" : "record",
+  "name" : "example_schema",
+  "namespace" : "com.example",
+  "fields" : [ {
+    "name" : "id",
+    "type" : "long",
+    "doc" : "Id of the user account"
+  }, {
+    "name" : "username",
+    "type" : "string",
+    "doc" : "Name of the user account"
+  }, {
+    "name" : "followers",
+    "type" : {"type": "array", "items": "string"},
+    "doc" : "Users followers"
+  }, {
+    "name": "rank",
+    "type": ["null", "int"],
+    "default": null
+  }, {
+    "name": "fmap",
+    "type": {"type": "map", "values": "long"}
+  }, {
+    "name": "address",
+    "type": {
+        "type": "record",
+        "name": "addressRecord",
+        "fields": [
+            {"name":"number", "type":"int"},
+            {"name":"street", "type":"string"},
+            {"name":"city", "type":"string"}]
+    }
+  }, {
+   "name": "relationship",
+    "type": {
+        "type": "enum",
+        "name": "relationshipEnum",
+        "symbols": ["MARRIED","LOVE","FRIEND","COLLEAGUE","STRANGER","ENEMY"]
+    }
+  }, {
+    "name" : "md5",
+    "type": {
+        "type" : "fixed",
+        "name" : "md5Fixed",
+        "size" : 4
+    }
+  } ],
+  "doc:" : "A basic schema for storing messages"
+}
+```
+
+#### <a id="topic_tr3_dpg_ts__section_spk_15g_ts"></a>Sample Avro Data (JSON)
+
+``` pre
+{"id":1, "username":"john","followers":["kate", "santosh"], "rank":null, 
"relationship": "FRIEND", "fmap": {"kate":10,"santosh":4},
+"address":{"street":"renaissance drive", "number":1,"city":"san jose"}, 
"md5":\u3F00\u007A\u0073\u0074}
+
+{"id":2, "username":"jim","followers":["john", "pam"], "rank":3, 
"relationship": "COLLEAGUE", "fmap": {"john":3,"pam":3}, 
+"address":{"street":"deer creek", "number":9,"city":"palo alto"}, 
"md5":\u0010\u0021\u0003\u0004}
+```
+
+To map this Avro file to an external table, the top-level primitive fields 
("id" of type long and "username" of type string) are mapped to their 
equivalent HAWQ types (bigint and text). The remaining complex fields are 
mapped to text columns:
+
+``` sql
+gpadmin=# CREATE EXTERNAL TABLE avro_complex 
+  (id bigint, 
+  username text, 
+  followers text, 
+  rank int, 
+  fmap text, 
+  address text, 
+  relationship text,
+  md5 bytea) 
+LOCATION ('pxf://namehost:51200/tmp/avro_complex?PROFILE=Avro')
+FORMAT 'CUSTOM' (FORMATTER='pxfwritable_import');
+```
+
+The above command uses default delimiters for separating components of the 
complex types. This command is equivalent to the one above, but it explicitly 
sets the delimiters using the Avro profile parameters:
+
+``` sql
+gpadmin=# CREATE EXTERNAL TABLE avro_complex 
+  (id bigint, 
+  username text, 
+  followers text, 
+  rank int, 
+  fmap text, 
+  address text, 
+  relationship text,
+  md5 bytea) 
+LOCATION 
('pxf://localhost:51200/tmp/avro_complex?PROFILE=Avro&COLLECTION_DELIM=,&MAPKEY_DELIM=:&RECORDKEY_DELIM=:')
+FORMAT 'CUSTOM' (FORMATTER='pxfwritable_import');
+```
+
+A simple query of the external table shows the components of the complex type 
data separated with delimiters:
+
+``` sql
+gpadmin=# select * from avro_complex;
+```
+
+``` pre
+id | username |  followers  |    rank     |  fmap   |    address  |  
relationship  |  md5
+1| john | [kate,santosh] |   | {kate:10,santosh:4} | 
{number:1,street:renaissance drive,city:san jose} | FRIEND | ?zst
+2| jim | [john,pam] | 3 | {john:3,pam:3} | {number:9,street:deer 
creek,city:palo alto} | COLLEAGUE | \020!\003\004
+```
+
+You can process the delimited components in the text columns as necessary for 
your application. For example, the following command uses the `string_to_array` 
function to convert entries in the "followers" field to a text array column in 
a new view. The view is then queried to filter rows based on whether a 
particular follower appears in the array:
+
+``` sql
+gpadmin=# create view followers_view as 
+  select username, address, string_to_array(substring(followers from 2 for 
(char_length(followers) - 2)), ',')::text[] 
+    as followers 
+  from avro_complex;
+
+gpadmin=# select username, address 
+from followers_view 
+where john = ANY(followers);
+```
+
+``` pre
+username | address
+jim | {number:9,street:deer creek,city:palo alto}
+```

http://git-wip-us.apache.org/repos/asf/incubator-hawq-docs/blob/7514e193/pxf/HawqExtensionFrameworkPXF.html.md.erb
----------------------------------------------------------------------
diff --git a/pxf/HawqExtensionFrameworkPXF.html.md.erb 
b/pxf/HawqExtensionFrameworkPXF.html.md.erb
new file mode 100644
index 0000000..b705984
--- /dev/null
+++ b/pxf/HawqExtensionFrameworkPXF.html.md.erb
@@ -0,0 +1,41 @@
+---
+title: Working with PXF and External Data
+---
+
+HAWQ Extension Framework (PXF) is an extensible framework that allows HAWQ to 
query external system data. 
+
+PXF includes built-in connectors for accessing data inside HDFS files, Hive 
tables, and HBase tables. PXF also integrates with HCatalog to query Hive 
tables directly.
+
+PXF allows users to create custom connectors to access other parallel data 
stores or processing engines. To create these connectors using Java plug-ins, 
see the [PXF External Tables and API](PXFExternalTableandAPIReference.html).
+
+-   **[Installing PXF Plug-ins](../pxf/InstallPXFPlugins.html)**
+
+    This topic describes how to install the built-in PXF service plug-ins that 
are required to connect PXF to HDFS, Hive, and HBase. You should install the 
appropriate RPMs on each node in your cluster.
+
+-   **[Configuring PXF](../pxf/ConfigurePXF.html)**
+
+    This topic describes how to configure the PXF service.
+
+-   **[Accessing HDFS File Data](../pxf/HDFSFileDataPXF.html)**
+
+    This topic describes how to access HDFS file data using PXF.
+
+-   **[Accessing Hive Data](../pxf/HivePXF.html)**
+
+    This topic describes how to access Hive data using PXF. You have several 
options for querying data stored in Hive. You can create external tables in PXF 
and then query those tables, or you can easily query Hive tables by using HAWQ 
and PXF's integration with HCatalog. HAWQ accesses Hive table metadata stored 
in HCatalog.
+
+-   **[Accessing HBase Data](../pxf/HBasePXF.html)**
+
+    This topic describes how to access HBase data using PXF.
+
+-   **[Using Profiles to Read and Write Data](../pxf/ReadWritePXF.html)**
+
+    PXF profiles are collections of common metadata attributes that can be 
used to simplify the reading and writing of data. You can use any of the 
built-in profiles that come with PXF or you can create your own.
+
+-   **[PXF External Tables and 
API](../pxf/PXFExternalTableandAPIReference.html)**
+
+    You can use the PXF API to create your own connectors to access any other 
type of parallel data store or processing engine.
+
+-   **[Troubleshooting PXF](../pxf/TroubleshootingPXF.html)**
+
+

http://git-wip-us.apache.org/repos/asf/incubator-hawq-docs/blob/7514e193/pxf/HivePXF.html.md.erb
----------------------------------------------------------------------
diff --git a/pxf/HivePXF.html.md.erb b/pxf/HivePXF.html.md.erb
new file mode 100644
index 0000000..db3e53c
--- /dev/null
+++ b/pxf/HivePXF.html.md.erb
@@ -0,0 +1,417 @@
+---
+title: Accessing Hive Data
+---
+
+This topic describes how to access Hive data using PXF. You have several 
options for querying data stored in Hive. You can create external tables in PXF 
and then query those tables, or you can easily query Hive tables by using HAWQ 
and PXF's integration with HCatalog. HAWQ accesses Hive table metadata stored 
in HCatalog.
+
+## <a id="installingthepxfhiveplugin"></a>Prerequisites
+
+Check the following before using PXF to access Hive:
+
+-   The PXF HDFS plug-in is installed on all cluster nodes.
+-   The PXF Hive plug-in is installed on all cluster nodes.
+-   The Hive JAR files and conf directory are installed on all cluster nodes.
+-   Test PXF on HDFS before connecting to Hive or HBase.
+-   You are running the Hive Metastore service on a machine in your cluster. 
+-   You have set the `hive.metastore.uris` property in the `hive-site.xml` 
on the NameNode.
+
+## <a id="hivecommandline"></a>Hive Command Line
+
+To start the Hive command line and work directly on a Hive table:
+
+``` shell
+$ hive
+```
+
+Here is an example of how to create and load data into a sample Hive table 
from an existing file.
+
+``` sql
+hive> CREATE TABLE test (name string, type string, supplier_key int, 
full_price double) row format delimited fields terminated by ',';
+hive> LOAD DATA local inpath '/local/path/data.txt' into table test;
+```
+
+## <a id="topic_p2s_lvl_25"></a>Using PXF Tables to Query Hive
+
+Hive tables are defined in a specific way in PXF, regardless of the underlying 
file storage format. The PXF Hive plug-ins automatically detect source tables 
in the following formats:
+
+-   Text based
+-   SequenceFile
+-   RCFile
+-   ORCFile
+-   Parquet
+-   Avro
+
+The source table can also be a combination of these types. The PXF Hive 
plug-in uses this information to query the data in runtime.
+
+-   **[Syntax Example](../pxf/HivePXF.html#syntax2)**
+
+-   **[Hive Complex Types](../pxf/HivePXF.html#topic_b4v_g3n_25)**
+
+### <a id="syntax2"></a>Syntax Example
+
+The following PXF table definition is valid for any Hive file storage type.
+
+``` sql
+CREATE [READABLE|WRITABLE] EXTERNAL TABLE table_name 
+    ( column_name data_type [, ...] | LIKE other_table )
+LOCATION ('pxf://namenode[:port]/hive-db-name.hive-table-name?<pxf 
parameters>[&custom-option=value...]')FORMAT 'CUSTOM' 
(formatter='pxfwritable_import')
+```
+
+where `<pxf parameters>` is:
+
+``` pre
+   FRAGMENTER=fragmenter_class&ACCESSOR=accessor_class&RESOLVER=resolver_class]
+ | PROFILE=profile-name
+```
+
+
+If `hive-db-name` is omitted, pxf will default to the Hive `default` database.
+
+**Note:** The port is the 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.
+
+PXF has three built-in profiles for Hive tables:
+
+-   Hive
+-   HiveRC
+-   HiveText
+
+The Hive profile works with any Hive storage type. 
+The following example creates a readable HAWQ external table representing a 
Hive table named `accessories` in the `inventory` Hive database using the PXF 
Hive profile:
+
+``` shell
+$ psql -d postgres
+```
+
+``` sql
+postgres=# CREATE EXTERNAL TABLE hivetest(id int, newid int)
+LOCATION ('pxf://namenode:51200/inventory.accessories?PROFILE=Hive')
+FORMAT 'custom' (formatter='pxfwritable_import');
+```
+
+
+Use HiveRC and HiveText to query RC and Text formats respectively. The HiveRC 
and HiveText profiles are faster than the generic Hive profile. When using the 
HiveRC and HiveText profiles, you must specify a DELIMITER option in the 
LOCATION clause. See [Using Profiles to Read and Write 
Data](ReadWritePXF.html#readingandwritingdatawithpxf) for more information on 
profiles.
+
+
+### <a id="topic_b4v_g3n_25"></a>Hive Complex Types
+
+PXF tables support Hive data types that are not primitive types. The supported 
Hive complex data types are array, struct, map, and union. This Hive `CREATE 
TABLE` statement, for example, creates a table with each of these complex data 
types:
+
+``` sql
+hive> CREATE TABLE sales_collections (
+  item STRING,
+  price FLOAT,
+  properties ARRAY<STRING>,
+  hash MAP<STRING,FLOAT>,
+  delivery_address STRUCT<street:STRING, city:STRING, state:STRING, zip:INT>,
+  others UNIONTYPE<FLOAT, BOOLEAN, STRING>
+)  
+ROW FORMAT DELIMITED FIELDS
+TERMINATED BY '\001' COLLECTION ITEMS TERMINATED BY '\002' MAP KEYS TERMINATED 
BY '\003' LINES TERMINATED BY '\n' STORED AS TEXTFILE;
+hive> LOAD DATA LOCAL INPATH '/local/path/<some data file>' INTO TABLE 
sales_collection;
+```
+
+You can use HAWQ functions or application code to extract the components of 
the complex data columns as needed.
+
+## <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.
+
+Previously, in order to query Hive tables using HAWQ and PXF, you needed to 
create an external table in PXF that described the target table's Hive 
metadata. Since HAWQ is now integrated with HCatalog, HAWQ can use metadata 
stored in HCatalog instead of external tables created for PXF. 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.
+
+-   **[Usage](../pxf/HivePXF.html#topic_j1l_y55_c5)**
+
+-   **[Limitations](../pxf/HivePXF.html#topic_r5k_pst_25)**
+
+### <a id="topic_j1l_y55_c5"></a>Usage
+
+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 a 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";
+    ``` 
+    
+3.  To query a Hive table with HCatalog integration, simply 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;
+    ```
+
+4.  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. For example, from the `psql` client interface:
+
+        ``` shell
+        $ psql -d postgres
+        postgres=# \d hcatalog.default.test
+
+        PXF Hive Table "default.test"
+            Column    |  Type  
+        --------------+--------
+         name         | text
+         type         | text
+         supplier_key | int4
+         full_price   | float8 
+        ```
+    -   Use `\d hcatalog.hive-db-name.*` to describe the whole database 
schema. For example:
+
+        ``` shell
+        postgres=# \d hcatalog.default.*
+
+        PXF Hive Table "default.test"
+            Column    |  Type  
+        --------------+--------
+         type         | text
+         name         | text
+         supplier_key | int4
+         full_price   | float8
+
+        PXF Hive Table "default.testabc"
+         Column | Type 
+        --------+------
+         type   | text
+         name   | text
+        ```
+    -   Use `\d hcatalog.*.*` to describe the whole schema:
+
+        ``` shell
+        postgres=# \d hcatalog.*.*
+
+        PXF Hive Table "default.test"
+            Column    |  Type  
+        --------------+--------
+         type         | text
+         name         | text
+         supplier_key | int4
+         full_price   | float8
+
+        PXF Hive Table "default.testabc"
+         Column | Type 
+        --------+------
+         type   | text
+         name   | text
+
+        PXF Hive Table "userdb.test"
+          Column  | Type 
+        ----------+------
+         address  | text
+         username | text
+         
+        ```
+
+    **Note:** 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.
+
+5.  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:** Currently the only supported input profile is `'Hive'`.
+
+    For example, 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.test');
+    ```
+    
+    ``` pre
+      path   | itemname |  fieldname   | fieldtype 
+    ---------+----------+--------------+-----------
+     default | test     | name         | text
+     default | test     | type         | text
+     default | test     | supplier_key | int4
+     default | test     | full_price   | float8
+    (4 rows)
+    ```
+
+    The following statement returns table descriptions from the default 
database.
+
+    ``` sql
+    postgres=# select * from pxf_get_item_fields('Hive','default.*');
+    ```
+    
+    ``` pre
+      path   | itemname |  fieldname   | fieldtype 
+    ---------+----------+--------------+-----------
+     default | test     | name         | text
+     default | test     | type         | text
+     default | test     | supplier_key | int4
+     default | test     | full_price   | float8
+     default | testabc  | name         | text
+     default | testabc  | type         | text
+    (6 rows)
+    ```
+
+    The following statement returns a description of the entire schema.
+
+    ``` sql
+    postgres=# select * from pxf_get_item_fields('Hive', '*.*');
+    ```
+    
+    ``` pre
+      path   | itemname |  fieldname   | fieldtype 
+    ---------+----------+--------------+-----------
+     default | test     | name         | text
+     default | test     | type         | text
+     default | test     | supplier_key | int4
+     default | test     | full_price   | float8
+     default | testabc  | name         | text
+     default | testabc  | type         | text
+     userdb  | test     | username     | text
+     userdb  | test     | address      | text
+    (8 rows)
+    ```
+
+### <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 instead. (See [Hive Complex Types](#topic_b4v_g3n_25) for 
example.)
+-   Even for primitive types, HCatalog metadata descriptions produced by `\d` 
and` \d+` are converted to HAWQ types. For example, the Hive type `tinyint` is 
converted to HAWQ type `int2`. (See [Data 
Types](../reference/HAWQDataTypes.html) for a list of data types in HAWQ.)
+-   HAWQ reserves the database name "hcatalog" for system use. You cannot 
connect to or alter the system "hcatalog" database.
+
+## <a id="partitionfiltering"></a>Partition Filtering
+
+The PXF Hive plug-in uses the Hive partitioning feature and directory 
structure. This enables partition exclusion on HDFS files that contain the 
Hive table. To use the partition filtering feature to reduce network traffic 
and I/O, run a PXF query using a WHERE clause that refers to a specific 
partition in the partitioned Hive table.
+
+To take advantage of PXF partition filtering push-down, name the partition 
fields in the external table. These names must be the same as the names stored 
in the Hive table. Otherwise, PXF ignores Partition filtering and the filtering 
is performed on the HAWQ side, impacting performance.
+
+**Note:** The Hive plug-in only filters on partition columns, not on other 
table attributes.
+
+### <a id="example2"></a>Example
+
+Create a Hive table `sales_part` with two partition columns, 
`delivery_state` and `delivery_city:`
+
+``` sql
+hive> CREATE TABLE sales_part (name STRING, type STRING, supplier_key INT, 
price DOUBLE)
+PARTITIONED BY (delivery_state STRING, delivery_city STRING)
+ROW FORMAT DELIMITED FIELDS TERMINATED BY ',';
+```
+
+Load data into this Hive table and add some partitions:
+
+``` sql
+hive> LOAD DATA LOCAL INPATH '/local/path/data1.txt' INTO TABLE sales_part 
PARTITION(delivery_state = 'CALIFORNIA', delivery_city = 'San Francisco');
+hive> LOAD DATA LOCAL INPATH '/local/path/data2.txt' INTO TABLE sales_part 
PARTITION(delivery_state = 'CALIFORNIA', delivery_city = 'Sacramento');
+hive> LOAD DATA LOCAL INPATH '/local/path/data3.txt' INTO TABLE sales_part 
PARTITION(delivery_state = 'NEVADA'    , delivery_city = 'Reno');
+hive> LOAD DATA LOCAL INPATH '/local/path/data4.txt' INTO TABLE sales_part 
PARTITION(delivery_state = 'NEVADA'    , delivery_city = 'Las Vegas');
+```
+
+The Hive storage directory should appears as follows:
+
+``` pre
+/hive/warehouse/sales_part/delivery_state=CALIFORNIA/delivery_city=’San 
Francisco’/data1.txt
+/hive/warehouse/sales_part/delivery_state=CALIFORNIA/delivery_city=Sacramento/data2.txt
+/hive/warehouse/sales_part/delivery_state=NEVADA/delivery_city=Reno/data3.txt
+/hive/warehouse/sales_part/delivery_state=NEVADA/delivery_city=’Las 
Vegas’/data4.txt
+```
+
+To define a HAWQ PXF table that will read this Hive table and take advantage 
of partition filter push-down, define the fields corresponding to the Hive 
partition fields at the end of the attribute list. In HiveQL, a `select      
                   *` statement on a partitioned table shows the partition 
fields at the end of the record.
+
+``` sql
+postgres=# CREATE EXTERNAL TABLE pxf_sales_part(
+  item_name TEXT, 
+  item_type TEXT, 
+  supplier_key INTEGER, 
+  item_price DOUBLE PRECISION, 
+  delivery_state TEXT, 
+  delivery_city TEXT
+)
+LOCATION ('pxf://namenode_host:51200/sales_part?Profile=Hive')
+FORMAT 'custom' (FORMATTER='pxfwritable_import');
+
+postgres=# SELECT * FROM pxf_sales_part;
+```
+
+### <a id="example3"></a>Example
+
+In the following example, the HAWQ query filters the `delivery_city` partition 
`Sacramento`. The filter on  `item_name` is not pushed down, since it is not a 
partition column. It is performed on the HAWQ side after all the data on 
`Sacramento` is transferred for processing.
+
+``` sql
+postgres=# SELECT * FROM pxf_sales_part WHERE delivery_city = 'Sacramento' AND 
item_name = 'shirt';
+```
+
+### <a id="example4"></a>Example
+
+The following HAWQ query reads all the data under `delivery_state` partition 
`CALIFORNIA`, regardless of the city.
+
+``` sql
+postgres=# SELECT * FROM pxf_sales_part WHERE delivery_state = 'CALIFORNIA';
+```
+
+## <a id="topic_fdm_zrh_1s"></a>Using PXF with Hive Default Partitions
+
+This topic describes a difference in query results between Hive and PXF 
queries when Hive tables use a default partition. When dynamic partitioning is 
enabled in Hive, a partitioned table may store data in a default partition. 
Hive creates a default partition when the value of a partitioning column does 
not match the defined type of the column (for example, when a NULL value is 
used for any partitioning column). In Hive, any query that includes a filter on 
a partition column *excludes* any data that is stored in the table's default 
partition.
+
+Similar to Hive, PXF represents a table's partitioning columns as columns that 
are appended to the end of the table. However, PXF translates any column value 
in a default partition to a NULL value. This means that a HAWQ query that 
includes an IS NULL filter on a partitioning column can return different 
results than the same Hive query.
+
+### <a id="topic_g4r_4wh_1s"></a>Example
+
+Consider a Hive partitioned table that is created with the statement:
+
+``` sql
+hive> CREATE TABLE sales (order_id bigint, order_amount float) PARTITIONED BY 
(date date);
+```
+
+The table is loaded with five rows that contain the following data:
+
+``` pre
+1    1.0    1900-01-01
+2    2.2    1994-04-14
+3    3.3    2011-03-31
+4    4.5    NULL
+5    5.0    2013-12-06
+```
+
+In this case, the insertion of row 4 creates a Hive default partition, because 
the partition column "date" contains a null value.
+
+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 date 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.
+
+Keep this behavior in mind when executing IS NULL queries on Hive partitioned 
tables.
+
+## <a id="topic_dbb_nz3_ts"></a>Accessing Hive Tables in Parquet Format
+
+The PXF Hive profile supports both non-partitioned and partitioned Hive tables 
that use the Parquet storage format in HDFS. Simply map the table columns using 
equivalent HAWQ data types. For example, in Hive if a table is created using:
+
+``` sql
+hive> create table hive_parquet_table (fname string, lname string, custid int, 
acctbalance double) stored as
+      parquet;
+```
+
+Then you would define the HAWQ external table using:
+
+``` sql
+postgres=# create external table pxf_parquet_table (fname text, lname text, 
custid int, acctbalance double precision) 
+    location 
('pxf://localhost:51200/hive-db-name.hive_parquet_table?profile=Hive') 
+    format 'custom' (formatter='pxfwritable_import'); 
+```
+
+

http://git-wip-us.apache.org/repos/asf/incubator-hawq-docs/blob/7514e193/pxf/InstallPXFPlugins.html.md.erb
----------------------------------------------------------------------
diff --git a/pxf/InstallPXFPlugins.html.md.erb 
b/pxf/InstallPXFPlugins.html.md.erb
new file mode 100644
index 0000000..8d2dfe4
--- /dev/null
+++ b/pxf/InstallPXFPlugins.html.md.erb
@@ -0,0 +1,141 @@
+---
+title: Installing PXF Plug-ins
+---
+
+This topic describes how to install the built-in PXF service plug-ins that are 
required to connect PXF to HDFS, Hive, and HBase. 
+
+**Note:** The PXF plug-ins require that you run Tomcat on the host machine. 
Tomcat reserves ports 8005, 8080, and 8009. If you have configured Oozie JXM 
reporting on a host that will run a PXF plug-in, make sure that the reporting 
service uses a port other than 8005. This helps to prevent port conflict errors 
from occurring when you start the PXF service.
+
+## <a id="directories_and_logs"></a>PXF Installation and Log File Directories
+
+Installing PXF plug-ins, regardless of method, creates directories and log 
files on each node receiving the plug-in installation:
+
+| Directory                      | Description                                 
                                                                                
                                                                                
                               |
+|--------------------------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
+| `/usr/lib/pxf`                 | PXF library location                        
                                                                                
                                                                                
                               |
+| `/etc/pxf/conf`                | PXF configuration directory. This directory 
contains the `pxf-public.classpath` and `pxf-private.classpath` configuration 
files. See [Setting up the Java 
Classpath](ConfigurePXF.html#settingupthejavaclasspath). |
+| `/var/pxf/pxf-service`         | PXF service instance location               
                                                                                
                                                                                
                               |
+| `/var/log/pxf` | This directory includes `pxf-service.log` and all 
Tomcat-related logs including `catalina.out`. Logs are owned by user:group 
`pxf`:`pxf`. Other users have read access.                                      
                                    |
+| `/var/run/pxf/catalina.pid`    | PXF Tomcat container PID location           
                                                                                
                                                                                
                               |
+
+
+## <a id="install_pxf_plug_ambari"></a>Installing PXF Plug-ins Using Ambari
+
+If you are using Ambari to install and manage your HAWQ cluster, you do *not* 
need to follow the manual installation steps in this topic. Installing using 
the Ambari web interface installs all of the necessary PXF plug-in components.
+
+## <a id="install_pxf_plug_cmdline"></a>Installing PXF Plug-ins from the 
Command Line
+
+Each PXF service plug-in resides in its own RPM.  You may have built these 
RPMs in the Apache HAWQ open source project repository (see [PXF Build 
Instructions](https://github.com/apache/incubator-hawq/blob/master/pxf/README.md)),
 or these RPMs may have been included in a commercial product download package.
+
+RPMs for PXF plug-ins must be installed on *each* node in your cluster.  The 
PXF service must be restarted on each node after installation of plug-ins.
+
+### <a id="topic_qsz_wq5_25"></a>Install Prerequisite Packages
+All PXF plug-ins require that both the Tomcat and PXF service packages be 
installed on each node in your cluster. 
+
+Install Tomcat:
+
+```shell
+$ sudo rpm -i apache-tomcat-x.x.x-el6.noarch.rpm
+```
+    
+where *x.x.x* corresponds to the version of Tomcat required by PXF.  The 
appropriate version of Tomcat is included in the PXF RPM bundle.
+
+Install the PXF service:
+
+```shell
+$ sudo rpm -i pxf-service-n.n.n-x.el6.noarch.rpm
+```
+
+where *n.n.n-x* corresponds to the PXF version and build number you wish to 
install.
+
+Installing the PXF service package:
+
+   - creates a `/usr/lib/pxf-n.n.n` directory, adding a softlink from 
`/usr/lib/pxf` to this directory
+   - copies the PXF service JAR file `pxf-service-n.n.n.jar` to 
`/usr/lib/pxf-n.n.n/`
+   - creates a softlink to `pxf-service.jar` in `/usr/lib/pxf-n.n.n/`
+   - sets up the PXF service configuration files
+   - starts the PXF service
+
+
+### <a id="topic_qsz_wq5_25"></a>Install the PXF HDFS Plug-in
+
+To install PXF support for HDFS, perform the following steps on *each* node in 
your cluster:
+
+1. Install the Tomcat and PXF service packages as described in the previous 
section.
+
+2. Install the PXF HDFS plug-in:
+
+    ```shell
+    $ sudo rpm -i pxf-hdfs-n.n.n-x.el6.noarch.rpm
+    ```
+
+    The install copies the HDFS JAR file `pxf-hdfs-n.n.n.jar` to 
`/usr/lib/pxf-n.n.n/` and creates a softlink to `pxf-hdfs.jar` in that 
directory.
+
+### <a id="topic_s1y_1v5_25"></a>Install the PXF Hive Plug-in
+
+To install PXF support for Hive, perform the following steps on *each* node in 
your cluster:
+
+1.  Install the Tomcat, PXF service, and PXF HDFS RPMs as previously described.
+
+2.  Install the PXF Hive plug-in:
+
+    ```shell
+    $ sudo rpm -i pxf-hive-n.n.n-x.el6.noarch.rpm
+    ```
+
+    The install copies the Hive JAR file `pxf-hive-n.n.n.jar` to 
`/usr/lib/pxf-n.n.n` and creates a softlink to `pxf-hive.jar` in that 
directory.
+
+### <a id="topic_std_cv5_25"></a>Install the PXF HBase Plug-in
+
+To install PXF support for HBase, perform the following steps on *each* node 
in your cluster. 
+
+1.  Install the Tomcat, PXF Service, and PXF HDFS RPMs as previously described.
+
+2.  Install the PXF HBase plug-in:
+
+    ```shell
+    $ sudo rpm -i pxf-hbase-n.n.n-x.el6.noarch.rpm
+    ```
+
+    The install copies the HBase JAR file `pxf-hbase-n.n.n.jar` to 
`/usr/lib/pxf-n.n.n` and creates a softlink to `pxf-hbase.jar` in that 
directory.
+
+3.  Add the PXF HBase plug-in JAR file to the HBase `CLASSPATH` by updating 
the `HBASE_CLASSPATH` environment variable setting in the HBase environment 
file `/etc/hbase/conf/hbase-env.sh`:
+
+    ```shell
+    export HBASE_CLASSPATH=${HBASE_CLASSPATH}:/usr/lib/pxf/pxf-hbase.jar
+    ```
+
+4.  Restart the HBase service after making this update to HBase configuration.
+
+    If you are on the HBase Master node:
+
+    ```shell
+    $ su -l hbase -c "/usr/hdp/current/hbase-master/bin/hbase-daemon.sh 
restart master; sleep 25"
+    ```
+
+    If you are on an HBase Region Server node:
+
+    ```shell
+    $ su -l hbase -c "/usr/hdp/current/hbase-regionserver/bin/hbase-daemon.sh 
restart regionserver"
+    ```
+
+### <a id="topic_s1y_1v5_25"></a>Install the PXF JSON Plug-in
+
+To install PXF support for JSON, perform the following steps on *each* node in 
your cluster:
+
+1.  Install the Tomcat, PXF service, and PXF HDFS RPMs as previously described.
+
+2.  Install the PXF JSON plug-in:
+
+    ```shell
+    $ sudo rpm -i pxf-json-n.n.n-x.el6.noarch.rpm
+    ```
+
+    The install copies the JSON JAR file `pxf-json-n.n.n.jar` to 
`/usr/lib/pxf-n.n.n` and creates a softlink to `pxf-json.jar` in that 
directory.
+
+### <a id="topic_qsz_restartpxf"></a>Restart the PXF Service
+Restart PXF on *each* node after installing the PXF service and any PXF 
plug-ins.
+
+``` shell
+$ sudo service pxf-service restart
+```
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-hawq-docs/blob/7514e193/pxf/JsonPXF.html.md.erb
----------------------------------------------------------------------
diff --git a/pxf/JsonPXF.html.md.erb b/pxf/JsonPXF.html.md.erb
new file mode 100644
index 0000000..5283349
--- /dev/null
+++ b/pxf/JsonPXF.html.md.erb
@@ -0,0 +1,197 @@
+---
+title: Accessing JSON File Data
+---
+
+The PXF JSON plug-in reads native JSON stored in HDFS.  The plug-in supports 
common data types, as well as basic (N-level) projection and arrays.
+
+To access JSON file data with HAWQ, the data must be stored in HDFS and an 
external table created from the HDFS data store.
+
+## Prerequisites<a id="jsonplugprereq"></a>
+
+Before working with JSON file data using HAWQ and PXF, ensure that:
+
+-   The PXF HDFS plug-in is installed on all cluster nodes.
+-   The PXF JSON plug-in is installed on all cluster nodes.
+-   You have tested PXF on HDFS.
+
+
+## Working with JSON Files<a id="topic_workwjson"></a>
+
+JSON is a text-based data-interchange format.  JSON data is typically stored 
in a file with a `.json` suffix. A `.json` file will contain a collection of 
objects.  A JSON object is a collection of unordered name/value pairs.  A value 
can be a string, a number, true, false, null, or an object or array. Objects 
and arrays can be nested.
+
+Refer to [Introducing JSON](http://www.json.org/) for specific information on 
JSON syntax.
+
+Sample JSON data file content:
+
+``` json
+  {
+    "created_at":"MonSep3004:04:53+00002013",
+    "id_str":"384529256681725952",
+    "user": {
+      "id":31424214,
+       "location":"COLUMBUS"
+    },
+    "coordinates":null
+  }
+```
+
+### JSON to HAWQ Data Type Mapping<a id="topic_workwjson"></a>
+
+To represent JSON data in HAWQ, map data values that use a primitive data type 
to HAWQ columns of the same type. JSON supports complex data types including 
projections and arrays. Use N-level projection to map members of nested objects 
and arrays to primitive data types.
+
+The following table summarizes external mapping rules for JSON data.
+
+<caption><span class="tablecap">Table 1. JSON Mapping</span></caption>
+
+<a id="topic_table_jsondatamap"></a>
+
+| JSON Data Type                                                    | HAWQ 
Data Type                                                                       
                                                                                
                                     |
+|-------------------------------------------------------------------|-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
+| Primitive type (integer, float, string, boolean, null) | Use the 
corresponding HAWQ built-in data type; see [Data 
Types](../reference/HAWQDataTypes.html). |
+| Array                         | Use `[]` brackets to identify a specific 
array index to a member of primitive type.                                      
                                                      |
+| Object                | Use dot `.` notation to specify each level of 
projection (nesting) to a member of a primitive type.                           
                                                              |
+
+
+### JSON File Read Modes<a id="topic_jsonreadmodes"></a>
+
+
+The PXF JSON plug-in reads data in one of two modes. The default mode expects 
one full JSON record per line.  The JSON plug-in also supports a read mode 
operating on multi-line JSON records.
+
+In the following discussion, a data set defined by a sample schema will be 
represented using each read mode of the PXF JSON plug-in.  The sample schema 
contains data fields with the following names and data types:
+
+   - "created_at" - text
+   - "id_str" - text
+   - "user" - object
+      - "id" - integer
+      - "location" - text
+   - "coordinates" - object (optional)
+      - "type" - text
+      - "values" - array
+         - [0] - integer
+         - [1] - integer
+
+
+Example 1 - Data Set for Single-JSON-Record-Per-Line Read Mode:
+
+``` pre
+{"created_at":"FriJun0722:45:03+00002013","id_str":"343136551322136576","user":{
+"id":395504494,"location":"NearCornwall"},"coordinates":{"type":"Point","values"
+: [ 6, 50 ]}},
+{"created_at":"FriJun0722:45:02+00002013","id_str":"343136547115253761","user":{
+"id":26643566,"location":"Austin,Texas"}, "coordinates": null},
+{"created_at":"FriJun0722:45:02+00002013","id_str":"343136547136233472","user":{
+"id":287819058,"location":""}, "coordinates": null}
+```  
+
+Example 2 - Data Set for Multi-Line JSON Record Read Mode:
+
+``` json
+{
+  "root":[
+    {
+      "record_obj":{
+        "created_at":"MonSep3004:04:53+00002013",
+        "id_str":"384529256681725952",
+        "user":{
+          "id":31424214,
+          "location":"COLUMBUS"
+        },
+        "coordinates":null
+      },
+      "record_obj":{
+        "created_at":"MonSep3004:04:54+00002013",
+        "id_str":"384529260872228864",
+        "user":{
+          "id":67600981,
+          "location":"KryberWorld"
+        },
+        "coordinates":{
+          "type":"Point",
+          "values":[
+             8,
+             52
+          ]
+        }
+      }
+    }
+  ]
+}
+```
+
+## Loading JSON Data to HDFS<a id="jsontohdfs"></a>
+
+The PXF JSON plug-in reads native JSON stored in HDFS. Before JSON data can 
be queried via HAWQ, it must first be loaded to an HDFS data store.
+
+Copy and paste the single line JSON record data set to a file named 
`singleline.json`.  Similarly, copy and paste the multi-line JSON record data 
set to `multiline.json`.
+
+**Note**:  Ensure there are **no** blank lines in your JSON files.
+
+Add the data set files to the HDFS data store:
+
+``` shell
+$ hdfs dfs -mkdir /user/data
+$ hdfs dfs -put singleline.json /user/data
+$ hdfs dfs -put multiline.json /user/data
+```
+
+Once loaded to HDFS, JSON data may be queried and analyzed via HAWQ.
+
+## Querying External JSON Data<a id="jsoncetsyntax1"></a>
+
+Use the following syntax to create an external table representing JSON data: 
+
+``` sql
+CREATE EXTERNAL TABLE table_name 
+    ( column_name data_type [, ...] | LIKE other_table )
+LOCATION ( 'pxf://host[:port]/path-to-data?PROFILE=Json[&IDENTIFIER=value]' )
+      FORMAT 'CUSTOM' ( FORMATTER='pxfwritable_import' );
+```
+JSON-plug-in-specific keywords and values used in the `CREATE EXTERNAL TABLE` 
call are described below.
+
+| Keyword  | Value |
+|-------|-------------------------------------|
+| host    | Specify the HDFS NameNode in the `host` field. |
+| PROFILE    | The `PROFILE` keyword must specify the value `Json`. |
+| IDENTIFIER  | Include the `IDENTIFIER` keyword and value in the `LOCATION` 
string only when accessing a JSON file with multi-line records. `value` should 
identify the member name used to determine the encapsulating JSON object to 
return.  (If the JSON file is the multi-line record Example 2 above, 
`&IDENTIFIER=created_at` would be specified.) |  
+| FORMAT    | The `FORMAT` clause must specify `CUSTOM`. |
+| FORMATTER    | The JSON `CUSTOM` format supports only the built-in 
`pxfwritable_import` `FORMATTER`. |
+
+
+### Example 1 <a id="jsonexample1"></a>
+
+The following `CREATE EXTERNAL TABLE` SQL call creates a queryable external 
table based on the data in the single-line-per-record JSON example.
+
+``` sql 
+CREATE EXTERNAL TABLE sample_json_singleline_tbl(
+  created_at TEXT,
+  id_str TEXT,
+  text TEXT,
+  "user.id" INTEGER,
+  "user.location" TEXT,
+  "coordinates.values[0]" INTEGER,
+  "coordinates.values[1]" INTEGER
+)
+LOCATION('pxf://namenode:51200/user/data/singleline.json?PROFILE=Json')
+FORMAT 'CUSTOM' (FORMATTER='pxfwritable_import');
+SELECT * FROM sample_json_singleline_tbl;
+```
+
+Notice the use of `.` projection to access the nested fields in the `user` and 
`coordinates` objects.  Also notice the use of `[]` to access the specific 
elements of the `coordinates.values` array.
+
+### Example 2 <a id="jsonexample2"></a>
+
+A `CREATE EXTERNAL TABLE` SQL call to create a queryable external table based 
on the multi-line-per-record JSON data set would be very similar to that of the 
single line data set above. You might specify a different database name, 
`sample_json_multiline_tbl` for example. 
+
+The `LOCATION` clause would differ.  The `IDENTIFIER` keyword and an 
associated value must be specified when reading from multi-line JSON records:
+
+``` sql
+LOCATION('pxf://namenode:51200/user/data/multiline.json?PROFILE=Json&IDENTIFIER=created_at')
+```
+
+`created_at` identifies the member name used to determine the encapsulating 
JSON object, `record_obj` in this case.
+
+To query this external table populated with JSON data:
+
+``` sql
+SELECT * FROM sample_json_multiline_tbl;
+```
\ No newline at end of file

Reply via email to