You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ja...@apache.org on 2014/04/22 05:09:56 UTC

[2/2] git commit: Updates to Pig Integration - James forgot to add some files (Ravi)

Updates to Pig Integration - James forgot to add some files (Ravi)


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

Branch: refs/heads/master
Commit: a615cebfb05e9a16df69ab94b7684961350869aa
Parents: 920a5cc
Author: James Taylor <ja...@apache.org>
Authored: Mon Apr 21 20:09:53 2014 -0700
Committer: James Taylor <ja...@apache.org>
Committed: Mon Apr 21 20:09:53 2014 -0700

----------------------------------------------------------------------
 .../phoenix/pig/PhoenixHBaseLoaderIT.java       | 440 +++++++++++++++++++
 .../phoenix/pig/PhoenixHBaseStorerIT.java       | 195 ++++++++
 .../phoenix/pig/PhoenixPigConfigurationIT.java  | 107 +++++
 .../apache/phoenix/pig/PhoenixHBaseLoader.java  | 253 +++++++++++
 .../phoenix/pig/hadoop/PhoenixInputFormat.java  | 167 +++++++
 .../phoenix/pig/hadoop/PhoenixInputSplit.java   | 104 +++++
 .../phoenix/pig/hadoop/PhoenixRecordReader.java | 133 ++++++
 .../util/ColumnInfoToStringEncoderDecoder.java  |  69 +++
 .../phoenix/pig/util/PhoenixPigSchemaUtil.java  |  70 +++
 .../pig/util/QuerySchemaParserFunction.java     | 121 +++++
 .../pig/util/TableSchemaParserFunction.java     |  54 +++
 .../org/apache/phoenix/pig/util/TypeUtil.java   | 312 +++++++++++++
 .../pig/PhoenixPigConfigurationTest.java        |  86 ++++
 .../ColumnInfoToStringEncoderDecoderTest.java   |  61 +++
 .../pig/util/PhoenixPigSchemaUtilTest.java      |  84 ++++
 .../pig/util/QuerySchemaParserFunctionTest.java | 109 +++++
 .../pig/util/TableSchemaParserFunctionTest.java |  56 +++
 17 files changed, 2421 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/a615cebf/phoenix-pig/src/it/java/org/apache/phoenix/pig/PhoenixHBaseLoaderIT.java
----------------------------------------------------------------------
diff --git a/phoenix-pig/src/it/java/org/apache/phoenix/pig/PhoenixHBaseLoaderIT.java b/phoenix-pig/src/it/java/org/apache/phoenix/pig/PhoenixHBaseLoaderIT.java
new file mode 100644
index 0000000..90f1c79
--- /dev/null
+++ b/phoenix-pig/src/it/java/org/apache/phoenix/pig/PhoenixHBaseLoaderIT.java
@@ -0,0 +1,440 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ *distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you maynot use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicablelaw or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.pig;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.phoenix.jdbc.PhoenixDriver;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.TestUtil;
+import org.apache.pig.ExecType;
+import org.apache.pig.PigServer;
+import org.apache.pig.backend.hadoop.datastorage.ConfigurationUtil;
+import org.apache.pig.builtin.mock.Storage;
+import org.apache.pig.builtin.mock.Storage.Data;
+import org.apache.pig.data.DataType;
+import org.apache.pig.data.Tuple;
+import org.apache.pig.impl.logicalLayer.schema.Schema;
+import org.apache.pig.impl.logicalLayer.schema.Schema.FieldSchema;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * 
+ * Test class to run all the integration tests against a virtual map reduce cluster.
+ */
+public class PhoenixHBaseLoaderIT {
+    
+    private static final Log LOG = LogFactory.getLog(PhoenixHBaseLoaderIT.class);
+    private static final String SCHEMA_NAME = "T";
+    private static final String TABLE_NAME = "A";
+    private static final String TABLE_FULL_NAME = SchemaUtil.getTableName(SCHEMA_NAME, TABLE_NAME);
+    private static HBaseTestingUtility hbaseTestUtil;
+    private static String zkQuorum;
+    private static Connection conn;
+    private static PigServer pigServer;
+    private static Configuration conf;
+
+    @BeforeClass
+    public static void setUp() throws Exception {
+        hbaseTestUtil = new HBaseTestingUtility();
+        hbaseTestUtil.getConfiguration().set(QueryServices.DROP_METADATA_ATTRIB, Boolean.toString(true));
+        hbaseTestUtil.startMiniCluster();
+
+        Class.forName(PhoenixDriver.class.getName());
+        zkQuorum = "localhost:" + hbaseTestUtil.getZkCluster().getClientPort();
+        Properties props = TestUtil.TEST_PROPERTIES;
+        props.put(QueryServices.DROP_METADATA_ATTRIB, Boolean.toString(true));
+        conn = DriverManager.getConnection(PhoenixRuntime.JDBC_PROTOCOL +
+                 PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR + zkQuorum,props);
+
+        conf = hbaseTestUtil.getConfiguration();
+     }
+    
+    @Before
+    public void beforeTest() throws Exception {
+        pigServer = new PigServer(ExecType.LOCAL,
+                ConfigurationUtil.toProperties(conf));
+    }
+
+    /**
+     * Validates the schema returned for a table with Pig data types.
+     * @throws Exception
+     */
+    @Test
+    public void testSchemaForTable() throws Exception {
+        final String ddl = String.format("CREATE TABLE %s "
+                + "  (a_string varchar not null, a_binary varbinary not null, a_integer integer, cf1.a_float float"
+                + "  CONSTRAINT pk PRIMARY KEY (a_string, a_binary))\n", TABLE_FULL_NAME);
+        conn.createStatement().execute(ddl);
+
+        pigServer.registerQuery(String.format(
+                "A = load 'hbase://table/%s' using org.apache.phoenix.pig.PhoenixHBaseLoader('%s');", TABLE_FULL_NAME,
+                zkQuorum));
+        
+        final Schema schema = pigServer.dumpSchema("A");
+        List<FieldSchema> fields = schema.getFields();
+        assertEquals(4, fields.size());
+        assertTrue(fields.get(0).alias.equalsIgnoreCase("a_string"));
+        assertTrue(fields.get(0).type == DataType.CHARARRAY);
+        assertTrue(fields.get(1).alias.equalsIgnoreCase("a_binary"));
+        assertTrue(fields.get(1).type == DataType.BYTEARRAY);
+        assertTrue(fields.get(2).alias.equalsIgnoreCase("a_integer"));
+        assertTrue(fields.get(2).type == DataType.INTEGER);
+        assertTrue(fields.get(3).alias.equalsIgnoreCase("a_float"));
+        assertTrue(fields.get(3).type == DataType.FLOAT);
+    }
+
+    /**
+     * Validates the schema returned when specific columns of a table are given as part of LOAD .
+     * @throws Exception
+     */
+    @Test
+    public void testSchemaForTableWithSpecificColumns() throws Exception {
+        
+        //create the table
+        final String ddl = "CREATE TABLE " + TABLE_FULL_NAME 
+                + "  (ID INTEGER NOT NULL PRIMARY KEY,NAME VARCHAR, AGE INTEGER) ";
+        conn.createStatement().execute(ddl);
+        
+        
+        final String selectColumns = "ID,NAME";
+        pigServer.registerQuery(String.format(
+                "A = load 'hbase://table/%s/%s' using org.apache.phoenix.pig.PhoenixHBaseLoader('%s');",
+                TABLE_FULL_NAME, selectColumns, zkQuorum));
+        
+        Schema schema = pigServer.dumpSchema("A");
+        List<FieldSchema> fields = schema.getFields();
+        assertEquals(2, fields.size());
+        assertTrue(fields.get(0).alias.equalsIgnoreCase("ID"));
+        assertTrue(fields.get(0).type == DataType.INTEGER);
+        assertTrue(fields.get(1).alias.equalsIgnoreCase("NAME"));
+        assertTrue(fields.get(1).type == DataType.CHARARRAY);
+        
+    }
+    
+    /**
+     * Validates the schema returned when a SQL SELECT query is given as part of LOAD .
+     * @throws Exception
+     */
+    @Test
+    public void testSchemaForQuery() throws Exception {
+        
+       //create the table.
+        String ddl = String.format("CREATE TABLE " + TABLE_FULL_NAME +
+                 "  (A_STRING VARCHAR NOT NULL, A_DECIMAL DECIMAL NOT NULL, CF1.A_INTEGER INTEGER, CF2.A_DOUBLE DOUBLE"
+                + "  CONSTRAINT pk PRIMARY KEY (A_STRING, A_DECIMAL))\n", TABLE_FULL_NAME);
+        conn.createStatement().execute(ddl);
+        
+        
+        
+        //sql query for LOAD
+        final String sqlQuery = "SELECT A_STRING,CF1.A_INTEGER,CF2.A_DOUBLE FROM " + TABLE_FULL_NAME;
+        pigServer.registerQuery(String.format(
+                "A = load 'hbase://query/%s' using org.apache.phoenix.pig.PhoenixHBaseLoader('%s');",
+                sqlQuery, zkQuorum));
+        
+        //assert the schema.
+        Schema schema = pigServer.dumpSchema("A");
+        List<FieldSchema> fields = schema.getFields();
+        assertEquals(3, fields.size());
+        assertTrue(fields.get(0).alias.equalsIgnoreCase("a_string"));
+        assertTrue(fields.get(0).type == DataType.CHARARRAY);
+        assertTrue(fields.get(1).alias.equalsIgnoreCase("a_integer"));
+        assertTrue(fields.get(1).type == DataType.INTEGER);
+        assertTrue(fields.get(2).alias.equalsIgnoreCase("a_double"));
+        assertTrue(fields.get(2).type == DataType.DOUBLE);
+    }
+    
+    /**
+     * Validates the schema when it is given as part of LOAD..AS
+     * @throws Exception
+     */
+    @Test
+    public void testSchemaForTableWithAlias() throws Exception {
+        
+        //create the table.
+        String ddl = "CREATE TABLE  " + TABLE_FULL_NAME 
+                + "  (A_STRING VARCHAR NOT NULL, A_DECIMAL DECIMAL NOT NULL, CF1.A_INTEGER INTEGER, CF2.A_DOUBLE DOUBLE"
+                + "  CONSTRAINT pk PRIMARY KEY (A_STRING, A_DECIMAL)) \n";
+        conn.createStatement().execute(ddl);
+        
+        //select query given as part of LOAD.
+        final String sqlQuery = "SELECT A_STRING,A_DECIMAL,CF1.A_INTEGER,CF2.A_DOUBLE FROM " + TABLE_FULL_NAME;
+        
+        LOG.info(String.format("Generated SQL Query [%s]",sqlQuery));
+        
+        pigServer.registerQuery(String.format(
+                "raw = load 'hbase://query/%s' using org.apache.phoenix.pig.PhoenixHBaseLoader('%s') AS (a:chararray,b:bigdecimal,c:int,d:double);",
+                sqlQuery, zkQuorum));
+        
+        //test the schema.
+        Schema schema = pigServer.dumpSchema("raw");
+        List<FieldSchema> fields = schema.getFields();
+        assertEquals(4, fields.size());
+        assertTrue(fields.get(0).alias.equalsIgnoreCase("a"));
+        assertTrue(fields.get(0).type == DataType.CHARARRAY);
+        assertTrue(fields.get(1).alias.equalsIgnoreCase("b"));
+        assertTrue(fields.get(1).type == DataType.BIGDECIMAL);
+        assertTrue(fields.get(2).alias.equalsIgnoreCase("c"));
+        assertTrue(fields.get(2).type == DataType.INTEGER);
+        assertTrue(fields.get(3).alias.equalsIgnoreCase("d"));
+        assertTrue(fields.get(3).type == DataType.DOUBLE);
+    }
+    
+    /**
+     * @throws Exception
+     */
+    @Test
+    public void testDataForTable() throws Exception {
+        
+         //create the table
+         String ddl = "CREATE TABLE  " + TABLE_FULL_NAME 
+                + "  (ID  INTEGER NOT NULL PRIMARY KEY, NAME VARCHAR, AGE INTEGER) ";
+                
+        conn.createStatement().execute(ddl);
+        
+        //prepare data with 10 rows having age 25 and the other 30.
+        final String dml = "UPSERT INTO " + TABLE_FULL_NAME + " VALUES(?,?,?)";
+        PreparedStatement stmt = conn.prepareStatement(dml);
+        int rows = 20;
+        for(int i = 0 ; i < rows; i++) {
+            stmt.setInt(1, i);
+            stmt.setString(2, "a"+i);
+            stmt.setInt(3, (i % 2 == 0) ? 25 : 30);
+            stmt.execute();    
+        }
+        conn.commit();
+         
+        //load data and filter rows whose age is > 25
+        pigServer.registerQuery(String.format(
+                "A = load 'hbase://table/%s' using org.apache.phoenix.pig.PhoenixHBaseLoader('%s');", TABLE_FULL_NAME,
+                zkQuorum));
+        pigServer.registerQuery("B = FILTER A BY AGE > 25;");
+        
+        final Iterator<Tuple> iterator = pigServer.openIterator("B");
+        int recordsRead = 0;
+        while (iterator.hasNext()) {
+            final Tuple each = iterator.next();
+            assertEquals(3, each.size());
+            recordsRead++;
+        }
+        assertEquals(rows/2, recordsRead);
+    }
+    
+    /**
+     * @throws Exception
+     */
+    @Test
+    public void testDataForSQLQuery() throws Exception {
+        
+         //create the table
+         String ddl = "CREATE TABLE  " + TABLE_FULL_NAME 
+                + "  (ID  INTEGER NOT NULL PRIMARY KEY, NAME VARCHAR, AGE INTEGER) ";
+                
+        conn.createStatement().execute(ddl);
+        
+        //prepare data with 10 rows having age 25 and the other 30.
+        final String dml = "UPSERT INTO " + TABLE_FULL_NAME + " VALUES(?,?,?)";
+        PreparedStatement stmt = conn.prepareStatement(dml);
+        int rows = 20;
+        for(int i = 0 ; i < rows; i++) {
+            stmt.setInt(1, i);
+            stmt.setString(2, "a"+i);
+            stmt.setInt(3, (i % 2 == 0) ? 25 : 30);
+            stmt.execute();    
+        }
+        conn.commit();
+        
+        //sql query
+        final String sqlQuery = " SELECT ID,NAME,AGE FROM " + TABLE_FULL_NAME + " WHERE AGE > 25";
+        //load data and filter rows whose age is > 25
+        pigServer.registerQuery(String.format(
+                "A = load 'hbase://query/%s' using org.apache.phoenix.pig.PhoenixHBaseLoader('%s');", sqlQuery,
+                zkQuorum));
+        
+        final Iterator<Tuple> iterator = pigServer.openIterator("A");
+        int recordsRead = 0;
+        while (iterator.hasNext()) {
+            iterator.next();
+            recordsRead++;
+        }
+        assertEquals(rows/2, recordsRead);
+    }
+    
+    /**
+     * @throws Exception
+     */
+    @Test
+    public void testGroupingOfDataForTable() throws Exception {
+        
+         //create the table
+         String ddl = "CREATE TABLE  " + TABLE_FULL_NAME 
+                + "  (ID  INTEGER NOT NULL PRIMARY KEY, NAME VARCHAR, AGE INTEGER, SAL INTEGER) ";
+                
+        conn.createStatement().execute(ddl);
+        
+        //prepare data with 10 rows having age 25 and the other 30.
+        final String dml = "UPSERT INTO " + TABLE_FULL_NAME + " VALUES(?,?,?,?)";
+        PreparedStatement stmt = conn.prepareStatement(dml);
+        int rows = 20;
+        int j = 0, k = 0;
+        for(int i = 0 ; i < rows; i++) {
+            stmt.setInt(1, i);
+            stmt.setString(2, "a"+i);
+            if(i % 2 == 0) {
+                stmt.setInt(3, 25);
+                stmt.setInt(4, 10 * 2 * j++);    
+            } else {
+                stmt.setInt(3, 30);
+                stmt.setInt(4, 10 * 3 * k++);
+            }
+            
+            stmt.execute();    
+        }
+        conn.commit();
+        
+        //prepare the mock storage with expected output
+        final Data data = Storage.resetData(pigServer);
+        List<Tuple> expectedList = new ArrayList<Tuple>();
+        expectedList.add(Storage.tuple(0,180));
+        expectedList.add(Storage.tuple(0,270));
+        
+         //load data and filter rows whose age is > 25
+        pigServer.setBatchOn();
+        pigServer.registerQuery(String.format(
+                "A = load 'hbase://table/%s' using org.apache.phoenix.pig.PhoenixHBaseLoader('%s');", TABLE_FULL_NAME,
+                zkQuorum));
+        
+        pigServer.registerQuery("B = GROUP A BY AGE;");
+        pigServer.registerQuery("C = FOREACH B GENERATE MIN(A.SAL),MAX(A.SAL);");
+        pigServer.registerQuery("STORE C INTO 'out' using mock.Storage();");
+        pigServer.executeBatch();
+        
+        List<Tuple> actualList = data.get("out");
+        assertEquals(expectedList, actualList);
+    }
+    
+    /**
+     * Tests both  {@link PhoenixHBaseLoader} and {@link PhoenixHBaseStorage} 
+     * @throws Exception
+     */
+    @Test
+    public void testLoadAndStore() throws Exception {
+        
+         //create the tables
+         final String sourceTableddl = "CREATE TABLE  " + TABLE_FULL_NAME 
+                + "  (ID  INTEGER NOT NULL PRIMARY KEY, NAME VARCHAR, AGE INTEGER, SAL INTEGER) ";
+         
+         final String targetTable = "AGGREGATE";
+         final String targetTableddl = "CREATE TABLE " + targetTable 
+                 +  "(AGE INTEGER NOT NULL PRIMARY KEY , MIN_SAL INTEGER , MAX_SAL INTEGER) ";
+                 
+        conn.createStatement().execute(sourceTableddl);
+        conn.createStatement().execute(targetTableddl);
+        
+        //prepare data with 10 rows having age 25 and the other 30.
+        final String dml = "UPSERT INTO " + TABLE_FULL_NAME + " VALUES(?,?,?,?)";
+        PreparedStatement stmt = conn.prepareStatement(dml);
+        int rows = 20;
+        int j = 0, k = 0;
+        for(int i = 0 ; i < rows; i++) {
+            stmt.setInt(1, i);
+            stmt.setString(2, "a"+i);
+            if(i % 2 == 0) {
+                stmt.setInt(3, 25);
+                stmt.setInt(4, 10 * 2 * j++);    
+            } else {
+                stmt.setInt(3, 30);
+                stmt.setInt(4, 10 * 3 * k++);
+            }
+            
+            stmt.execute();    
+        }
+        conn.commit();
+        
+            
+         //load data and filter rows whose age is > 25
+        pigServer.setBatchOn();
+        pigServer.registerQuery(String.format(
+                "A = load 'hbase://table/%s' using org.apache.phoenix.pig.PhoenixHBaseLoader('%s');", TABLE_FULL_NAME,
+                zkQuorum));
+        
+        pigServer.registerQuery("B = GROUP A BY AGE;");
+        pigServer.registerQuery("C = FOREACH B GENERATE group as AGE,MIN(A.SAL),MAX(A.SAL);");
+        pigServer.registerQuery("STORE C INTO 'hbase://" + targetTable 
+                + "' using " + PhoenixHBaseStorage.class.getName() + "('"
+                 + zkQuorum + "', '-batchSize 1000');");
+        pigServer.executeBatch();
+        
+        //validate the data with what is stored.
+        final String selectQuery = "SELECT AGE , MIN_SAL ,MAX_SAL FROM " + targetTable + " ORDER BY AGE";
+        final ResultSet rs = conn.createStatement().executeQuery(selectQuery);
+        rs.next();
+        assertEquals(25, rs.getInt("AGE"));
+        assertEquals(0, rs.getInt("MIN_SAL"));
+        assertEquals(180, rs.getInt("MAX_SAL"));
+        rs.next();
+        assertEquals(30, rs.getInt("AGE"));
+        assertEquals(0, rs.getInt("MIN_SAL"));
+        assertEquals(270, rs.getInt("MAX_SAL"));
+    }
+    
+    @After
+    public void tearDown() throws Exception {
+        dropTable(TABLE_FULL_NAME);
+        pigServer.shutdown();
+    }
+
+
+    private void dropTable(String tableFullName) throws SQLException {
+      Preconditions.checkNotNull(conn);
+      conn.createStatement().execute(String.format("DROP TABLE %s",tableFullName));
+    }
+
+    @AfterClass
+    public static void tearDownAfterClass() throws Exception {
+        conn.close();
+        PhoenixDriver.INSTANCE.close();
+        hbaseTestUtil.shutdownMiniCluster();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/a615cebf/phoenix-pig/src/it/java/org/apache/phoenix/pig/PhoenixHBaseStorerIT.java
----------------------------------------------------------------------
diff --git a/phoenix-pig/src/it/java/org/apache/phoenix/pig/PhoenixHBaseStorerIT.java b/phoenix-pig/src/it/java/org/apache/phoenix/pig/PhoenixHBaseStorerIT.java
new file mode 100644
index 0000000..32d0ff9
--- /dev/null
+++ b/phoenix-pig/src/it/java/org/apache/phoenix/pig/PhoenixHBaseStorerIT.java
@@ -0,0 +1,195 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ *distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you maynot use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicablelaw or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.pig;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.Statement;
+import java.util.Collection;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.phoenix.jdbc.PhoenixDriver;
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.apache.pig.ExecType;
+import org.apache.pig.PigServer;
+import org.apache.pig.backend.executionengine.ExecJob.JOB_STATUS;
+import org.apache.pig.backend.hadoop.datastorage.ConfigurationUtil;
+import org.apache.pig.builtin.mock.Storage;
+import org.apache.pig.builtin.mock.Storage.Data;
+import org.apache.pig.data.Tuple;
+import org.apache.pig.data.TupleFactory;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import com.google.common.collect.Lists;
+
+public class PhoenixHBaseStorerIT {
+
+    private static TupleFactory tupleFactory;
+    private static HBaseTestingUtility hbaseTestUtil;
+    private static String zkQuorum;
+    private static Connection conn;
+    private static PigServer pigServer;
+    private static Configuration conf;
+
+    @BeforeClass
+    public static void setUp() throws Exception {
+        hbaseTestUtil = new HBaseTestingUtility();
+        hbaseTestUtil.startMiniCluster();
+
+        Class.forName(PhoenixDriver.class.getName());
+        zkQuorum = "localhost:" + hbaseTestUtil.getZkCluster().getClientPort();
+        conn = DriverManager.getConnection(PhoenixRuntime.JDBC_PROTOCOL +
+                 PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR + zkQuorum);
+        conf = hbaseTestUtil.getConfiguration();
+        // Pig variables
+        tupleFactory = TupleFactory.getInstance();
+    }
+    
+    @Before
+    public void beforeTest() throws Exception {
+        pigServer = new PigServer(ExecType.LOCAL,
+                ConfigurationUtil.toProperties(conf));
+    }
+    
+    @After
+    public void tearDown() throws Exception {
+         pigServer.shutdown();
+    }
+
+    @AfterClass
+    public static void tearDownAfterClass() throws Exception {
+        conn.close();
+        PhoenixDriver.INSTANCE.close();
+        hbaseTestUtil.shutdownMiniCluster();
+    }
+
+    /**
+     * Basic test - writes data to a Phoenix table and compares the data written
+     * to expected
+     * 
+     * @throws Exception
+     */
+    @Test
+    public void testStorer() throws Exception {
+        final String tableName = "TABLE1";
+        final Statement stmt = conn.createStatement();
+
+        stmt.execute("CREATE TABLE " + tableName +
+                 " (ID INTEGER NOT NULL PRIMARY KEY, NAME VARCHAR)");
+
+        final Data data = Storage.resetData(pigServer);
+        final Collection<Tuple> list = Lists.newArrayList();
+
+        // Create input dataset
+        int rows = 100;
+        for (int i = 0; i < rows; i++) {
+            Tuple t = tupleFactory.newTuple();
+            t.append(i);
+            t.append("a" + i);
+            list.add(t);
+        }
+        data.set("in", "id:int, name:chararray", list);
+
+        pigServer.setBatchOn();
+        pigServer.registerQuery("A = LOAD 'in' USING mock.Storage();");
+
+        pigServer.registerQuery("Store A into 'hbase://" + tableName
+                               + "' using " + PhoenixHBaseStorage.class.getName() + "('"
+                                + zkQuorum + "', '-batchSize 1000');");
+
+         // Now run the Pig script
+        if (pigServer.executeBatch().get(0).getStatus() != JOB_STATUS.COMPLETED) {
+            throw new RuntimeException("Job failed", pigServer.executeBatch()
+                    .get(0).getException());
+        }
+
+        // Compare data in Phoenix table to the expected
+        final ResultSet rs = stmt
+                .executeQuery("SELECT id, name FROM table1 ORDER BY id");
+
+        for (int i = 0; i < rows; i++) {
+            assertTrue(rs.next());
+            assertEquals(i, rs.getInt(1));
+            assertEquals("a" +  i, rs.getString(2));
+        }
+    }
+    
+    /**
+     * Basic test - writes specific columns data to a Phoenix table and compares the data written
+     * to expected
+     * 
+     * @throws Exception
+     */
+    @Test
+    public void testStorerForSpecificColumns() throws Exception {
+        final String tableName = "TABLE2";
+        final Statement stmt = conn.createStatement();
+
+        stmt.execute("CREATE TABLE " + tableName +
+                 " (ID INTEGER NOT NULL PRIMARY KEY, NAME VARCHAR, AGE INTEGER)");
+
+        final Data data = Storage.resetData(pigServer);
+        final Collection<Tuple> list = Lists.newArrayList();
+
+        // Create input dataset
+        int rows = 100;
+        for (int i = 0; i < rows; i++) {
+            Tuple t = tupleFactory.newTuple();
+            t.append(i);
+            t.append("a" + i);
+            t.append(i * 2);
+            list.add(t);
+        }
+        data.set("in", "id:int, name:chararray,age:int", list);
+
+        pigServer.setBatchOn();
+        pigServer.registerQuery("A = LOAD 'in' USING mock.Storage();");
+        pigServer.registerQuery("B = FOREACH A GENERATE id,name;");
+        pigServer.registerQuery("Store B into 'hbase://" + tableName + "/ID,NAME"
+                               + "' using " + PhoenixHBaseStorage.class.getName() + "('"
+                                + zkQuorum + "', '-batchSize 1000');");
+
+         // Now run the Pig script
+        if (pigServer.executeBatch().get(0).getStatus() != JOB_STATUS.COMPLETED) {
+            throw new RuntimeException("Job failed", pigServer.executeBatch()
+                    .get(0).getException());
+        }
+
+        // Compare data in Phoenix table to the expected
+        final ResultSet rs = stmt
+                .executeQuery("SELECT id, name,age FROM " + tableName + "ORDER BY id");
+
+        for (int i = 0; i < rows; i++) {
+            assertTrue(rs.next());
+            assertEquals(i, rs.getInt(1));
+            assertEquals("a" +  i, rs.getString(2));
+            assertEquals(0, rs.getInt(3));
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/a615cebf/phoenix-pig/src/it/java/org/apache/phoenix/pig/PhoenixPigConfigurationIT.java
----------------------------------------------------------------------
diff --git a/phoenix-pig/src/it/java/org/apache/phoenix/pig/PhoenixPigConfigurationIT.java b/phoenix-pig/src/it/java/org/apache/phoenix/pig/PhoenixPigConfigurationIT.java
new file mode 100644
index 0000000..e867a4d
--- /dev/null
+++ b/phoenix-pig/src/it/java/org/apache/phoenix/pig/PhoenixPigConfigurationIT.java
@@ -0,0 +1,107 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ *distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you maynot use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicablelaw or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.pig;
+
+import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL_TERMINATOR;
+import static org.apache.phoenix.util.PhoenixRuntime.PHOENIX_TEST_DRIVER_URL_PARAM;
+import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.junit.Assert.assertEquals;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.util.Properties;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
+import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.TestUtil;
+import org.junit.Test;
+
+public class PhoenixPigConfigurationIT extends BaseHBaseManagedTimeIT {
+    private static final String zkQuorum = TestUtil.LOCALHOST + JDBC_PROTOCOL_TERMINATOR + PHOENIX_TEST_DRIVER_URL_PARAM;
+    
+    @Test
+    public void testUpsertStatement() throws Exception {
+        Properties props = new Properties(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        conn.setAutoCommit(false);
+        final String tableName = "TEST_TABLE";
+        try {
+            String ddl = "CREATE TABLE "+ tableName + 
+                    "  (a_string varchar not null, a_binary varbinary not null, col1 integer" +
+                    "  CONSTRAINT pk PRIMARY KEY (a_string, a_binary))\n";
+            createTestTable(getUrl(), ddl);
+            final PhoenixPigConfiguration configuration = newConfiguration (tableName);
+            final String upserStatement = configuration.getUpsertStatement();
+            final String expectedUpsertStatement = "UPSERT INTO " + tableName + " VALUES (?, ?, ?)"; 
+            assertEquals(expectedUpsertStatement, upserStatement);
+        } finally {
+            conn.close();
+        }
+    }
+    
+    @Test
+    public void testSelectStatement() throws Exception {
+        Properties props = new Properties(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        conn.setAutoCommit(false);
+        final String tableName = "TEST_TABLE";
+        try {
+            String ddl = "CREATE TABLE "+ tableName + 
+                    "  (a_string varchar not null, a_binary varbinary not null, col1 integer" +
+                    "  CONSTRAINT pk PRIMARY KEY (a_string, a_binary))\n";
+            createTestTable(getUrl(), ddl);
+            final PhoenixPigConfiguration configuration = newConfiguration (tableName);
+            final String selectStatement = configuration.getSelectStatement();
+            final String expectedSelectStatement = "SELECT \"A_STRING\",\"A_BINARY\",\"0\".\"COL1\" FROM " + SchemaUtil.getEscapedArgument(tableName) ; 
+            assertEquals(expectedSelectStatement, selectStatement);
+        } finally {
+            conn.close();
+        }
+    }
+    
+    @Test
+    public void testSelectStatementForSpecificColumns() throws Exception {
+        Properties props = new Properties(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        conn.setAutoCommit(false);
+        final String tableName = "TEST_TABLE";
+        try {
+            String ddl = "CREATE TABLE "+ tableName + 
+                    "  (a_string varchar not null, a_binary varbinary not null, col1 integer" +
+                    "  CONSTRAINT pk PRIMARY KEY (a_string, a_binary))\n";
+            createTestTable(getUrl(), ddl);
+            final PhoenixPigConfiguration configuration = newConfiguration (tableName);
+            configuration.setSelectColumns("a_binary");
+            final String selectStatement = configuration.getSelectStatement();
+            final String expectedSelectStatement = "SELECT \"A_BINARY\" FROM " + SchemaUtil.getEscapedArgument(tableName) ; 
+            assertEquals(expectedSelectStatement, selectStatement);
+        } finally {
+            conn.close();
+        }
+    }
+
+    private PhoenixPigConfiguration newConfiguration(String tableName) {
+        final Configuration configuration = new Configuration();
+        final PhoenixPigConfiguration phoenixConfiguration = new PhoenixPigConfiguration(configuration);
+        phoenixConfiguration.configure(zkQuorum, tableName.toUpperCase(), 100);
+        return phoenixConfiguration;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/a615cebf/phoenix-pig/src/main/java/org/apache/phoenix/pig/PhoenixHBaseLoader.java
----------------------------------------------------------------------
diff --git a/phoenix-pig/src/main/java/org/apache/phoenix/pig/PhoenixHBaseLoader.java b/phoenix-pig/src/main/java/org/apache/phoenix/pig/PhoenixHBaseLoader.java
new file mode 100644
index 0000000..1f81ad6
--- /dev/null
+++ b/phoenix-pig/src/main/java/org/apache/phoenix/pig/PhoenixHBaseLoader.java
@@ -0,0 +1,253 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ *distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you maynot use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicablelaw or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.pig;
+
+import static org.apache.commons.lang.StringUtils.isEmpty;
+
+import java.io.IOException;
+import java.util.Properties;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.phoenix.pig.hadoop.PhoenixInputFormat;
+import org.apache.phoenix.pig.hadoop.PhoenixRecord;
+import org.apache.phoenix.pig.util.PhoenixPigSchemaUtil;
+import org.apache.phoenix.pig.util.QuerySchemaParserFunction;
+import org.apache.phoenix.pig.util.TableSchemaParserFunction;
+import org.apache.phoenix.pig.util.TypeUtil;
+import org.apache.pig.Expression;
+import org.apache.pig.LoadFunc;
+import org.apache.pig.LoadMetadata;
+import org.apache.pig.PigException;
+import org.apache.pig.ResourceSchema;
+import org.apache.pig.ResourceStatistics;
+import org.apache.pig.backend.executionengine.ExecException;
+import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigSplit;
+import org.apache.pig.data.Tuple;
+import org.apache.pig.impl.util.ObjectSerializer;
+import org.apache.pig.impl.util.UDFContext;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * LoadFunc to load data from HBase using Phoenix .
+ * 
+ * Example usage: 
+ * a) TABLE
+ *   i)   A = load 'hbase://table/HIRES'  using
+ * org.apache.phoenix.pig.PhoenixHBaseLoader('localhost');
+ *               
+ *       The above loads the data from a table 'HIRES'
+ *       
+ *   ii)  A = load 'hbase://table/HIRES/id,name' using
+ *       org.apache.phoenix.pig.PhoenixHBaseLoader('localhost');
+ *       
+ *       Here, only id, name are returned from the table HIRES as part of LOAD.
+ * 
+ * b)  QUERY
+ *   i)   B = load 'hbase://query/SELECT fname, lname FROM HIRES' using
+ *             org.apache.phoenix.pig.PhoenixHBaseLoader('localhost');
+ *       
+ *        The above loads fname and lname columns from 'HIRES' table.
+ * 
+ */
+public final class PhoenixHBaseLoader extends LoadFunc implements LoadMetadata {
+
+    private static final Log LOG = LogFactory.getLog(PhoenixHBaseLoader.class);
+    private static final String PHOENIX_TABLE_NAME_SCHEME = "hbase://table/";
+    private static final String PHOENIX_QUERY_SCHEME      = "hbase://query/";
+    private static final String RESOURCE_SCHEMA_SIGNATURE = "phoenix.pig.schema";
+   
+    private PhoenixPigConfiguration config;
+    private String tableName;
+    private String selectQuery;
+    private String zkQuorum ;
+    private PhoenixInputFormat inputFormat;
+    private RecordReader<NullWritable, PhoenixRecord> reader;
+    private String contextSignature;
+    private ResourceSchema schema;
+       
+    /**
+     * @param zkQuorum
+     */
+    public PhoenixHBaseLoader(String zkQuorum) {
+        super();
+        Preconditions.checkNotNull(zkQuorum);
+        Preconditions.checkState(zkQuorum.length() > 0, "Zookeeper quorum cannot be empty!");
+        this.zkQuorum = zkQuorum;
+    }
+    
+    @Override
+    public void setLocation(String location, Job job) throws IOException {        
+        final Configuration configuration = job.getConfiguration();
+        //explicitly turning off combining splits. 
+        configuration.setBoolean("pig.noSplitCombination", true);
+        this.initializePhoenixPigConfiguration(location, configuration);
+    }
+
+    /**
+     * Initialize PhoenixPigConfiguration if it is null. Called by {@link #setLocation} and {@link #getSchema}
+     * @param location
+     * @param configuration
+     * @throws PigException
+     */
+    private void initializePhoenixPigConfiguration(final String location, final Configuration configuration) throws PigException {
+        if(this.config != null) {
+            return;
+        }
+        this.config = new PhoenixPigConfiguration(configuration);
+        this.config.setServerName(this.zkQuorum);
+        Pair<String,String> pair = null;
+        try {
+            if (location.startsWith(PHOENIX_TABLE_NAME_SCHEME)) {
+                String tableSchema = location.substring(PHOENIX_TABLE_NAME_SCHEME.length());
+                final TableSchemaParserFunction parseFunction = new TableSchemaParserFunction();
+                pair =  parseFunction.apply(tableSchema);
+             } else if (location.startsWith(PHOENIX_QUERY_SCHEME)) {
+                this.selectQuery = location.substring(PHOENIX_QUERY_SCHEME.length());
+                final QuerySchemaParserFunction queryParseFunction = new QuerySchemaParserFunction(this.config);
+                pair = queryParseFunction.apply(this.selectQuery);
+                config.setSelectStatement(this.selectQuery);
+            }
+            this.tableName = pair.getFirst();
+            final String selectedColumns = pair.getSecond();
+            
+            if(isEmpty(this.tableName) && isEmpty(this.selectQuery)) {
+                printUsage(location);
+            }
+            this.config.setTableName(this.tableName);
+            if(!isEmpty(selectedColumns)) {
+                this.config.setSelectColumns(selectedColumns);    
+            }
+        } catch(IllegalArgumentException iae) {
+            printUsage(location);
+        } 
+    }
+
+  
+    @Override
+    public String relativeToAbsolutePath(String location, Path curDir) throws IOException {
+        return location;
+    }
+
+    @Override
+    public InputFormat getInputFormat() throws IOException {
+        if(inputFormat == null) {
+            inputFormat = new PhoenixInputFormat();
+        }
+        return inputFormat;
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public void prepareToRead(RecordReader reader, PigSplit split) throws IOException {
+        this.reader = reader;
+        final String resourceSchemaAsStr = getValueFromUDFContext(this.contextSignature,RESOURCE_SCHEMA_SIGNATURE);
+        if (resourceSchemaAsStr == null) {
+            throw new IOException("Could not find schema in UDF context");
+        }
+       schema = (ResourceSchema)ObjectSerializer.deserialize(resourceSchemaAsStr); 
+    }
+
+     /*
+     * @see org.apache.pig.LoadFunc#setUDFContextSignature(java.lang.String)
+     */
+    @Override
+    public void setUDFContextSignature(String signature) {
+        this.contextSignature = signature;
+    }
+    
+    @Override
+    public Tuple getNext() throws IOException {
+        try {
+            if(!reader.nextKeyValue()) {
+               return null; 
+            }
+            final PhoenixRecord phoenixRecord = reader.getCurrentValue();
+            if(phoenixRecord == null) {
+                return null;
+            }
+            final Tuple tuple = TypeUtil.transformToTuple(phoenixRecord,schema.getFields());
+            return tuple;
+       } catch (InterruptedException e) {
+            int errCode = 6018;
+            final String errMsg = "Error while reading input";
+            throw new ExecException(errMsg, errCode,PigException.REMOTE_ENVIRONMENT, e);
+        }
+    }
+    
+    private void printUsage(final String location) throws PigException {
+        String locationErrMsg = String.format("The input location in load statement should be of the form " +
+                "%s<table name> or %s<query>. Got [%s] ",PHOENIX_TABLE_NAME_SCHEME,PHOENIX_QUERY_SCHEME,location);
+        LOG.error(locationErrMsg);
+        throw new PigException(locationErrMsg);
+    }
+    
+    @Override
+    public ResourceSchema getSchema(String location, Job job) throws IOException {
+        if(schema != null) {
+            return schema;
+        }
+        final Configuration configuration = job.getConfiguration();
+        this.initializePhoenixPigConfiguration(location, configuration);
+        this.schema = PhoenixPigSchemaUtil.getResourceSchema(this.config);
+        if(LOG.isDebugEnabled()) {
+            LOG.debug(String.format("Resource Schema generated for location [%s] is [%s]", location, schema.toString()));
+        }
+        this.storeInUDFContext(this.contextSignature, RESOURCE_SCHEMA_SIGNATURE, ObjectSerializer.serialize(schema));
+        return schema;
+    }
+
+    @Override
+    public ResourceStatistics getStatistics(String location, Job job) throws IOException {
+       // not implemented
+        return null;
+    }
+
+    @Override
+    public String[] getPartitionKeys(String location, Job job) throws IOException {
+     // not implemented
+        return null;
+    }
+
+    @Override
+    public void setPartitionFilter(Expression partitionFilter) throws IOException {
+     // not implemented
+    }
+ 
+    private void storeInUDFContext(final String signature,final String key,final String value) {
+        final UDFContext udfContext = UDFContext.getUDFContext();
+        final Properties props = udfContext.getUDFProperties(this.getClass(), new String[]{signature});
+        props.put(key, value);
+    }
+    
+    private String getValueFromUDFContext(final String signature,final String key) {
+        final UDFContext udfContext = UDFContext.getUDFContext();
+        final Properties props = udfContext.getUDFProperties(this.getClass(), new String[]{signature});
+        return props.getProperty(key);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/a615cebf/phoenix-pig/src/main/java/org/apache/phoenix/pig/hadoop/PhoenixInputFormat.java
----------------------------------------------------------------------
diff --git a/phoenix-pig/src/main/java/org/apache/phoenix/pig/hadoop/PhoenixInputFormat.java b/phoenix-pig/src/main/java/org/apache/phoenix/pig/hadoop/PhoenixInputFormat.java
new file mode 100644
index 0000000..ebb9023
--- /dev/null
+++ b/phoenix-pig/src/main/java/org/apache/phoenix/pig/hadoop/PhoenixInputFormat.java
@@ -0,0 +1,167 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ *distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you maynot use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicablelaw or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.pig.hadoop;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.phoenix.compile.QueryPlan;
+import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.iterate.ResultIterator;
+import org.apache.phoenix.jdbc.PhoenixStatement;
+import org.apache.phoenix.pig.PhoenixPigConfiguration;
+import org.apache.phoenix.query.KeyRange;
+import org.apache.phoenix.schema.SaltingUtil;
+import org.apache.phoenix.schema.TableRef;
+import org.apache.phoenix.util.ScanUtil;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Throwables;
+import com.google.common.collect.Lists;
+
+/**
+ * The InputFormat class for generating the splits and creating the record readers.
+ * 
+ */
+public final class PhoenixInputFormat extends InputFormat<NullWritable, PhoenixRecord> {
+
+    private static final Log LOG = LogFactory.getLog(PhoenixInputFormat.class);
+    private PhoenixPigConfiguration phoenixConfiguration;
+    private Connection connection;
+    private QueryPlan  queryPlan;
+    
+    /**
+     * instantiated by framework
+     */
+    public PhoenixInputFormat() {
+    }
+
+    @Override
+    public RecordReader<NullWritable, PhoenixRecord> createRecordReader(InputSplit split, TaskAttemptContext context)
+            throws IOException, InterruptedException {       
+        setConf(context.getConfiguration());
+        final QueryPlan queryPlan = getQueryPlan(context);
+        try {
+            return new PhoenixRecordReader(phoenixConfiguration,queryPlan);    
+        }catch(SQLException sqle) {
+            throw new IOException(sqle);
+        }
+    }
+    
+   
+
+    @Override
+    public List<InputSplit> getSplits(JobContext context) throws IOException, InterruptedException {        
+        List<InputSplit> splits = null;
+        try{
+            setConf(context.getConfiguration());
+            final QueryPlan queryPlan = getQueryPlan(context);
+            @SuppressWarnings("unused")
+            final ResultIterator iterator = queryPlan.iterator();
+            final List<KeyRange> allSplits = queryPlan.getSplits();
+            splits = generateSplits(queryPlan,allSplits);
+        } catch(SQLException sqlE) {
+            LOG.error(String.format(" Error [%s] in getSplits of PhoenixInputFormat ", sqlE.getMessage()));
+            Throwables.propagate(sqlE);
+        }
+        return splits;
+    }
+
+    private List<InputSplit> generateSplits(final QueryPlan qplan, final List<KeyRange> splits) throws IOException {
+        Preconditions.checkNotNull(qplan);
+        Preconditions.checkNotNull(splits);
+        final List<InputSplit> psplits = Lists.newArrayListWithExpectedSize(splits.size());
+        final StatementContext context = qplan.getContext();
+        final TableRef tableRef = qplan.getTableRef();
+        for (KeyRange split : splits) {
+            final Scan splitScan = new Scan(context.getScan());
+            if (tableRef.getTable().getBucketNum() != null) {
+                KeyRange minMaxRange = context.getMinMaxRange();
+                if (minMaxRange != null) {
+                    minMaxRange = SaltingUtil.addSaltByte(split.getLowerRange(), minMaxRange);
+                    split = split.intersect(minMaxRange);
+                }
+            }
+            // as the intersect code sets the actual start and stop row within the passed splitScan, we are fetching it back below.
+            if (ScanUtil.intersectScanRange(splitScan, split.getLowerRange(), split.getUpperRange(), context.getScanRanges().useSkipScanFilter())) {
+                final PhoenixInputSplit inputSplit = new PhoenixInputSplit(KeyRange.getKeyRange(splitScan.getStartRow(), splitScan.getStopRow()));
+                psplits.add(inputSplit);     
+            }
+        }
+        return psplits;
+    }
+    
+    public void setConf(Configuration configuration) {
+        this.phoenixConfiguration = new PhoenixPigConfiguration(configuration);
+    }
+
+    public PhoenixPigConfiguration getConf() {
+        return this.phoenixConfiguration;
+    }
+    
+    private Connection getConnection() {
+        try {
+            if (this.connection == null) {
+                this.connection = phoenixConfiguration.getConnection();
+           }
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+        return connection;
+    }
+    
+    /**
+     * Returns the query plan associated with the select query.
+     * @param context
+     * @return
+     * @throws IOException
+     * @throws SQLException
+     */
+    private QueryPlan getQueryPlan(final JobContext context) throws IOException {
+        Preconditions.checkNotNull(context);
+        if(queryPlan == null) {
+            try{
+                final Connection connection = getConnection();
+                final String selectStatement = getConf().getSelectStatement();
+                Preconditions.checkNotNull(selectStatement);
+                final Statement statement = connection.createStatement();
+                final PhoenixStatement pstmt = statement.unwrap(PhoenixStatement.class);
+                this.queryPlan = pstmt.compileQuery(selectStatement);
+            } catch(Exception exception) {
+                LOG.error(String.format("Failed to get the query plan with error [%s]",exception.getMessage()));
+                throw new RuntimeException(exception);
+            }
+        }
+        return queryPlan;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/a615cebf/phoenix-pig/src/main/java/org/apache/phoenix/pig/hadoop/PhoenixInputSplit.java
----------------------------------------------------------------------
diff --git a/phoenix-pig/src/main/java/org/apache/phoenix/pig/hadoop/PhoenixInputSplit.java b/phoenix-pig/src/main/java/org/apache/phoenix/pig/hadoop/PhoenixInputSplit.java
new file mode 100644
index 0000000..43d69b3
--- /dev/null
+++ b/phoenix-pig/src/main/java/org/apache/phoenix/pig/hadoop/PhoenixInputSplit.java
@@ -0,0 +1,104 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ *distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you maynot use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicablelaw or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.pig.hadoop;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.phoenix.query.KeyRange;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * 
+ * Input split class to hold the lower and upper bound range. {@link KeyRange}
+ * 
+ */
+public class PhoenixInputSplit extends InputSplit implements Writable {
+
+    private KeyRange keyRange;
+   
+    /**
+     * No Arg constructor
+     */
+    public PhoenixInputSplit() {
+    }
+    
+   /**
+    * 
+    * @param keyRange
+    */
+    public PhoenixInputSplit(final KeyRange keyRange) {
+        Preconditions.checkNotNull(keyRange);
+        this.keyRange = keyRange;
+    }
+    
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        this.keyRange = new KeyRange ();
+        this.keyRange.readFields(input);
+    }
+    
+    @Override
+    public void write(DataOutput output) throws IOException {
+        Preconditions.checkNotNull(keyRange);
+        keyRange.write(output);
+    }
+
+    @Override
+    public long getLength() throws IOException, InterruptedException {
+         return 0;
+    }
+
+    @Override
+    public String[] getLocations() throws IOException, InterruptedException {
+        return new String[]{};
+    }
+
+    /**
+     * @return Returns the keyRange.
+     */
+    public KeyRange getKeyRange() {
+        return keyRange;
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + ((keyRange == null) ? 0 : keyRange.hashCode());
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj) { return true; }
+        if (obj == null) { return false; }
+        if (!(obj instanceof PhoenixInputSplit)) { return false; }
+        PhoenixInputSplit other = (PhoenixInputSplit)obj;
+        if (keyRange == null) {
+            if (other.keyRange != null) { return false; }
+        } else if (!keyRange.equals(other.keyRange)) { return false; }
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/a615cebf/phoenix-pig/src/main/java/org/apache/phoenix/pig/hadoop/PhoenixRecordReader.java
----------------------------------------------------------------------
diff --git a/phoenix-pig/src/main/java/org/apache/phoenix/pig/hadoop/PhoenixRecordReader.java b/phoenix-pig/src/main/java/org/apache/phoenix/pig/hadoop/PhoenixRecordReader.java
new file mode 100644
index 0000000..24ad1ee
--- /dev/null
+++ b/phoenix-pig/src/main/java/org/apache/phoenix/pig/hadoop/PhoenixRecordReader.java
@@ -0,0 +1,133 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ *distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you maynot use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicablelaw or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.pig.hadoop;
+
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.phoenix.compile.QueryPlan;
+import org.apache.phoenix.iterate.ResultIterator;
+import org.apache.phoenix.iterate.TableResultIterator;
+import org.apache.phoenix.jdbc.PhoenixResultSet;
+import org.apache.phoenix.pig.PhoenixPigConfiguration;
+import org.apache.phoenix.query.KeyRange;
+import org.apache.phoenix.util.ColumnInfo;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Throwables;
+
+/**
+ * RecordReader that process the scan and returns PhoenixRecord
+ * 
+ */
+public final class PhoenixRecordReader extends RecordReader<NullWritable,PhoenixRecord>{
+    
+    private static final Log LOG = LogFactory.getLog(PhoenixRecordReader.class);
+    private final PhoenixPigConfiguration phoenixConfiguration;
+    private final QueryPlan queryPlan;
+    private final List<ColumnInfo> columnInfos;
+    private NullWritable key =  NullWritable.get();
+    private PhoenixRecord value = null;
+    private ResultIterator resultIterator = null;
+    private PhoenixResultSet resultSet;
+    
+    public PhoenixRecordReader(final PhoenixPigConfiguration pConfiguration,final QueryPlan qPlan) throws SQLException {
+        
+        Preconditions.checkNotNull(pConfiguration);
+        Preconditions.checkNotNull(qPlan);
+        this.phoenixConfiguration = pConfiguration;
+        this.queryPlan = qPlan;
+        this.columnInfos = phoenixConfiguration.getSelectColumnMetadataList();
+     }
+
+    @Override
+    public void close() throws IOException {
+       if(resultIterator != null) {
+           try {
+               resultIterator.close();
+        } catch (SQLException e) {
+           LOG.error(" Error closing resultset.");
+        }
+       }
+    }
+
+    @Override
+    public NullWritable getCurrentKey() throws IOException, InterruptedException {
+        return key;
+    }
+
+    @Override
+    public PhoenixRecord getCurrentValue() throws IOException, InterruptedException {
+        return value;
+    }
+
+    @Override
+    public float getProgress() throws IOException, InterruptedException {
+        return 0;
+    }
+
+    @Override
+    public void initialize(InputSplit split, TaskAttemptContext context) throws IOException, InterruptedException {
+        final PhoenixInputSplit pSplit = (PhoenixInputSplit)split;
+        final KeyRange keyRange = pSplit.getKeyRange();
+        final Scan splitScan = queryPlan.getContext().getScan();
+        final Scan scan = new Scan(splitScan);
+        scan.setStartRow(keyRange.getLowerRange());
+        scan.setStopRow(keyRange.getUpperRange());
+         try {
+            this.resultIterator = new TableResultIterator(queryPlan.getContext(), queryPlan.getTableRef(),scan);
+            this.resultSet = new PhoenixResultSet(this.resultIterator, queryPlan.getProjector(),queryPlan.getContext().getStatement());
+        } catch (SQLException e) {
+            LOG.error(String.format(" Error [%s] initializing PhoenixRecordReader. ",e.getMessage()));
+            Throwables.propagate(e);
+        }
+        
+   }
+
+    @Override
+    public boolean nextKeyValue() throws IOException, InterruptedException {
+        if (key == null) {
+            key = NullWritable.get();
+        }
+        if (value == null) {
+            value =  new PhoenixRecord();
+        }
+        Preconditions.checkNotNull(this.resultSet);
+        try {
+            if(!resultSet.next()) {
+                return false;
+            }
+            value.read(resultSet,columnInfos.size());
+            return true;
+        } catch (SQLException e) {
+            LOG.error(String.format(" Error [%s] occurred while iterating over the resultset. ",e.getMessage()));
+            Throwables.propagate(e);
+        }
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/a615cebf/phoenix-pig/src/main/java/org/apache/phoenix/pig/util/ColumnInfoToStringEncoderDecoder.java
----------------------------------------------------------------------
diff --git a/phoenix-pig/src/main/java/org/apache/phoenix/pig/util/ColumnInfoToStringEncoderDecoder.java b/phoenix-pig/src/main/java/org/apache/phoenix/pig/util/ColumnInfoToStringEncoderDecoder.java
new file mode 100644
index 0000000..3ea9b5b
--- /dev/null
+++ b/phoenix-pig/src/main/java/org/apache/phoenix/pig/util/ColumnInfoToStringEncoderDecoder.java
@@ -0,0 +1,69 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ *distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you maynot use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicablelaw or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.pig.util;
+
+import java.util.List;
+
+import org.apache.phoenix.util.ColumnInfo;
+
+import com.google.common.base.Function;
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Splitter;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+
+/**
+ * 
+ * A codec to transform a {@link ColumnInfo} to a {@link String} and decode back.
+ *
+ */
+public final class ColumnInfoToStringEncoderDecoder {
+
+    private static final String COLUMN_INFO_DELIMITER = "|";
+    
+    private ColumnInfoToStringEncoderDecoder() {
+        
+    }
+    
+    public static String encode(List<ColumnInfo> columnInfos) {
+        Preconditions.checkNotNull(columnInfos);
+        return Joiner.on(COLUMN_INFO_DELIMITER).
+                        skipNulls().join(columnInfos);
+    }
+    
+    public static List<ColumnInfo> decode(final String columnInfoStr) {
+        Preconditions.checkNotNull(columnInfoStr);
+        List<ColumnInfo> columnInfos = Lists.newArrayList(
+                                Iterables.transform(
+                                        Splitter.on(COLUMN_INFO_DELIMITER).omitEmptyStrings().split(columnInfoStr),
+                                        new Function<String, ColumnInfo>() {
+                                            @Override
+                                            public ColumnInfo apply(String colInfo) {
+                                                if (colInfo.isEmpty()) {
+                                                      return null;
+                                                }
+                                                return ColumnInfo.fromString(colInfo);
+                                            }
+                                        }));
+        return columnInfos;
+        
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/a615cebf/phoenix-pig/src/main/java/org/apache/phoenix/pig/util/PhoenixPigSchemaUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-pig/src/main/java/org/apache/phoenix/pig/util/PhoenixPigSchemaUtil.java b/phoenix-pig/src/main/java/org/apache/phoenix/pig/util/PhoenixPigSchemaUtil.java
new file mode 100644
index 0000000..695b506
--- /dev/null
+++ b/phoenix-pig/src/main/java/org/apache/phoenix/pig/util/PhoenixPigSchemaUtil.java
@@ -0,0 +1,70 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ *distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you maynot use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicablelaw or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.pig.util;
+
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.phoenix.pig.PhoenixPigConfiguration;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.phoenix.util.ColumnInfo;
+import org.apache.pig.ResourceSchema;
+import org.apache.pig.ResourceSchema.ResourceFieldSchema;
+
+/**
+ * 
+ * Utility to generate the ResourceSchema from the list of {@link ColumnInfo}
+ *
+ */
+public final class PhoenixPigSchemaUtil {
+
+    private static final Log LOG = LogFactory.getLog(PhoenixPigSchemaUtil.class);
+    
+    private PhoenixPigSchemaUtil() {
+    }
+    
+    public static ResourceSchema getResourceSchema(final PhoenixPigConfiguration phoenixConfiguration) throws IOException {
+        
+        final ResourceSchema schema = new ResourceSchema();
+        try {
+            final List<ColumnInfo> columns = phoenixConfiguration.getSelectColumnMetadataList();
+            ResourceFieldSchema fields[] = new ResourceFieldSchema[columns.size()];
+            int i = 0;
+            for(ColumnInfo cinfo : columns) {
+                int sqlType = cinfo.getSqlType();
+                PDataType phoenixDataType = PDataType.fromTypeId(sqlType);
+                byte pigType = TypeUtil.getPigDataTypeForPhoenixType(phoenixDataType);
+                ResourceFieldSchema field = new ResourceFieldSchema();
+                field.setType(pigType).setName(cinfo.getDisplayName());
+                fields[i++] = field;
+            }
+            schema.setFields(fields);    
+        } catch(SQLException sqle) {
+            LOG.error(String.format("Error: SQLException [%s] ",sqle.getMessage()));
+            throw new IOException(sqle);
+        }
+        
+        return schema;
+        
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/a615cebf/phoenix-pig/src/main/java/org/apache/phoenix/pig/util/QuerySchemaParserFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-pig/src/main/java/org/apache/phoenix/pig/util/QuerySchemaParserFunction.java b/phoenix-pig/src/main/java/org/apache/phoenix/pig/util/QuerySchemaParserFunction.java
new file mode 100644
index 0000000..1b3a90a
--- /dev/null
+++ b/phoenix-pig/src/main/java/org/apache/phoenix/pig/util/QuerySchemaParserFunction.java
@@ -0,0 +1,121 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ *distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you maynot use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicablelaw or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.pig.util;
+
+import java.sql.Connection;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.phoenix.compile.ColumnProjector;
+import org.apache.phoenix.compile.QueryPlan;
+import org.apache.phoenix.jdbc.PhoenixStatement;
+import org.apache.phoenix.pig.PhoenixPigConfiguration;
+
+import com.google.common.base.Function;
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Throwables;
+import com.google.common.collect.Lists;
+
+/**
+ * 
+ *  A function to parse the select query passed to LOAD into a Pair of <table Name, List<columns>
+ *
+ */
+public class QuerySchemaParserFunction implements Function<String,Pair<String,String>> {
+
+    private static final Log LOG = LogFactory.getLog(QuerySchemaParserFunction.class);
+    private PhoenixPigConfiguration phoenixConfiguration;
+    
+    public QuerySchemaParserFunction(PhoenixPigConfiguration phoenixConfiguration) {
+        Preconditions.checkNotNull(phoenixConfiguration);
+        this.phoenixConfiguration = phoenixConfiguration;
+    }
+    
+    @Override
+    public Pair<String, String> apply(final String selectStatement) {
+        Preconditions.checkNotNull(selectStatement);
+        Preconditions.checkArgument(!selectStatement.isEmpty(), "Select Query is empty!!");
+        Preconditions.checkNotNull(this.phoenixConfiguration);
+        Connection connection = null;
+        try {
+            connection = this.phoenixConfiguration.getConnection();
+            final Statement  statement = connection.createStatement();
+            final PhoenixStatement pstmt = statement.unwrap(PhoenixStatement.class);
+            final QueryPlan queryPlan = pstmt.compileQuery(selectStatement);
+            isValidStatement(queryPlan);
+            final String tableName = queryPlan.getTableRef().getTable().getName().getString();
+            final List<? extends ColumnProjector> projectedColumns = queryPlan.getProjector().getColumnProjectors();
+            final List<String> columns = Lists.transform(projectedColumns,
+                                                            new Function<ColumnProjector,String>() {
+                                                                @Override
+                                                                public String apply(ColumnProjector column) {
+                                                                    return column.getName();
+                                                                }
+                                                            });
+            final String columnsAsStr = Joiner.on(",").join(columns);
+            return new Pair<String, String>(tableName, columnsAsStr);
+        } catch (SQLException e) {
+            LOG.error(String.format(" Error [%s] parsing SELECT query [%s] ",e.getMessage(),selectStatement));
+            Throwables.propagate(e);
+        } finally {
+            if(connection != null) {
+                try {
+                    connection.close();
+                } catch(SQLException sqle) {
+                    Throwables.propagate(sqle);
+                }
+            }
+        }
+        return null;
+    }
+    
+    /**
+     * The method validates the statement passed to the query plan. List of conditions are
+     * <ol>
+     *   <li>Is a SELECT statement</li>
+     *   <li>doesn't contain ORDER BY expression</li>
+     *   <li>doesn't contain LIMIT</li>
+     *   <li>doesn't contain GROUP BY expression</li>
+     *   <li>doesn't contain DISTINCT</li>
+     *   <li>doesn't contain AGGREGATE functions</li>
+     * </ol>  
+     * @param queryPlan
+     * @return
+     */
+    private boolean isValidStatement(final QueryPlan queryPlan) {
+        if(queryPlan.getStatement().getOperation() != PhoenixStatement.Operation.QUERY) {
+            throw new IllegalArgumentException("Query passed isn't a SELECT statement");
+        }
+        if(!queryPlan.getOrderBy().getOrderByExpressions().isEmpty() 
+                || queryPlan.getLimit() != null 
+                || (queryPlan.getGroupBy() != null && !queryPlan.getGroupBy().isEmpty()) 
+                || queryPlan.getStatement().isDistinct()
+                || queryPlan.getStatement().isAggregate()) {
+            throw new IllegalArgumentException("SELECT statement shouldn't contain DISTINCT or ORDER BY or LIMIT or GROUP BY expressions");
+        }
+        return true;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/a615cebf/phoenix-pig/src/main/java/org/apache/phoenix/pig/util/TableSchemaParserFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-pig/src/main/java/org/apache/phoenix/pig/util/TableSchemaParserFunction.java b/phoenix-pig/src/main/java/org/apache/phoenix/pig/util/TableSchemaParserFunction.java
new file mode 100644
index 0000000..5bca30e
--- /dev/null
+++ b/phoenix-pig/src/main/java/org/apache/phoenix/pig/util/TableSchemaParserFunction.java
@@ -0,0 +1,54 @@
+/*
+ * Copyright 2010 The Apache Software Foundation
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ *distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you maynot use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicablelaw or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.pig.util;
+
+
+
+import org.apache.hadoop.hbase.util.Pair;
+
+import com.google.common.base.Function;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Splitter;
+import com.google.common.collect.Iterables;
+
+/**
+ * 
+ * A function to parse the table schema passed to LOAD/STORE into a Pair of <table Name, columns>
+ *
+ */
+public final class TableSchemaParserFunction implements Function<String,Pair<String,String>> {
+
+    private static final char TABLE_COLUMN_DELIMITER    = '/';
+    
+    @Override
+    public Pair<String, String> apply(final String tableSchema) {
+        Preconditions.checkNotNull(tableSchema);
+        Preconditions.checkArgument(!tableSchema.isEmpty(), "HBase Table name is empty!!");
+        
+        final String  tokens[] = Iterables.toArray(Splitter.on(TABLE_COLUMN_DELIMITER).
+                                    trimResults().omitEmptyStrings().split(tableSchema) , String.class); 
+        final String tableName = tokens[0];
+        String columns = null;
+        if(tokens.length > 1) {
+            columns = tokens[1];    
+        }
+        return new Pair<String, String>(tableName, columns);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-phoenix/blob/a615cebf/phoenix-pig/src/main/java/org/apache/phoenix/pig/util/TypeUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-pig/src/main/java/org/apache/phoenix/pig/util/TypeUtil.java b/phoenix-pig/src/main/java/org/apache/phoenix/pig/util/TypeUtil.java
new file mode 100644
index 0000000..f3cacfd
--- /dev/null
+++ b/phoenix-pig/src/main/java/org/apache/phoenix/pig/util/TypeUtil.java
@@ -0,0 +1,312 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.phoenix.pig.util;
+
+import java.io.IOException;
+import java.sql.Date;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.sql.Types;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.phoenix.pig.hadoop.PhoenixRecord;
+import org.apache.phoenix.schema.PDataType;
+import org.apache.pig.PigException;
+import org.apache.pig.ResourceSchema.ResourceFieldSchema;
+import org.apache.pig.builtin.Utf8StorageConverter;
+import org.apache.pig.data.DataByteArray;
+import org.apache.pig.data.DataType;
+import org.apache.pig.data.Tuple;
+import org.apache.pig.data.TupleFactory;
+import org.joda.time.DateTime;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableMap.Builder;
+
+public final class TypeUtil {
+	
+    private static final Log LOG = LogFactory.getLog(TypeUtil.class);
+	private static final Utf8StorageConverter utf8Converter = new Utf8StorageConverter();
+	private static final ImmutableMap<PDataType,Byte> phoenixTypeToPigDataType = init();	
+	
+	private TypeUtil(){
+	}
+	
+	/**
+	 * A map of Phoenix to Pig data types.
+	 * @return
+	 */
+	private static ImmutableMap<PDataType, Byte> init() {
+        final ImmutableMap.Builder<PDataType,Byte> builder = new Builder<PDataType,Byte> ();
+        builder.put(PDataType.LONG,DataType.LONG);
+        builder.put(PDataType.VARBINARY,DataType.BYTEARRAY);
+        builder.put(PDataType.CHAR,DataType.CHARARRAY);
+        builder.put(PDataType.VARCHAR,DataType.CHARARRAY);
+        builder.put(PDataType.DOUBLE,DataType.DOUBLE);
+        builder.put(PDataType.FLOAT,DataType.FLOAT);
+        builder.put(PDataType.INTEGER,DataType.INTEGER);
+        builder.put(PDataType.TINYINT,DataType.INTEGER);
+        builder.put(PDataType.SMALLINT,DataType.INTEGER);
+        builder.put(PDataType.DECIMAL,DataType.BIGDECIMAL);
+        builder.put(PDataType.TIME,DataType.DATETIME);
+        builder.put(PDataType.TIMESTAMP,DataType.DATETIME);
+        builder.put(PDataType.BOOLEAN,DataType.BOOLEAN);
+        builder.put(PDataType.DATE,DataType.DATETIME);
+        builder.put(PDataType.UNSIGNED_DATE,DataType.DATETIME);
+        builder.put(PDataType.UNSIGNED_DOUBLE,DataType.DOUBLE);
+        builder.put(PDataType.UNSIGNED_FLOAT,DataType.FLOAT);
+        builder.put(PDataType.UNSIGNED_INT,DataType.INTEGER);
+        builder.put(PDataType.UNSIGNED_LONG,DataType.LONG);
+        builder.put(PDataType.UNSIGNED_SMALLINT,DataType.INTEGER);
+        builder.put(PDataType.UNSIGNED_TIME,DataType.DATETIME);
+        builder.put(PDataType.UNSIGNED_TIMESTAMP,DataType.DATETIME);
+        builder.put(PDataType.UNSIGNED_TINYINT,DataType.INTEGER);
+        return builder.build();
+    }
+    /**
+	 * This method returns the most appropriate PDataType associated with 
+	 * the incoming Pig type. Note for Pig DataType DATETIME, returns DATE as 
+	 * inferredSqlType. 
+	 * 
+	 * This is later used to make a cast to targetPhoenixType accordingly. See
+	 * {@link #castPigTypeToPhoenix(Object, byte, PDataType)}
+	 * 
+	 * @param obj
+	 * @return PDataType
+	 */
+	public static PDataType getType(Object obj, byte type) {
+		if (obj == null) {
+			return null;
+		}
+	
+		PDataType sqlType;
+
+		switch (type) {
+		case DataType.BYTEARRAY:
+			sqlType = PDataType.VARBINARY;
+			break;
+		case DataType.CHARARRAY:
+			sqlType = PDataType.VARCHAR;
+			break;
+		case DataType.DOUBLE:
+			sqlType = PDataType.DOUBLE;
+			break;
+		case DataType.FLOAT:
+			sqlType = PDataType.FLOAT;
+			break;
+		case DataType.INTEGER:
+			sqlType = PDataType.INTEGER;
+			break;
+		case DataType.LONG:
+			sqlType = PDataType.LONG;
+			break;
+		case DataType.BOOLEAN:
+			sqlType = PDataType.BOOLEAN;
+			break;
+		case DataType.DATETIME:
+			sqlType = PDataType.DATE;
+			break;
+		default:
+			throw new RuntimeException("Unknown type " + obj.getClass().getName()
+					+ " passed to PhoenixHBaseStorage");
+		}
+
+		return sqlType;
+
+	}
+
+	/**
+	 * This method encodes a value with Phoenix data type. It begins
+	 * with checking whether an object is BINARY and makes a call to
+	 * {@link #castBytes(Object, PDataType)} to convery bytes to
+	 * targetPhoenixType
+	 * 
+	 * @param o
+	 * @param targetPhoenixType
+	 * @return Object
+	 */
+	public static Object castPigTypeToPhoenix(Object o, byte objectType, PDataType targetPhoenixType) {
+		PDataType inferredPType = getType(o, objectType);
+		
+		if(inferredPType == null) {
+			return null;
+		}
+		
+		if(inferredPType == PDataType.VARBINARY && targetPhoenixType != PDataType.VARBINARY) {
+			try {
+				o = castBytes(o, targetPhoenixType);
+				inferredPType = getType(o, DataType.findType(o));
+			} catch (IOException e) {
+				throw new RuntimeException("Error while casting bytes for object " +o);
+			}
+		}
+
+		if(inferredPType == PDataType.DATE) {
+			int inferredSqlType = targetPhoenixType.getSqlType();
+
+			if(inferredSqlType == Types.DATE) {
+				return new Date(((DateTime)o).getMillis());
+			} 
+			if(inferredSqlType == Types.TIME) {
+				return new Time(((DateTime)o).getMillis());
+			}
+			if(inferredSqlType == Types.TIMESTAMP) {
+				return new Timestamp(((DateTime)o).getMillis());
+			}
+		}
+		
+		if (targetPhoenixType == inferredPType || inferredPType.isCoercibleTo(targetPhoenixType)) {
+			return inferredPType.toObject(o, targetPhoenixType);
+		}
+		
+		throw new RuntimeException(o.getClass().getName()
+				+ " cannot be coerced to "+targetPhoenixType.toString());
+	}
+	
+	/**
+	 * This method converts bytes to the target type required
+	 * for Phoenix. It uses {@link Utf8StorageConverter} for
+	 * the conversion.
+	 * 
+	 * @param o
+	 * @param targetPhoenixType
+	 * @return Object
+	 * @throws IOException
+	 */
+    public static Object castBytes(Object o, PDataType targetPhoenixType) throws IOException {
+        byte[] bytes = ((DataByteArray)o).get();
+        
+        switch(targetPhoenixType) {
+        case CHAR:
+        case VARCHAR:
+            return utf8Converter.bytesToCharArray(bytes);
+        case UNSIGNED_SMALLINT:
+        case SMALLINT:
+            return utf8Converter.bytesToInteger(bytes).shortValue();
+        case UNSIGNED_TINYINT:
+        case TINYINT:
+            return utf8Converter.bytesToInteger(bytes).byteValue();
+        case UNSIGNED_INT:
+        case INTEGER:
+            return utf8Converter.bytesToInteger(bytes);
+        case BOOLEAN:
+            return utf8Converter.bytesToBoolean(bytes);
+        case DECIMAL:
+            return utf8Converter.bytesToBigDecimal(bytes);
+        case FLOAT:
+        case UNSIGNED_FLOAT:
+            return utf8Converter.bytesToFloat(bytes);
+        case DOUBLE:
+        case UNSIGNED_DOUBLE:
+            return utf8Converter.bytesToDouble(bytes);
+        case UNSIGNED_LONG:
+        case LONG:
+            return utf8Converter.bytesToLong(bytes);
+        case TIME:
+        case TIMESTAMP:
+        case DATE:
+        case UNSIGNED_TIME:
+        case UNSIGNED_TIMESTAMP:
+        case UNSIGNED_DATE:
+        	return utf8Converter.bytesToDateTime(bytes);
+        default:
+        	return o;
+        }        
+    }
+    
+    /**
+     * Transforms the PhoenixRecord to Pig {@link Tuple}.
+     * @param record
+     * @param projectedColumns
+     * @return
+     * @throws IOException
+     */
+    public static Tuple transformToTuple(final PhoenixRecord record, final ResourceFieldSchema[] projectedColumns) throws IOException {
+        
+        List<Object> columnValues = record.getValues();
+        if(columnValues == null || columnValues.size() == 0 || projectedColumns == null || projectedColumns.length != columnValues.size()) {
+            return null;
+        }
+        int columns = columnValues.size();
+        Tuple tuple = TupleFactory.getInstance().newTuple(columns);
+        try {
+            for(int i = 0 ; i < columns ; i++) {
+                final ResourceFieldSchema fieldSchema = projectedColumns[i];
+                Object object = columnValues.get(i);
+                if (object == null) {
+                    tuple.set(i, null);
+                    continue;
+                }
+                
+                switch(fieldSchema.getType()) {
+                    case DataType.BYTEARRAY:
+                        byte[] bytes = PDataType.fromTypeId(PDataType.BINARY.getSqlType()).toBytes(object);
+                        tuple.set(i,new DataByteArray(bytes,0,bytes.length));
+                        break;
+                    case DataType.CHARARRAY:
+                        tuple.set(i,DataType.toString(object));
+                        break;
+                    case DataType.DOUBLE:
+                        tuple.set(i,DataType.toDouble(object));
+                        break;
+                    case DataType.FLOAT:
+                        tuple.set(i,DataType.toFloat(object));
+                        break;
+                    case DataType.INTEGER:
+                        tuple.set(i,DataType.toInteger(object));
+                        break;
+                    case DataType.LONG:
+                        tuple.set(i,DataType.toLong(object));
+                        break;
+                    case DataType.BOOLEAN:
+                        tuple.set(i,DataType.toBoolean(object));
+                        break;
+                    case DataType.DATETIME:
+                        tuple.set(i,DataType.toDateTime(object));
+                        break;
+                    default:
+                        throw new RuntimeException(String.format(" Not supported [%s] pig type" , fieldSchema));
+                }
+            }
+        } catch( Exception ex) {
+            final String errorMsg = String.format(" Error transforming PhoenixRecord to Tuple [%s] ", ex.getMessage());
+            LOG.error(errorMsg);
+            throw new PigException(errorMsg);
+        }
+          return tuple;
+    }
+    
+    /**
+     * Returns the mapping pig data type for a given phoenix data type.
+     * @param phoenixDataType
+     * @return
+     */
+    public static Byte getPigDataTypeForPhoenixType(final PDataType phoenixDataType) {
+        Preconditions.checkNotNull(phoenixDataType);
+        final Byte pigDataType = phoenixTypeToPigDataType.get(phoenixDataType);
+        if(LOG.isDebugEnabled()) {
+            LOG.debug(String.format(" For PhoenixDataType [%s] , pigDataType is [%s] " , phoenixDataType.getSqlTypeName() , DataType.findTypeName(pigDataType)));    
+        }
+        return pigDataType;
+    }
+
+}