You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ma...@apache.org on 2017/01/27 01:18:45 UTC

[01/26] phoenix git commit: PHOENIX-3540 Fix Time data type in Phoenix Spark integration

Repository: phoenix
Updated Branches:
  refs/heads/calcite 948d009f9 -> 405499047


PHOENIX-3540 Fix Time data type in Phoenix Spark integration


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

Branch: refs/heads/calcite
Commit: bd2acd5404ce03fb330a72bbf346546b7f4fbd2b
Parents: 108b78d
Author: Ankit Singhal <an...@gmail.com>
Authored: Thu Dec 22 13:17:20 2016 +0530
Committer: Ankit Singhal <an...@gmail.com>
Committed: Thu Dec 22 13:17:20 2016 +0530

----------------------------------------------------------------------
 phoenix-spark/src/it/resources/globalSetup.sql       |  2 ++
 .../org/apache/phoenix/spark/PhoenixSparkIT.scala    | 12 ++++++++++++
 .../scala/org/apache/phoenix/spark/PhoenixRDD.scala  | 15 ++++++++-------
 3 files changed, 22 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/bd2acd54/phoenix-spark/src/it/resources/globalSetup.sql
----------------------------------------------------------------------
diff --git a/phoenix-spark/src/it/resources/globalSetup.sql b/phoenix-spark/src/it/resources/globalSetup.sql
index 852687e..72f8620 100644
--- a/phoenix-spark/src/it/resources/globalSetup.sql
+++ b/phoenix-spark/src/it/resources/globalSetup.sql
@@ -48,6 +48,8 @@ CREATE TABLE TEST_SMALL_TINY (ID BIGINT NOT NULL PRIMARY KEY, COL1 SMALLINT, COL
 UPSERT INTO TEST_SMALL_TINY VALUES (1, 32767, 127)
 CREATE TABLE DATE_TEST(ID BIGINT NOT NULL PRIMARY KEY, COL1 DATE)
 UPSERT INTO DATE_TEST VALUES(1, CURRENT_DATE())
+CREATE TABLE TIME_TEST(ID BIGINT NOT NULL PRIMARY KEY, COL1 TIME)
+UPSERT INTO TIME_TEST VALUES(1, CURRENT_TIME())
 CREATE TABLE "space" ("key" VARCHAR PRIMARY KEY, "first name" VARCHAR)
 UPSERT INTO "space" VALUES ('key1', 'xyz')
 CREATE TABLE "small" ("key" VARCHAR PRIMARY KEY, "first name" VARCHAR, "salary" INTEGER )

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bd2acd54/phoenix-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkIT.scala
----------------------------------------------------------------------
diff --git a/phoenix-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkIT.scala b/phoenix-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkIT.scala
index 8aeba09..dbcc4f1 100644
--- a/phoenix-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkIT.scala
+++ b/phoenix-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkIT.scala
@@ -621,4 +621,16 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT {
 
     assert(Math.abs(epoch - ts) < 300000)
   }
+
+  test("Can load Phoenix Time columns through DataFrame API") {
+    val sqlContext = new SQLContext(sc)
+    val df = sqlContext.read
+      .format("org.apache.phoenix.spark")
+      .options(Map("table" -> "TIME_TEST", "zkUrl" -> quorumAddress))
+      .load
+    val time = df.select("COL1").first().getTimestamp(0).getTime
+    val epoch = new Date().getTime
+    assert(Math.abs(epoch - time) < 86400000)
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bd2acd54/phoenix-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRDD.scala
----------------------------------------------------------------------
diff --git a/phoenix-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRDD.scala b/phoenix-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRDD.scala
index 505de1b..204a7ef 100644
--- a/phoenix-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRDD.scala
+++ b/phoenix-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRDD.scala
@@ -130,13 +130,14 @@ class PhoenixRDD(sc: SparkContext, table: String, columns: Seq[String],
       val rowSeq = columns.map { case (name, sqlType) =>
         val res = pr.resultMap(name)
 
-        // Special handling for data types
-        if(dateAsTimestamp && sqlType == 91) { // 91 is the defined type for Date
-          new java.sql.Timestamp(res.asInstanceOf[java.sql.Date].getTime)
-        }
-        else {
-          res
-        }
+          // Special handling for data types
+          if (dateAsTimestamp && sqlType == 91) { // 91 is the defined type for Date
+            new java.sql.Timestamp(res.asInstanceOf[java.sql.Date].getTime)
+          } else if (sqlType == 92) { // 92 is the defined type for Time
+            new java.sql.Timestamp(res.asInstanceOf[java.sql.Time].getTime)
+          } else {
+            res
+          }
       }
 
       // Create a Spark Row from the sequence


[10/26] phoenix git commit: PHOENIX-3333 Support Spark 2.0

Posted by ma...@apache.org.
PHOENIX-3333 Support Spark 2.0

Note that the default maven profile will compile for Spark 2.0.2
and Scala 2.11. To switch to the previous behavior, use Maven
profile 'spark16'.


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

Branch: refs/heads/calcite
Commit: a0e5efcec5a1a732b2dce9794251242c3d66eea6
Parents: 07f9273
Author: Josh Mahonin <jm...@gmail.com>
Authored: Tue Jan 3 09:57:46 2017 -0500
Committer: Josh Mahonin <jm...@gmail.com>
Committed: Tue Jan 3 09:57:46 2017 -0500

----------------------------------------------------------------------
 .../apache/phoenix/spark/PhoenixSparkIT.scala   | 26 ++++++++++++++++----
 .../phoenix/spark/DataFrameFunctions.scala      |  6 ++---
 .../org/apache/phoenix/spark/PhoenixRDD.scala   |  6 ++---
 .../phoenix/spark/ProductRDDFunctions.scala     |  3 +--
 pom.xml                                         | 14 ++++++++---
 5 files changed, 39 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/a0e5efce/phoenix-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkIT.scala
----------------------------------------------------------------------
diff --git a/phoenix-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkIT.scala b/phoenix-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkIT.scala
index 9def354..d53b5ee 100644
--- a/phoenix-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkIT.scala
+++ b/phoenix-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkIT.scala
@@ -302,11 +302,21 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT {
 
     // Load TABLE1, save as TABLE1_COPY
     val sqlContext = new SQLContext(sc)
-    val df = sqlContext.load("org.apache.phoenix.spark", Map("table" -> "TABLE1",
-      "zkUrl" -> quorumAddress))
+    val df = sqlContext
+      .read
+      .format("org.apache.phoenix.spark")
+      .option("table", "TABLE1")
+      .option("zkUrl", quorumAddress)
+      .load()
 
     // Save to TABLE21_COPY
-    df.save("org.apache.phoenix.spark", SaveMode.Overwrite, Map("table" -> "TABLE1_COPY", "zkUrl" -> quorumAddress))
+    df
+      .write
+      .format("org.apache.phoenix.spark")
+      .mode(SaveMode.Overwrite)
+      .option("table", "TABLE1_COPY")
+      .option("zkUrl", quorumAddress)
+      .save()
 
     // Verify results
     stmt = conn.createStatement()
@@ -632,12 +642,18 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT {
     val epoch = new Date().getTime
     assert(Math.abs(epoch - time) < 86400000)
   }
+
   test("can read all Phoenix data types") {
     val sqlContext = new SQLContext(sc)
     val df = sqlContext.load("org.apache.phoenix.spark", Map("table" -> "GIGANTIC_TABLE",
       "zkUrl" -> quorumAddress))
-    df.save("org.apache.phoenix.spark",SaveMode.Overwrite, Map("table" -> "OUTPUT_GIGANTIC_TABLE",
-     "zkUrl" -> quorumAddress))
+
+    df.write
+      .format("org.apache.phoenix.spark")
+      .options(Map("table" -> "OUTPUT_GIGANTIC_TABLE", "zkUrl" -> quorumAddress))
+      .mode(SaveMode.Overwrite)
+      .save()
+
     df.count() shouldEqual 1
   }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a0e5efce/phoenix-spark/src/main/scala/org/apache/phoenix/spark/DataFrameFunctions.scala
----------------------------------------------------------------------
diff --git a/phoenix-spark/src/main/scala/org/apache/phoenix/spark/DataFrameFunctions.scala b/phoenix-spark/src/main/scala/org/apache/phoenix/spark/DataFrameFunctions.scala
index bb2efd5..ddf4fab 100644
--- a/phoenix-spark/src/main/scala/org/apache/phoenix/spark/DataFrameFunctions.scala
+++ b/phoenix-spark/src/main/scala/org/apache/phoenix/spark/DataFrameFunctions.scala
@@ -18,12 +18,12 @@ import org.apache.hadoop.io.NullWritable
 import org.apache.phoenix.mapreduce.PhoenixOutputFormat
 import org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil
 import org.apache.phoenix.util.SchemaUtil
-import org.apache.spark.Logging
 import org.apache.spark.sql.DataFrame
 
 import scala.collection.JavaConversions._
 
-class DataFrameFunctions(data: DataFrame) extends Logging with Serializable {
+
+class DataFrameFunctions(data: DataFrame) extends Serializable {
 
   def saveToPhoenix(tableName: String, conf: Configuration = new Configuration,
                     zkUrl: Option[String] = None, tenantId: Option[String] = None): Unit = {
@@ -39,7 +39,7 @@ class DataFrameFunctions(data: DataFrame) extends Logging with Serializable {
     val zkUrlFinal = ConfigurationUtil.getZookeeperURL(outConfig)
 
     // Map the row objects into PhoenixRecordWritable
-    val phxRDD = data.mapPartitions{ rows =>
+    val phxRDD = data.rdd.mapPartitions{ rows =>
  
        // Create a within-partition config to retrieve the ColumnInfo list
        @transient val partitionConfig = ConfigurationUtil.getOutputConfiguration(tableName, fieldArray, zkUrlFinal, tenantId)

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a0e5efce/phoenix-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRDD.scala
----------------------------------------------------------------------
diff --git a/phoenix-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRDD.scala b/phoenix-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRDD.scala
index 63289a0..01a9077 100644
--- a/phoenix-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRDD.scala
+++ b/phoenix-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRDD.scala
@@ -37,7 +37,7 @@ class PhoenixRDD(sc: SparkContext, table: String, columns: Seq[String],
                  @transient conf: Configuration, dateAsTimestamp: Boolean = false,
                  tenantId: Option[String] = None
                 )
-  extends RDD[PhoenixRecordWritable](sc, Nil) with Logging {
+  extends RDD[PhoenixRecordWritable](sc, Nil) {
 
   // Make sure to register the Phoenix driver
   DriverManager.registerDriver(new PhoenixDriver)
@@ -163,7 +163,7 @@ class PhoenixRDD(sc: SparkContext, table: String, columns: Seq[String],
     case t if t.isInstanceOf[PDouble] || t.isInstanceOf[PUnsignedDouble] => DoubleType
     // Use Spark system default precision for now (explicit to work with < 1.5)
     case t if t.isInstanceOf[PDecimal] => 
-      if (columnInfo.getPrecision < 0) DecimalType(38, 18) else DecimalType(columnInfo.getPrecision, columnInfo.getScale)
+      if (columnInfo.getPrecision == null || columnInfo.getPrecision < 0) DecimalType(38, 18) else DecimalType(columnInfo.getPrecision, columnInfo.getScale)
     case t if t.isInstanceOf[PTimestamp] || t.isInstanceOf[PUnsignedTimestamp] => TimestampType
     case t if t.isInstanceOf[PTime] || t.isInstanceOf[PUnsignedTime] => TimestampType
     case t if (t.isInstanceOf[PDate] || t.isInstanceOf[PUnsignedDate]) && !dateAsTimestamp => DateType
@@ -180,7 +180,7 @@ class PhoenixRDD(sc: SparkContext, table: String, columns: Seq[String],
     case t if t.isInstanceOf[PFloatArray] || t.isInstanceOf[PUnsignedFloatArray] => ArrayType(FloatType, containsNull = true)
     case t if t.isInstanceOf[PDoubleArray] || t.isInstanceOf[PUnsignedDoubleArray] => ArrayType(DoubleType, containsNull = true)
     case t if t.isInstanceOf[PDecimalArray] => ArrayType(
-      if (columnInfo.getPrecision < 0) DecimalType(38, 18) else DecimalType(columnInfo.getPrecision, columnInfo.getScale), containsNull = true)
+      if (columnInfo.getPrecision == null || columnInfo.getPrecision < 0) DecimalType(38, 18) else DecimalType(columnInfo.getPrecision, columnInfo.getScale), containsNull = true)
     case t if t.isInstanceOf[PTimestampArray] || t.isInstanceOf[PUnsignedTimestampArray] => ArrayType(TimestampType, containsNull = true)
     case t if t.isInstanceOf[PDateArray] || t.isInstanceOf[PUnsignedDateArray] => ArrayType(TimestampType, containsNull = true)
     case t if t.isInstanceOf[PTimeArray] || t.isInstanceOf[PUnsignedTimeArray] => ArrayType(TimestampType, containsNull = true)

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a0e5efce/phoenix-spark/src/main/scala/org/apache/phoenix/spark/ProductRDDFunctions.scala
----------------------------------------------------------------------
diff --git a/phoenix-spark/src/main/scala/org/apache/phoenix/spark/ProductRDDFunctions.scala b/phoenix-spark/src/main/scala/org/apache/phoenix/spark/ProductRDDFunctions.scala
index b59592b..9b368b6 100644
--- a/phoenix-spark/src/main/scala/org/apache/phoenix/spark/ProductRDDFunctions.scala
+++ b/phoenix-spark/src/main/scala/org/apache/phoenix/spark/ProductRDDFunctions.scala
@@ -17,12 +17,11 @@ import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.io.NullWritable
 import org.apache.phoenix.mapreduce.PhoenixOutputFormat
 import org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil
-import org.apache.spark.Logging
 import org.apache.spark.rdd.RDD
 
 import scala.collection.JavaConversions._
 
-class ProductRDDFunctions[A <: Product](data: RDD[A]) extends Logging with Serializable {
+class ProductRDDFunctions[A <: Product](data: RDD[A]) extends Serializable {
 
   def saveToPhoenix(tableName: String, cols: Seq[String],
                     conf: Configuration = new Configuration, zkUrl: Option[String] = None, tenantId: Option[String] = None)

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a0e5efce/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index f27f2cc..4a0292d 100644
--- a/pom.xml
+++ b/pom.xml
@@ -99,9 +99,9 @@
     <avatica.version>1.9.0</avatica.version>
     <jettyVersion>8.1.7.v20120910</jettyVersion>
     <tephra.version>0.9.0-incubating</tephra.version>
-    <spark.version>1.6.1</spark.version>
-    <scala.version>2.10.4</scala.version>
-    <scala.binary.version>2.10</scala.binary.version>
+    <spark.version>2.0.2</spark.version>
+    <scala.version>2.11.8</scala.version>
+    <scala.binary.version>2.11</scala.binary.version>
 
     <!-- Test Dependencies -->
     <mockito-all.version>1.8.5</mockito-all.version>
@@ -982,5 +982,13 @@
         </plugins>
       </build>
     </profile>
+    <profile>
+      <id>spark16</id>
+      <properties>
+        <spark.version>1.6.1</spark.version>
+        <scala.version>2.10.4</scala.version>
+        <scala.binary.version>2.10</scala.binary.version>
+      </properties>
+    </profile>
   </profiles>
 </project>


[14/26] phoenix git commit: PHOENIX-3584 Expose metrics for ConnectionQueryServices instances and their allocators in the JVM

Posted by ma...@apache.org.
PHOENIX-3584 Expose metrics for ConnectionQueryServices instances and their allocators in the JVM


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

Branch: refs/heads/calcite
Commit: f5de28b6d79141c555c994a94af7b4078872d845
Parents: d8f4594
Author: Samarth <sa...@salesforce.com>
Authored: Tue Jan 10 17:36:42 2017 -0800
Committer: Samarth <sa...@salesforce.com>
Committed: Tue Jan 10 17:36:42 2017 -0800

----------------------------------------------------------------------
 .../phoenix/monitoring/PhoenixMetricsIT.java    | 135 +++++++++++++++++++
 .../phoenix/monitoring/GlobalClientMetrics.java |   6 +-
 .../apache/phoenix/monitoring/MetricType.java   |   4 +-
 .../query/ConnectionQueryServicesImpl.java      |  10 ++
 4 files changed, 153 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/f5de28b6/phoenix-core/src/it/java/org/apache/phoenix/monitoring/PhoenixMetricsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/monitoring/PhoenixMetricsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/monitoring/PhoenixMetricsIT.java
index 3af8ce7..16a66df 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/monitoring/PhoenixMetricsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/monitoring/PhoenixMetricsIT.java
@@ -10,12 +10,14 @@
 package org.apache.phoenix.monitoring;
 
 import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_FAILED_QUERY_COUNTER;
+import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_HCONNECTIONS_COUNTER;
 import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_MUTATION_BATCH_SIZE;
 import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_MUTATION_BYTES;
 import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_MUTATION_COMMIT_TIME;
 import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_MUTATION_SQL_COUNTER;
 import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_NUM_PARALLEL_SCANS;
 import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_OPEN_PHOENIX_CONNECTIONS;
+import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_QUERY_SERVICES_COUNTER;
 import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_QUERY_TIME;
 import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_QUERY_TIMEOUT_COUNTER;
 import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_REJECTED_TASK_COUNTER;
@@ -28,6 +30,7 @@ import static org.apache.phoenix.monitoring.MetricType.MEMORY_CHUNK_BYTES;
 import static org.apache.phoenix.monitoring.MetricType.SCAN_BYTES;
 import static org.apache.phoenix.monitoring.MetricType.TASK_EXECUTED_COUNTER;
 import static org.apache.phoenix.monitoring.MetricType.TASK_EXECUTION_TIME;
+import static org.apache.phoenix.util.PhoenixRuntime.TENANT_ID_ATTRIB;
 import static org.apache.phoenix.util.PhoenixRuntime.UPSERT_BATCH_SIZE_ATTRIB;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
@@ -44,10 +47,16 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Properties;
 import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.phoenix.compile.StatementContext;
 import org.apache.phoenix.end2end.BaseUniqueNamesOwnClusterIT;
 import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixDriver;
 import org.apache.phoenix.jdbc.PhoenixResultSet;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.util.PhoenixRuntime;
@@ -67,6 +76,8 @@ public class PhoenixMetricsIT extends BaseUniqueNamesOwnClusterIT {
             .newArrayList(MetricType.MUTATION_COMMIT_TIME.name());
     private static final List<String> readMetricsToSkip = Lists.newArrayList(MetricType.TASK_QUEUE_WAIT_TIME.name(),
             MetricType.TASK_EXECUTION_TIME.name(), MetricType.TASK_END_TO_END_TIME.name());
+    private static final String CUSTOM_URL_STRING = "SESSION";
+    private static final AtomicInteger numConnections = new AtomicInteger(0); 
 
     @BeforeClass
     public static void doSetup() throws Exception {
@@ -76,6 +87,8 @@ public class PhoenixMetricsIT extends BaseUniqueNamesOwnClusterIT {
         // disable renewing leases as this will force spooling to happen.
         props.put(QueryServices.RENEW_LEASE_ENABLED, String.valueOf(false));
         setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
+        // need the non-test driver for some tests that check number of hconnections, etc.
+        DriverManager.registerDriver(PhoenixDriver.INSTANCE);
     }
 
     @Test
@@ -827,5 +840,127 @@ public class PhoenixMetricsIT extends BaseUniqueNamesOwnClusterIT {
             }
         }
     }
+    
+    @Test
+    public void testGetConnectionsForSameUrlConcurrently()  throws Exception {
+        // establish url and quorum. Need to use PhoenixDriver and not PhoenixTestDriver
+        String zkQuorum = "localhost:" + getUtility().getZkCluster().getClientPort();
+        String url = PhoenixRuntime.JDBC_PROTOCOL + PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR + zkQuorum;
+        ExecutorService exec = Executors.newFixedThreadPool(10);
+        try {
+            GLOBAL_HCONNECTIONS_COUNTER.getMetric().reset();
+            GLOBAL_QUERY_SERVICES_COUNTER.getMetric().reset();
+            assertEquals(0, GLOBAL_HCONNECTIONS_COUNTER.getMetric().getValue());
+            assertEquals(0, GLOBAL_QUERY_SERVICES_COUNTER.getMetric().getValue());
+            List<Callable<Connection>> callables = new ArrayList<>(100);
+            List<Future<Connection>> futures = new ArrayList<>(100);
+            int expectedHConnections = numConnections.get() > 0 ? 0 : 1;
+            for (int i = 1; i <= 100; i++) {
+                Callable<Connection> c = new GetConnectionCallable(url);
+                callables.add(c);
+                futures.add(exec.submit(c));
+            }
+            for (int i = 0; i < futures.size(); i++) {
+                Connection c = futures.get(i).get();
+                try {
+                    c.close();
+                } catch (Exception ignore) {}
+            }
+            assertEquals(expectedHConnections, GLOBAL_HCONNECTIONS_COUNTER.getMetric().getValue());
+            assertEquals(expectedHConnections, GLOBAL_QUERY_SERVICES_COUNTER.getMetric().getValue());
+        } finally {
+            exec.shutdownNow();
+        }
+    }
+    
+    @Test
+    public void testGetConnectionsForDifferentTenantsConcurrently()  throws Exception {
+        // establish url and quorum. Need to use PhoenixDriver and not PhoenixTestDriver
+        String zkQuorum = "localhost:" + getUtility().getZkCluster().getClientPort();
+        String url = PhoenixRuntime.JDBC_PROTOCOL + PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR + zkQuorum;
+        ExecutorService exec = Executors.newFixedThreadPool(10);
+        try {
+            GLOBAL_HCONNECTIONS_COUNTER.getMetric().reset();
+            GLOBAL_QUERY_SERVICES_COUNTER.getMetric().reset();
+            assertEquals(0, GLOBAL_HCONNECTIONS_COUNTER.getMetric().getValue());
+            assertEquals(0, GLOBAL_QUERY_SERVICES_COUNTER.getMetric().getValue());
+            int expectedHConnections = numConnections.get() > 0 ? 0 : 1;
+            List<Callable<Connection>> callables = new ArrayList<>(100);
+            List<Future<Connection>> futures = new ArrayList<>(100);
+            for (int i = 1; i <= 100; i++) {
+                String tenantUrl = url + ';' + TENANT_ID_ATTRIB + '=' + i;
+                Callable<Connection> c = new GetConnectionCallable(tenantUrl + ";");
+                callables.add(c);
+                futures.add(exec.submit(c));
+            }
+            for (int i = 0; i < futures.size(); i++) {
+                Connection c = futures.get(i).get();
+                try {
+                    c.close();
+                } catch (Exception ignore) {}
+            }
+            assertEquals(expectedHConnections, GLOBAL_HCONNECTIONS_COUNTER.getMetric().getValue());
+            assertEquals(expectedHConnections, GLOBAL_QUERY_SERVICES_COUNTER.getMetric().getValue());
+        } finally {
+            exec.shutdownNow();
+        }
+    }
+    
+    @Test
+    public void testGetConnectionsWithDifferentJDBCParamsConcurrently()  throws Exception {
+        DriverManager.registerDriver(PhoenixDriver.INSTANCE);
+        ExecutorService exec = Executors.newFixedThreadPool(4);
+        // establish url and quorum. Need to use PhoenixDriver and not PhoenixTestDriver
+        String zkQuorum = "localhost:" + getUtility().getZkCluster().getClientPort();
+        String baseUrl = PhoenixRuntime.JDBC_PROTOCOL + PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR + zkQuorum;
+        int numConnections = 20;
+        List<Callable<Connection>> callables = new ArrayList<>(numConnections);
+        List<Future<Connection>> futures = new ArrayList<>(numConnections);
+        try {
+            GLOBAL_HCONNECTIONS_COUNTER.getMetric().reset();
+            GLOBAL_QUERY_SERVICES_COUNTER.getMetric().reset();
+            assertEquals(0, GLOBAL_HCONNECTIONS_COUNTER.getMetric().getValue());
+            assertEquals(0, GLOBAL_QUERY_SERVICES_COUNTER.getMetric().getValue());
+            for (int i = 1; i <= numConnections; i++) {
+                String customUrl = baseUrl + ':' +  CUSTOM_URL_STRING + '=' + i;
+                Callable<Connection> c = new GetConnectionCallable(customUrl + ";");
+                callables.add(c);
+                futures.add(exec.submit(c));
+            }
+            for (int i = 0; i < futures.size(); i++) {
+                futures.get(i).get();
+            }
+            assertEquals(numConnections, GLOBAL_HCONNECTIONS_COUNTER.getMetric().getValue());
+            assertEquals(numConnections, GLOBAL_QUERY_SERVICES_COUNTER.getMetric().getValue());
+        } finally {
+            exec.shutdownNow();
+            for (int i = 0; i < futures.size(); i++) {
+                try {
+                    Connection c = futures.get(i).get();
+                    // close the query services instance because we created a lot of HConnections.
+                    c.unwrap(PhoenixConnection.class).getQueryServices().close();
+                    c.close();
+                } catch (Exception ignore) {}
+            }
+        } 
+    }
+    
+    private static class GetConnectionCallable implements Callable<Connection> {
+        private final String url;
+        GetConnectionCallable(String url) {
+            this.url = url;
+        }
+        @Override
+        public Connection call() throws Exception {
+            Connection c = DriverManager.getConnection(url);
+            if (!url.contains(CUSTOM_URL_STRING)) {
+                // check to detect whether a connection was established using the PhoenixDriver
+                // This is used in our tests to figure out whether a new hconnection and query
+                // services will be created.
+                numConnections.incrementAndGet();
+            }
+            return c;
+        }
+    }
 
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f5de28b6/phoenix-core/src/main/java/org/apache/phoenix/monitoring/GlobalClientMetrics.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/monitoring/GlobalClientMetrics.java b/phoenix-core/src/main/java/org/apache/phoenix/monitoring/GlobalClientMetrics.java
index c3a7261..fab4d27 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/monitoring/GlobalClientMetrics.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/monitoring/GlobalClientMetrics.java
@@ -17,6 +17,7 @@
  */
 package org.apache.phoenix.monitoring;
 
+import static org.apache.phoenix.monitoring.MetricType.HCONNECTIONS_COUNTER;
 import static org.apache.phoenix.monitoring.MetricType.OPEN_PHOENIX_CONNECTIONS_COUNTER;
 import static org.apache.phoenix.monitoring.MetricType.QUERY_FAILED_COUNTER;
 import static org.apache.phoenix.monitoring.MetricType.MEMORY_CHUNK_BYTES;
@@ -26,6 +27,7 @@ import static org.apache.phoenix.monitoring.MetricType.MUTATION_BYTES;
 import static org.apache.phoenix.monitoring.MetricType.MUTATION_COMMIT_TIME;
 import static org.apache.phoenix.monitoring.MetricType.MUTATION_SQL_COUNTER;
 import static org.apache.phoenix.monitoring.MetricType.NUM_PARALLEL_SCANS;
+import static org.apache.phoenix.monitoring.MetricType.QUERY_SERVICES_COUNTER;
 import static org.apache.phoenix.monitoring.MetricType.QUERY_TIME;
 import static org.apache.phoenix.monitoring.MetricType.QUERY_TIMEOUT_COUNTER;
 import static org.apache.phoenix.monitoring.MetricType.TASK_REJECTED_COUNTER;
@@ -73,7 +75,9 @@ public enum GlobalClientMetrics {
     GLOBAL_QUERY_TIMEOUT_COUNTER(QUERY_TIMEOUT_COUNTER),
     GLOBAL_FAILED_QUERY_COUNTER(QUERY_FAILED_COUNTER),
     GLOBAL_SPOOL_FILE_COUNTER(SPOOL_FILE_COUNTER),
-    GLOBAL_OPEN_PHOENIX_CONNECTIONS(OPEN_PHOENIX_CONNECTIONS_COUNTER);
+    GLOBAL_OPEN_PHOENIX_CONNECTIONS(OPEN_PHOENIX_CONNECTIONS_COUNTER),
+    GLOBAL_QUERY_SERVICES_COUNTER(QUERY_SERVICES_COUNTER),
+    GLOBAL_HCONNECTIONS_COUNTER(HCONNECTIONS_COUNTER);
     
     private static final boolean isGlobalMetricsEnabled = QueryServicesOptions.withDefaults().isGlobalMetricsEnabled();
     private GlobalMetric metric;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f5de28b6/phoenix-core/src/main/java/org/apache/phoenix/monitoring/MetricType.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/monitoring/MetricType.java b/phoenix-core/src/main/java/org/apache/phoenix/monitoring/MetricType.java
index 6cfe977..b420b75 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/monitoring/MetricType.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/monitoring/MetricType.java
@@ -41,7 +41,9 @@ public enum MetricType {
     CACHE_REFRESH_SPLITS_COUNTER("Number of times cache was refreshed because of splits"),
     WALL_CLOCK_TIME_MS("Wall clock time elapsed for the overall query execution"),
     RESULT_SET_TIME_MS("Wall clock time elapsed for reading all records using resultSet.next()"),
-    OPEN_PHOENIX_CONNECTIONS_COUNTER("Number of open phoenix connections");
+    OPEN_PHOENIX_CONNECTIONS_COUNTER("Number of open phoenix connections"),
+    QUERY_SERVICES_COUNTER("Number of ConnectionQueryServicesImpl instantiated"),
+    HCONNECTIONS_COUNTER("Number of HConnections created by phoenix driver");
     
     private final String description;
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f5de28b6/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
index be34f66..f2eb8e4 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
@@ -26,6 +26,8 @@ import static org.apache.phoenix.coprocessor.MetaDataProtocol.getVersion;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_STATS_NAME;
+import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_HCONNECTIONS_COUNTER;
+import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_QUERY_SERVICES_COUNTER;
 import static org.apache.phoenix.query.QueryServicesOptions.DEFAULT_DROP_METADATA;
 import static org.apache.phoenix.query.QueryServicesOptions.DEFAULT_RENEW_LEASE_ENABLED;
 import static org.apache.phoenix.query.QueryServicesOptions.DEFAULT_RENEW_LEASE_THREAD_POOL_SIZE;
@@ -151,6 +153,8 @@ import org.apache.phoenix.iterate.TableResultIterator.RenewLeaseStatus;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.jdbc.PhoenixEmbeddedDriver.ConnectionInfo;
+import org.apache.phoenix.monitoring.GlobalClientMetrics;
+import org.apache.phoenix.monitoring.GlobalMetric;
 import org.apache.phoenix.parse.PFunction;
 import org.apache.phoenix.parse.PSchema;
 import org.apache.phoenix.protobuf.ProtobufUtil;
@@ -392,6 +396,8 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     QueryServices.TRANSACTIONS_ENABLED,
                     QueryServicesOptions.DEFAULT_TRANSACTIONS_ENABLED);
             this.connection = HBaseFactoryProvider.getHConnectionFactory().createConnection(this.config);
+            GLOBAL_HCONNECTIONS_COUNTER.increment();
+            logger.info("HConnnection established. Details: " + connection + " " +  Throwables.getStackTraceAsString(new Exception()));
             // only initialize the tx service client if needed and if we succeeded in getting a connection
             // to HBase
             if (transactionsEnabled) {
@@ -457,6 +463,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 return;
             }
             closed = true;
+            GLOBAL_QUERY_SERVICES_COUNTER.decrement();
             SQLException sqlE = null;
             try {
                 // Attempt to return any unused sequences.
@@ -473,6 +480,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     try {
                         // close the HBase connection
                         if (connection != null) connection.close();
+                        GLOBAL_HCONNECTIONS_COUNTER.decrement();
                     } finally {
                         if (renewLeaseExecutor != null) {
                             renewLeaseExecutor.shutdownNow();
@@ -2360,6 +2368,8 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                         boolean hConnectionEstablished = false;
                         boolean success = false;
                         try {
+                            GLOBAL_QUERY_SERVICES_COUNTER.increment();
+                            logger.info("An instance of ConnectionQueryServices was created: " + Throwables.getStackTraceAsString(new Exception()));
                             openConnection();
                             hConnectionEstablished = true;
                             boolean isDoNotUpgradePropSet = UpgradeUtil.isNoUpgradeSet(props);


[07/26] phoenix git commit: PHOENIX-2890 Extend IndexTool to allow incremental index rebuilds(addendum)

Posted by ma...@apache.org.
PHOENIX-2890 Extend IndexTool to allow incremental index rebuilds(addendum)


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

Branch: refs/heads/calcite
Commit: e906841fb2c96615f0d50b06c48bdeda78379c94
Parents: 83827cd
Author: Ankit Singhal <an...@gmail.com>
Authored: Mon Dec 26 12:28:14 2016 +0530
Committer: Ankit Singhal <an...@gmail.com>
Committed: Mon Dec 26 12:28:14 2016 +0530

----------------------------------------------------------------------
 .../end2end/IndexToolForPartialBuildWithNamespaceEnabledIT.java  | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/e906841f/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolForPartialBuildWithNamespaceEnabledIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolForPartialBuildWithNamespaceEnabledIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolForPartialBuildWithNamespaceEnabledIT.java
index 5e16b05..4b2371c 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolForPartialBuildWithNamespaceEnabledIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolForPartialBuildWithNamespaceEnabledIT.java
@@ -36,10 +36,10 @@ import com.google.common.collect.Maps;
  * Tests for the {@link IndexToolForPartialBuildWithNamespaceEnabled}
  */
 @RunWith(Parameterized.class)
-public class IndexToolForPartialBuildWithNamespaceEnabled extends IndexToolForPartialBuildIT {
+public class IndexToolForPartialBuildWithNamespaceEnabledIT extends IndexToolForPartialBuildIT {
     
     
-    public IndexToolForPartialBuildWithNamespaceEnabled(boolean localIndex, boolean isNamespaceEnabled) {
+    public IndexToolForPartialBuildWithNamespaceEnabledIT(boolean localIndex, boolean isNamespaceEnabled) {
         super(localIndex);
         this.isNamespaceEnabled=isNamespaceEnabled;
     }


[03/26] phoenix git commit: PHOENIX-3505 Avoid NPE on close() in OrderedResultIterator

Posted by ma...@apache.org.
PHOENIX-3505 Avoid NPE on close() in OrderedResultIterator


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

Branch: refs/heads/calcite
Commit: e23634a358929516ce210fe06d668ce475eccccb
Parents: c504604
Author: Josh Elser <el...@apache.org>
Authored: Wed Nov 23 11:16:35 2016 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Sat Dec 24 23:11:10 2016 -0500

----------------------------------------------------------------------
 .../phoenix/iterate/OrderedResultIterator.java  |  5 ++-
 .../iterate/OrderedResultIteratorTest.java      | 41 ++++++++++++++++++++
 2 files changed, 45 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/e23634a3/phoenix-core/src/main/java/org/apache/phoenix/iterate/OrderedResultIterator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/OrderedResultIterator.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/OrderedResultIterator.java
index 8dcb2e8..da75bb7 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/iterate/OrderedResultIterator.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/OrderedResultIterator.java
@@ -279,7 +279,10 @@ public class OrderedResultIterator implements PeekingResultIterator {
 
     @Override
     public void close() throws SQLException {
-        resultIterator.close();
+        // Guard against resultIterator being null
+        if (null != resultIterator) {
+            resultIterator.close();
+        }
         resultIterator = PeekingResultIterator.EMPTY_ITERATOR;
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/e23634a3/phoenix-core/src/test/java/org/apache/phoenix/iterate/OrderedResultIteratorTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/iterate/OrderedResultIteratorTest.java b/phoenix-core/src/test/java/org/apache/phoenix/iterate/OrderedResultIteratorTest.java
new file mode 100644
index 0000000..50ed8e9
--- /dev/null
+++ b/phoenix-core/src/test/java/org/apache/phoenix/iterate/OrderedResultIteratorTest.java
@@ -0,0 +1,41 @@
+/*
+ * 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.iterate;
+
+import java.sql.SQLException;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.phoenix.expression.OrderByExpression;
+import org.junit.Test;
+
+/**
+ * Test class for {@link OrderedResultIterator}.
+ */
+public class OrderedResultIteratorTest {
+
+  @Test
+  public void testNullIteratorOnClose() throws SQLException {
+      ResultIterator delegate =  ResultIterator.EMPTY_ITERATOR;
+      List<OrderByExpression> orderByExpressions = Collections.singletonList(null);
+      int thresholdBytes = Integer.MAX_VALUE;
+      OrderedResultIterator iterator = new OrderedResultIterator(delegate, orderByExpressions, thresholdBytes);
+      // Should not throw an exception
+      iterator.close();
+  }
+
+}


[05/26] phoenix git commit: PHOENIX-2890 Extend IndexTool to allow incremental index rebuilds

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/83827cd8/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexTool.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexTool.java b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexTool.java
index 82b353c..e594e0d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexTool.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexTool.java
@@ -17,10 +17,17 @@
  */
 package org.apache.phoenix.mapreduce.index;
 
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ASYNC_REBUILD_TIMESTAMP;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_DISABLE_TIMESTAMP;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_NAME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_SCHEM;
+
 import java.sql.Connection;
 import java.sql.DatabaseMetaData;
 import java.sql.ResultSet;
 import java.sql.SQLException;
+import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.commons.cli.CommandLine;
@@ -36,11 +43,15 @@ import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.mapreduce.HFileOutputFormat;
 import org.apache.hadoop.hbase.mapreduce.LoadIncrementalHFiles;
+import org.apache.hadoop.hbase.mapreduce.TableInputFormat;
 import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
 import org.apache.hadoop.hbase.mapreduce.TableOutputFormat;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -48,9 +59,13 @@ import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.phoenix.compile.PostIndexDDLCompiler;
+import org.apache.phoenix.coprocessor.BaseScannerRegionObserver;
+import org.apache.phoenix.hbase.index.util.IndexManagementUtil;
+import org.apache.phoenix.index.IndexMaintainer;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.mapreduce.CsvBulkImportUtil;
 import org.apache.phoenix.mapreduce.util.ColumnInfoToStringEncoderDecoder;
@@ -62,9 +77,9 @@ import org.apache.phoenix.schema.PIndexState;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTable.IndexType;
 import org.apache.phoenix.schema.TableRef;
+import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.ColumnInfo;
 import org.apache.phoenix.util.IndexUtil;
-import org.apache.phoenix.util.MetaDataUtil;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.SchemaUtil;
@@ -72,6 +87,8 @@ import org.apache.phoenix.util.TransactionUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.collect.Lists;
+
 /**
  * An MR job to populate the index table from the data table.
  *
@@ -85,7 +102,11 @@ public class IndexTool extends Configured implements Tool {
     private static final Option DATA_TABLE_OPTION = new Option("dt", "data-table", true,
             "Data table name (mandatory)");
     private static final Option INDEX_TABLE_OPTION = new Option("it", "index-table", true,
-            "Index table name(mandatory)");
+            "Index table name(not required in case of partial rebuilding)");
+    
+    private static final Option PARTIAL_REBUILD_OPTION = new Option("pr", "partial-rebuild", false,
+            "To build indexes for a data table from least disabledTimeStamp");
+    
     private static final Option DIRECT_API_OPTION = new Option("direct", "direct", false,
             "If specified, we avoid the bulk load (optional)");
     private static final Option RUN_FOREGROUND_OPTION =
@@ -105,6 +126,7 @@ public class IndexTool extends Configured implements Tool {
         options.addOption(SCHEMA_NAME_OPTION);
         options.addOption(DATA_TABLE_OPTION);
         options.addOption(INDEX_TABLE_OPTION);
+        options.addOption(PARTIAL_REBUILD_OPTION);
         options.addOption(DIRECT_API_OPTION);
         options.addOption(RUN_FOREGROUND_OPTION);
         options.addOption(OUTPUT_PATH_OPTION);
@@ -139,18 +161,18 @@ public class IndexTool extends Configured implements Tool {
                     + "parameter");
         }
 
-        if (!cmdLine.hasOption(INDEX_TABLE_OPTION.getOpt())) {
-            throw new IllegalStateException(INDEX_TABLE_OPTION.getLongOpt() + " is a mandatory "
-                    + "parameter");
-        }
-
-        if (!cmdLine.hasOption(OUTPUT_PATH_OPTION.getOpt())) {
-            throw new IllegalStateException(OUTPUT_PATH_OPTION.getLongOpt() + " is a mandatory "
-                    + "parameter");
-        }
-
-        if (!cmdLine.hasOption(DIRECT_API_OPTION.getOpt())
-                && cmdLine.hasOption(RUN_FOREGROUND_OPTION.getOpt())) {
+		if (!(cmdLine.hasOption(PARTIAL_REBUILD_OPTION.getOpt()) || cmdLine.hasOption(DIRECT_API_OPTION.getOpt()))
+				&& !cmdLine.hasOption(OUTPUT_PATH_OPTION.getOpt())) {
+			throw new IllegalStateException(OUTPUT_PATH_OPTION.getLongOpt() + " is a mandatory " + "parameter");
+		}
+        
+		if (cmdLine.hasOption(PARTIAL_REBUILD_OPTION.getOpt()) && cmdLine.hasOption(INDEX_TABLE_OPTION.getOpt())) {
+			throw new IllegalStateException("Index name should not be passed with " + PARTIAL_REBUILD_OPTION.getLongOpt());
+		}
+        		
+        if (!(cmdLine.hasOption(DIRECT_API_OPTION.getOpt())) && cmdLine.hasOption(INDEX_TABLE_OPTION.getOpt())
+                && cmdLine.hasOption(RUN_FOREGROUND_OPTION
+                        .getOpt())) {
             throw new IllegalStateException(RUN_FOREGROUND_OPTION.getLongOpt()
                     + " is applicable only for " + DIRECT_API_OPTION.getLongOpt());
         }
@@ -167,50 +189,155 @@ public class IndexTool extends Configured implements Tool {
         formatter.printHelp("help", options);
         System.exit(exitCode);
     }
+    
+    class JobFactory {
+        Connection connection;
+        Configuration configuration;
+        private Path outputPath;
 
-    @Override
-    public int run(String[] args) throws Exception {
-        Connection connection = null;
-        try {
-            CommandLine cmdLine = null;
-            try {
-                cmdLine = parseOptions(args);
-            } catch (IllegalStateException e) {
-                printHelpAndExit(e.getMessage(), getOptions());
+        public JobFactory(Connection connection, Configuration configuration, Path outputPath) {
+            this.connection = connection;
+            this.configuration = configuration;
+            this.outputPath = outputPath;
+
+        }
+
+        public Job getJob(String schemaName, String indexTable, String dataTable, boolean useDirectApi, boolean isPartialBuild) throws Exception {
+            if (isPartialBuild) {
+                return configureJobForPartialBuild(schemaName, dataTable);
+            } else {
+                return configureJobForAysncIndex(schemaName, indexTable, dataTable, useDirectApi);
             }
-            final Configuration configuration = HBaseConfiguration.addHbaseResources(getConf());
-            final String schemaName = cmdLine.getOptionValue(SCHEMA_NAME_OPTION.getOpt());
-            final String dataTable = cmdLine.getOptionValue(DATA_TABLE_OPTION.getOpt());
-            final String indexTable = cmdLine.getOptionValue(INDEX_TABLE_OPTION.getOpt());
+        }
+        
+        private Job configureJobForPartialBuild(String schemaName, String dataTable) throws Exception {
             final String qDataTable = SchemaUtil.getQualifiedTableName(schemaName, dataTable);
-            final String qIndexTable = SchemaUtil.getQualifiedTableName(schemaName, indexTable);
-
+            final PTable pdataTable = PhoenixRuntime.getTable(connection, qDataTable);
             connection = ConnectionUtil.getInputConnection(configuration);
-            if (!isValidIndexTable(connection, qDataTable, indexTable)) {
-                throw new IllegalArgumentException(String.format(
-                    " %s is not an index table for %s ", qIndexTable, qDataTable));
+            long minDisableTimestamp = HConstants.LATEST_TIMESTAMP;
+            PTable indexWithMinDisableTimestamp = null;
+            
+            //Get Indexes in building state, minDisabledTimestamp 
+            List<String> disableIndexes = new ArrayList<String>();
+            List<PTable> disabledPIndexes = new ArrayList<PTable>();
+            for (PTable index : pdataTable.getIndexes()) {
+                if (index.getIndexState().equals(PIndexState.BUILDING)) {
+                    disableIndexes.add(index.getTableName().getString());
+                    disabledPIndexes.add(index);
+                    if (minDisableTimestamp > index.getIndexDisableTimestamp()) {
+                        minDisableTimestamp = index.getIndexDisableTimestamp();
+                        indexWithMinDisableTimestamp = index;
+                    }
+                }
+            }
+            
+            if (indexWithMinDisableTimestamp == null) {
+                throw new Exception("There is no index for a datatable to be rebuild:" + qDataTable);
+            }
+            if (minDisableTimestamp == 0) {
+                throw new Exception("It seems Index " + indexWithMinDisableTimestamp
+                        + " has disable timestamp as 0 , please run IndexTool with IndexName to build it first");
+                // TODO probably we can initiate the job by ourself or can skip them while making the list for partial build with a warning
+            }
+            
+            long maxTimestamp = getMaxRebuildAsyncDate(schemaName, disableIndexes);
+            
+            //serialize index maintaienr in job conf with Base64 TODO: Need to find better way to serialize them in conf.
+            List<IndexMaintainer> maintainers = Lists.newArrayListWithExpectedSize(disabledPIndexes.size());
+            for (PTable index : disabledPIndexes) {
+                maintainers.add(index.getIndexMaintainer(pdataTable, connection.unwrap(PhoenixConnection.class)));
+            }
+            ImmutableBytesWritable indexMetaDataPtr = new ImmutableBytesWritable(ByteUtil.EMPTY_BYTE_ARRAY);
+            IndexMaintainer.serializeAdditional(pdataTable, indexMetaDataPtr, disabledPIndexes, connection.unwrap(PhoenixConnection.class));
+            PhoenixConfigurationUtil.setIndexMaintainers(configuration, indexMetaDataPtr);
+            
+            //Prepare raw scan 
+            Scan scan = IndexManagementUtil.newLocalStateScan(maintainers);
+            scan.setTimeRange(minDisableTimestamp - 1, maxTimestamp);
+            scan.setRaw(true);
+            scan.setCacheBlocks(false);
+            if (pdataTable.isTransactional()) {
+                long maxTimeRange = pdataTable.getTimeStamp() + 1;
+                scan.setAttribute(BaseScannerRegionObserver.TX_SCN,
+                        Bytes.toBytes(Long.valueOf(Long.toString(TransactionUtil.convertToNanoseconds(maxTimeRange)))));
+            }
+            
+          
+            String physicalTableName=pdataTable.getPhysicalName().getString();
+            final String jobName = String.format("Phoenix Indexes build for " + pdataTable.getName().toString());
+            
+            PhoenixConfigurationUtil.setInputTableName(configuration, qDataTable);
+            PhoenixConfigurationUtil.setPhysicalTableName(configuration, physicalTableName);
+            
+            //TODO: update disable indexes
+            PhoenixConfigurationUtil.setDisableIndexes(configuration, StringUtils.join(",",disableIndexes));
+            
+            final Job job = Job.getInstance(configuration, jobName);
+			if (outputPath != null) {
+				FileOutputFormat.setOutputPath(job, outputPath);
+			}
+            job.setJarByClass(IndexTool.class);
+            TableMapReduceUtil.initTableMapperJob(physicalTableName, scan, PhoenixIndexPartialBuildMapper.class, null,
+                    null, job);
+            TableMapReduceUtil.initCredentials(job);
+            TableInputFormat.configureSplitTable(job, TableName.valueOf(physicalTableName));
+            return configureSubmittableJobUsingDirectApi(job, true);
+        }
+        
+        private long getMaxRebuildAsyncDate(String schemaName, List<String> disableIndexes) throws SQLException {
+            Long maxRebuilAsyncDate=HConstants.LATEST_TIMESTAMP;
+            Long maxDisabledTimeStamp=0L;
+            if (disableIndexes == null || disableIndexes.isEmpty()) { return 0; }
+            List<String> quotedIndexes = new ArrayList<String>(disableIndexes.size());
+            for (String index : disableIndexes) {
+                quotedIndexes.add("'" + index + "'");
+            }
+            ResultSet rs = connection.createStatement()
+                    .executeQuery("SELECT MAX(" + ASYNC_REBUILD_TIMESTAMP + "),MAX("+INDEX_DISABLE_TIMESTAMP+") FROM " + SYSTEM_CATALOG_NAME + " ("
+                            + ASYNC_REBUILD_TIMESTAMP + " BIGINT) WHERE " + TABLE_SCHEM
+                            + (schemaName != null && schemaName.length() > 0 ? "='" + schemaName + "'" : " IS NULL")
+                            + " and " + TABLE_NAME + " IN (" + StringUtils.join(",", quotedIndexes) + ")");
+            if (rs.next()) {
+                maxRebuilAsyncDate = rs.getLong(1);
+                maxDisabledTimeStamp = rs.getLong(2);
+            }
+            // Do check if table is disabled again after user invoked async rebuilding during the run of the job
+            if (maxRebuilAsyncDate > maxDisabledTimeStamp) {
+                return maxRebuilAsyncDate;
+            } else {
+                throw new RuntimeException(
+                        "Inconsistent state we have one or more index tables which are disabled after the async is called!!");
             }
+            
+        }
 
+        private Job configureJobForAysncIndex(String schemaName, String indexTable, String dataTable, boolean useDirectApi)
+                throws Exception {
+            final String qDataTable = SchemaUtil.getQualifiedTableName(schemaName, dataTable);
+            final String qIndexTable;
+            if (schemaName != null && !schemaName.isEmpty()) {
+                qIndexTable = SchemaUtil.getQualifiedTableName(schemaName, indexTable);
+            } else {
+                qIndexTable = indexTable;
+            }
             final PTable pdataTable = PhoenixRuntime.getTable(connection, qDataTable);
+            
             final PTable pindexTable = PhoenixRuntime.getTable(connection, qIndexTable);
-
+            
+            long maxTimeRange = pindexTable.getTimeStamp() + 1;
             // this is set to ensure index tables remains consistent post population.
-            long maxTimeRange = pindexTable.getTimeStamp()+1;
+
             if (pdataTable.isTransactional()) {
                 configuration.set(PhoenixConfigurationUtil.TX_SCN_VALUE,
                     Long.toString(TransactionUtil.convertToNanoseconds(maxTimeRange)));
             }
             configuration.set(PhoenixConfigurationUtil.CURRENT_SCN_VALUE,
                 Long.toString(maxTimeRange));
-
+            
             // check if the index type is LOCAL, if so, derive and set the physicalIndexName that is
             // computed from the qDataTable name.
             String physicalIndexTable = pindexTable.getPhysicalName().getString();
-            boolean isLocalIndexBuild = false;
-            if (IndexType.LOCAL.equals(pindexTable.getIndexType())) {
-                physicalIndexTable = qDataTable;
-                isLocalIndexBuild = true;
-            }
+            
 
             final PhoenixConnection pConnection = connection.unwrap(PhoenixConnection.class);
             final PostIndexDDLCompiler ddlCompiler =
@@ -224,18 +351,15 @@ public class IndexTool extends Configured implements Tool {
 
             configuration.set(PhoenixConfigurationUtil.UPSERT_STATEMENT, upsertQuery);
             PhoenixConfigurationUtil.setPhysicalTableName(configuration, physicalIndexTable);
-            PhoenixConfigurationUtil.setOutputTableName(configuration, indexTable);
+            PhoenixConfigurationUtil.setDisableIndexes(configuration, indexTable);
             PhoenixConfigurationUtil.setUpsertColumnNames(configuration,
                 indexColumns.toArray(new String[indexColumns.size()]));
             final List<ColumnInfo> columnMetadataList =
                     PhoenixRuntime.generateColumnInfo(connection, qIndexTable, indexColumns);
             ColumnInfoToStringEncoderDecoder.encode(configuration, columnMetadataList);
-
-            final Path outputPath = CsvBulkImportUtil
-                    .getOutputPath(new Path(cmdLine.getOptionValue(OUTPUT_PATH_OPTION.getOpt())), physicalIndexTable);
             FileSystem.get(configuration).delete(outputPath, true);
             
-            final String jobName = String.format(INDEX_JOB_NAME_TEMPLATE, dataTable, indexTable);
+            final String jobName = String.format(INDEX_JOB_NAME_TEMPLATE, pdataTable.getName().toString(), indexTable);
             final Job job = Job.getInstance(configuration, jobName);
             job.setJarByClass(IndexTool.class);
             job.setMapOutputKeyClass(ImmutableBytesWritable.class);
@@ -245,17 +369,146 @@ public class IndexTool extends Configured implements Tool {
                 selectQuery);
             TableMapReduceUtil.initCredentials(job);
             
-            boolean useDirectApi = cmdLine.hasOption(DIRECT_API_OPTION.getOpt());
+            
             if (useDirectApi) {
-                configureSubmittableJobUsingDirectApi(job, outputPath,
-                    cmdLine.hasOption(RUN_FOREGROUND_OPTION.getOpt()));
+                return configureSubmittableJobUsingDirectApi(job, false);
             } else {
-                configureRunnableJobUsingBulkLoad(job, outputPath, isLocalIndexBuild);
-                // Without direct API, we need to update the index state to ACTIVE from client.
-                IndexToolUtil.updateIndexState(connection, qDataTable, indexTable,
-                        PIndexState.ACTIVE);
+                return configureRunnableJobUsingBulkLoad(job, outputPath);
+                
+            }
+            
+        }
+
+        /**
+         * Submits the job and waits for completion.
+         * @param job
+         * @param outputPath
+         * @return
+         * @throws Exception
+         */
+        private Job configureRunnableJobUsingBulkLoad(Job job, Path outputPath) throws Exception {
+            job.setMapperClass(PhoenixIndexImportMapper.class);
+            job.setMapOutputKeyClass(ImmutableBytesWritable.class);
+            job.setMapOutputValueClass(KeyValue.class);
+            final Configuration configuration = job.getConfiguration();
+            final String physicalIndexTable =
+                    PhoenixConfigurationUtil.getPhysicalTableName(configuration);
+            final HTable htable = new HTable(configuration, physicalIndexTable);
+            HFileOutputFormat.configureIncrementalLoad(job, htable);
+            return job;
+               
+        }
+        
+        /**
+         * Uses the HBase Front Door Api to write to index table. Submits the job and either returns or
+         * waits for the job completion based on runForeground parameter.
+         * 
+         * @param job
+         * @param outputPath
+         * @param runForeground - if true, waits for job completion, else submits and returns
+         *            immediately.
+         * @return
+         * @throws Exception
+         */
+        private Job configureSubmittableJobUsingDirectApi(Job job, boolean isPartialRebuild)
+                throws Exception {
+            if (!isPartialRebuild) {
+                //Don't configure mapper for partial build as it is configured already
+                job.setMapperClass(PhoenixIndexImportDirectMapper.class);
+            }
+            job.setReducerClass(PhoenixIndexImportDirectReducer.class);
+            Configuration conf = job.getConfiguration();
+            HBaseConfiguration.merge(conf, HBaseConfiguration.create(conf));
+            // Set the Physical Table name for use in DirectHTableWriter#write(Mutation)
+            conf.set(TableOutputFormat.OUTPUT_TABLE,
+                PhoenixConfigurationUtil.getPhysicalTableName(job.getConfiguration()));
+            //Set the Output classes
+            job.setMapOutputKeyClass(ImmutableBytesWritable.class);
+            job.setMapOutputValueClass(IntWritable.class);
+            job.setOutputKeyClass(NullWritable.class);
+            job.setOutputValueClass(NullWritable.class);
+            TableMapReduceUtil.addDependencyJars(job);
+            job.setNumReduceTasks(1);
+            return job;
+        }
+        
+    }
+
+    @Override
+    public int run(String[] args) throws Exception {
+        Connection connection = null;
+        HTable htable = null;
+        try {
+            CommandLine cmdLine = null;
+            try {
+                cmdLine = parseOptions(args);
+            } catch (IllegalStateException e) {
+                printHelpAndExit(e.getMessage(), getOptions());
+            }
+            final Configuration configuration = HBaseConfiguration.addHbaseResources(getConf());
+            final String schemaName = cmdLine.getOptionValue(SCHEMA_NAME_OPTION.getOpt());
+            final String dataTable = cmdLine.getOptionValue(DATA_TABLE_OPTION.getOpt());
+            final String indexTable = cmdLine.getOptionValue(INDEX_TABLE_OPTION.getOpt());
+            final boolean isPartialBuild = cmdLine.hasOption(PARTIAL_REBUILD_OPTION.getOpt());
+            final String qDataTable = SchemaUtil.getQualifiedTableName(schemaName, dataTable);
+            boolean useDirectApi = cmdLine.hasOption(DIRECT_API_OPTION.getOpt());
+            String basePath=cmdLine.getOptionValue(OUTPUT_PATH_OPTION.getOpt());
+            boolean isForeground = cmdLine.hasOption(RUN_FOREGROUND_OPTION.getOpt());
+            connection = ConnectionUtil.getInputConnection(configuration);
+            byte[][] splitKeysBeforeJob = null;
+            boolean isLocalIndexBuild = false;
+            PTable pindexTable = null;
+            if (indexTable != null) {
+                if (!isValidIndexTable(connection, qDataTable,indexTable)) {
+                    throw new IllegalArgumentException(String.format(
+                        " %s is not an index table for %s ", indexTable, qDataTable));
+                }
+                pindexTable = PhoenixRuntime.getTable(connection, schemaName != null && !schemaName.isEmpty()
+                        ? SchemaUtil.getQualifiedTableName(schemaName, indexTable) : indexTable);
+                htable = (HTable)connection.unwrap(PhoenixConnection.class).getQueryServices()
+                        .getTable(pindexTable.getPhysicalName().getBytes());
+                if (IndexType.LOCAL.equals(pindexTable.getIndexType())) {
+                    isLocalIndexBuild = true;
+                    splitKeysBeforeJob = htable.getRegionLocator().getStartKeys();
+                }
+            }
+            
+            PTable pdataTable = PhoenixRuntime.getTableNoCache(connection, qDataTable);
+			Path outputPath = null;
+			if (basePath != null) {
+				outputPath = CsvBulkImportUtil.getOutputPath(new Path(basePath), pindexTable == null
+						? pdataTable.getPhysicalName().getString() : pindexTable.getPhysicalName().getString());
+				FileSystem.get(configuration).delete(outputPath, true);
+			}
+            
+            Job job = new JobFactory(connection, configuration, outputPath).getJob(schemaName, indexTable, dataTable,
+                    useDirectApi, isPartialBuild);
+            if (!isForeground && useDirectApi) {
+                LOG.info("Running Index Build in Background - Submit async and exit");
+                job.submit();
+                return 0;
+            }
+            LOG.info("Running Index Build in Foreground. Waits for the build to complete. This may take a long time!.");
+            boolean result = job.waitForCompletion(true);
+            
+            if (result) {
+                if (!useDirectApi && indexTable != null) {
+                    if (isLocalIndexBuild) {
+                        validateSplitForLocalIndex(splitKeysBeforeJob, htable);
+                    }
+                    LOG.info("Loading HFiles from {}", outputPath);
+                    LoadIncrementalHFiles loader = new LoadIncrementalHFiles(configuration);
+                    loader.doBulkLoad(outputPath, htable);
+                    htable.close();
+                    // Without direct API, we need to update the index state to ACTIVE from client.
+                    IndexToolUtil.updateIndexState(connection, qDataTable, indexTable, PIndexState.ACTIVE);
+                    FileSystem.get(configuration).delete(outputPath, true);
+                }
+                return 0;
+            } else {
+                LOG.error("IndexTool job failed! Check logs for errors..");
+                return -1;
             }
-            return 0;
         } catch (Exception ex) {
             LOG.error("An exception occurred while performing the indexing job: "
                     + ExceptionUtils.getMessage(ex) + " at:\n" + ExceptionUtils.getStackTrace(ex));
@@ -265,6 +518,9 @@ public class IndexTool extends Configured implements Tool {
                 if (connection != null) {
                     connection.close();
                 }
+                if (htable != null) {
+                    htable.close();
+                }
             } catch (SQLException sqle) {
                 LOG.error("Failed to close connection ", sqle.getMessage());
                 throw new RuntimeException("Failed to close connection");
@@ -272,91 +528,18 @@ public class IndexTool extends Configured implements Tool {
         }
     }
 
-    /**
-     * Submits the job and waits for completion.
-     * @param job
-     * @param outputPath
-     * @return
-     * @throws Exception
-     */
-    private void configureRunnableJobUsingBulkLoad(Job job, Path outputPath, boolean isLocalIndexBuild) throws Exception {
-        job.setMapperClass(PhoenixIndexImportMapper.class);
-        job.setMapOutputKeyClass(ImmutableBytesWritable.class);
-        job.setMapOutputValueClass(KeyValue.class);
-        final Configuration configuration = job.getConfiguration();
-        final String physicalIndexTable =
-                PhoenixConfigurationUtil.getPhysicalTableName(configuration);
-        final HTable htable = new HTable(configuration, physicalIndexTable);
-        HFileOutputFormat.configureIncrementalLoad(job, htable);
-        byte[][] splitKeysBeforeJob = null;
-        if(isLocalIndexBuild) {
-            splitKeysBeforeJob = htable.getRegionLocator().getStartKeys();
-        }
-        boolean status = job.waitForCompletion(true);
-        if (!status) {
-            LOG.error("IndexTool job failed!");
-            htable.close();
-            throw new Exception("IndexTool job failed: " + job.toString());
-        } else {
-            if (isLocalIndexBuild
-                    && !IndexUtil.matchingSplitKeys(splitKeysBeforeJob, htable.getRegionLocator()
-                            .getStartKeys())) {
-                String errMsg = "The index to build is local index and the split keys are not matching"
-                        + " before and after running the job. Please rerun the job otherwise"
-                        + " there may be inconsistencies between actual data and index data";
-                LOG.error(errMsg);
-                throw new Exception(errMsg);
-            }
-        }
-
-        LOG.info("Loading HFiles from {}", outputPath);
-        LoadIncrementalHFiles loader = new LoadIncrementalHFiles(configuration);
-        loader.doBulkLoad(outputPath, htable);
-        htable.close();
-        
-        FileSystem.get(configuration).delete(outputPath, true);
-    }
     
-    /**
-     * Uses the HBase Front Door Api to write to index table. Submits the job and either returns or
-     * waits for the job completion based on runForeground parameter.
-     * 
-     * @param job
-     * @param outputPath
-     * @param runForeground - if true, waits for job completion, else submits and returns
-     *            immediately.
-     * @return
-     * @throws Exception
-     */
-    private void configureSubmittableJobUsingDirectApi(Job job, Path outputPath, boolean runForeground)
-            throws Exception {
-        job.setMapperClass(PhoenixIndexImportDirectMapper.class);
-        job.setReducerClass(PhoenixIndexImportDirectReducer.class);
-        Configuration conf = job.getConfiguration();
-        HBaseConfiguration.merge(conf, HBaseConfiguration.create(conf));
-        // Set the Physical Table name for use in DirectHTableWriter#write(Mutation)
-        conf.set(TableOutputFormat.OUTPUT_TABLE,
-            PhoenixConfigurationUtil.getPhysicalTableName(job.getConfiguration()));
-        
-        //Set the Output classes
-        job.setMapOutputValueClass(IntWritable.class);
-        job.setOutputKeyClass(NullWritable.class);
-        job.setOutputValueClass(NullWritable.class);
-        TableMapReduceUtil.addDependencyJars(job);
-        job.setNumReduceTasks(1);
-
-        if (!runForeground) {
-            LOG.info("Running Index Build in Background - Submit async and exit");
-            job.submit();
-            return;
-        }
-        LOG.info("Running Index Build in Foreground. Waits for the build to complete. This may take a long time!.");
-        boolean result = job.waitForCompletion(true);
-        if (!result) {
-            LOG.error("IndexTool job failed!");
-            throw new Exception("IndexTool job failed: " + job.toString());
+
+    private boolean validateSplitForLocalIndex(byte[][] splitKeysBeforeJob, HTable htable) throws Exception {
+        if (splitKeysBeforeJob != null
+                && !IndexUtil.matchingSplitKeys(splitKeysBeforeJob, htable.getRegionLocator().getStartKeys())) {
+            String errMsg = "The index to build is local index and the split keys are not matching"
+                    + " before and after running the job. Please rerun the job otherwise"
+                    + " there may be inconsistencies between actual data and index data";
+            LOG.error(errMsg);
+            throw new Exception(errMsg);
         }
-        FileSystem.get(conf).delete(outputPath, true);
+        return true;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/phoenix/blob/83827cd8/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexToolUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexToolUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexToolUtil.java
index 1058670..2dc7551 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexToolUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/IndexToolUtil.java
@@ -48,11 +48,13 @@ public class IndexToolUtil {
 	 */
 	public static void updateIndexState(Configuration configuration,PIndexState state) throws SQLException {
 		final String masterTable = PhoenixConfigurationUtil.getInputTableName(configuration);
-		final String indexTable = PhoenixConfigurationUtil.getOutputTableName(configuration);
+		final String[] indexTables = PhoenixConfigurationUtil.getDisableIndexes(configuration).split(",");
 		final Properties overrideProps = new Properties();
 		final Connection connection = ConnectionUtil.getOutputConnection(configuration, overrideProps);
 		try {
-			updateIndexState(connection, masterTable, indexTable , state);
+            for (String indexTable : indexTables) {
+                updateIndexState(connection, masterTable, indexTable, state);
+            }
 		} finally {
 			if(connection != null) {
 				connection.close();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/83827cd8/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/PhoenixIndexImportDirectMapper.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/PhoenixIndexImportDirectMapper.java b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/PhoenixIndexImportDirectMapper.java
index 9c64efc..15e55dd 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/PhoenixIndexImportDirectMapper.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/PhoenixIndexImportDirectMapper.java
@@ -99,7 +99,7 @@ public class PhoenixIndexImportDirectMapper extends
             this.pStatement = connection.prepareStatement(upsertQuery);
 
         } catch (SQLException e) {
-            throw new RuntimeException(e.getMessage());
+            throw new RuntimeException(e);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/83827cd8/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/PhoenixIndexPartialBuildMapper.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/PhoenixIndexPartialBuildMapper.java b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/PhoenixIndexPartialBuildMapper.java
new file mode 100644
index 0000000..47a38a7
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/PhoenixIndexPartialBuildMapper.java
@@ -0,0 +1,182 @@
+/*
+ * 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.mapreduce.index;
+
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Properties;
+import java.util.UUID;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.mapreduce.TableMapper;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.phoenix.cache.ServerCacheClient;
+import org.apache.phoenix.coprocessor.BaseScannerRegionObserver;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
+import org.apache.phoenix.index.PhoenixIndexCodec;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.mapreduce.PhoenixJobCounters;
+import org.apache.phoenix.mapreduce.util.ConnectionUtil;
+import org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil;
+import org.apache.phoenix.query.ConnectionQueryServices;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.query.QueryServicesOptions;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Lists;
+
+/**
+ * Mapper that hands over rows from data table to the index table.
+ */
+public class PhoenixIndexPartialBuildMapper extends TableMapper<ImmutableBytesWritable, IntWritable> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(PhoenixIndexPartialBuildMapper.class);
+
+    private PhoenixConnection connection;
+
+    private DirectHTableWriter writer;
+
+    private int batchSize;
+
+    private List<Mutation> mutations ;
+    
+    private ImmutableBytesPtr maintainers;
+
+    @Override
+    protected void setup(final Context context) throws IOException, InterruptedException {
+        super.setup(context);
+        final Configuration configuration = context.getConfiguration();
+        writer = new DirectHTableWriter(configuration);
+
+        try {
+            final Properties overrideProps = new Properties();
+            String scn = configuration.get(PhoenixConfigurationUtil.CURRENT_SCN_VALUE);
+            String txScnValue = configuration.get(PhoenixConfigurationUtil.TX_SCN_VALUE);
+            if(txScnValue==null && scn!=null) {
+                overrideProps.put(PhoenixRuntime.CURRENT_SCN_ATTRIB, scn);
+            }
+            connection = ConnectionUtil.getOutputConnection(configuration, overrideProps).unwrap(PhoenixConnection.class);
+            connection.setAutoCommit(false);
+            // Get BatchSize
+            ConnectionQueryServices services = connection.getQueryServices();
+            int maxSize =
+                    services.getProps().getInt(QueryServices.MAX_MUTATION_SIZE_ATTRIB,
+                        QueryServicesOptions.DEFAULT_MAX_MUTATION_SIZE);
+            batchSize = Math.min(connection.getMutateBatchSize(), maxSize);
+            LOG.info("Mutation Batch Size = " + batchSize);
+            this.mutations = Lists.newArrayListWithExpectedSize(batchSize);
+            maintainers=new ImmutableBytesPtr(PhoenixConfigurationUtil.getIndexMaintainers(configuration));
+        } catch (SQLException e) {
+            throw new RuntimeException(e.getMessage());
+        } 
+    }
+
+    @Override
+    protected void map(ImmutableBytesWritable row, Result value, Context context)
+            throws IOException, InterruptedException {
+        context.getCounter(PhoenixJobCounters.INPUT_RECORDS).increment(1);
+        try {
+            byte[] attribValue = ByteUtil.copyKeyBytesIfNecessary(maintainers);
+            byte[] uuidValue = ServerCacheClient.generateId();
+            Put put = null;
+            Delete del = null;
+            for (Cell cell : value.rawCells()) {
+                if (KeyValue.Type.codeToType(cell.getTypeByte()) == KeyValue.Type.Put) {
+                    if (put == null) {
+                        put = new Put(CellUtil.cloneRow(cell));
+                        put.setAttribute(PhoenixIndexCodec.INDEX_UUID, uuidValue);
+                        put.setAttribute(PhoenixIndexCodec.INDEX_MD, attribValue);
+                        put.setAttribute(BaseScannerRegionObserver.IGNORE_NEWER_MUTATIONS, PDataType.TRUE_BYTES);
+                        mutations.add(put);
+                    }
+                    put.add(cell);
+                } else {
+                    if (del == null) {
+                        del = new Delete(CellUtil.cloneRow(cell));
+                        del.setAttribute(PhoenixIndexCodec.INDEX_UUID, uuidValue);
+                        del.setAttribute(PhoenixIndexCodec.INDEX_MD, attribValue);
+                        del.setAttribute(BaseScannerRegionObserver.IGNORE_NEWER_MUTATIONS, PDataType.TRUE_BYTES);
+                        mutations.add(del);
+                    }
+                    del.addDeleteMarker(cell);
+                }
+            }
+            // Write Mutation Batch
+            if (context.getCounter(PhoenixJobCounters.INPUT_RECORDS).getValue() % batchSize == 0) {
+                writeBatch(mutations, context);
+                mutations.clear();
+            }
+            // Make sure progress is reported to Application Master.
+            context.progress();
+        } catch (SQLException e) {
+            LOG.error(" Error {}  while read/write of a record ", e.getMessage());
+            context.getCounter(PhoenixJobCounters.FAILED_RECORDS).increment(1);
+            throw new RuntimeException(e);
+        }
+    }
+
+    private void writeBatch(List<Mutation> mutations, Context context)
+            throws IOException, SQLException, InterruptedException {
+        writer.write(mutations);
+        context.getCounter(PhoenixJobCounters.OUTPUT_RECORDS).increment(mutations.size());
+    }
+
+    @Override
+    protected void cleanup(Context context) throws IOException, InterruptedException {
+        try {
+            // Write the last & final Mutation Batch
+            if (!mutations.isEmpty()) {
+                writeBatch(mutations, context);
+            }
+            // We are writing some dummy key-value as map output here so that we commit only one
+            // output to reducer.
+            context.write(new ImmutableBytesWritable(UUID.randomUUID().toString().getBytes()),
+                new IntWritable(0));
+            super.cleanup(context);
+        } catch (SQLException e) {
+            LOG.error(" Error {}  while read/write of a record ", e.getMessage());
+            context.getCounter(PhoenixJobCounters.FAILED_RECORDS).increment(1);
+            throw new RuntimeException(e);
+        } finally {
+            if (connection != null) {
+                try {
+                    connection.close();
+                } catch (SQLException e) {
+                    LOG.error("Error {} while closing connection in the PhoenixIndexMapper class ",
+                        e.getMessage());
+                }
+            }
+            if (writer != null) {
+                writer.close();
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/83827cd8/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/util/PhoenixConfigurationUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/util/PhoenixConfigurationUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/util/PhoenixConfigurationUtil.java
index 2264acd..f3e4450 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/util/PhoenixConfigurationUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/util/PhoenixConfigurationUtil.java
@@ -30,7 +30,9 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
+import org.apache.hadoop.hbase.util.Base64;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.lib.db.DBInputFormat.NullDBWritable;
 import org.apache.hadoop.mapreduce.lib.db.DBWritable;
@@ -98,6 +100,12 @@ public final class PhoenixConfigurationUtil {
     
     public static final String MAPREDUCE_OUTPUT_CLUSTER_QUORUM = "phoneix.mapreduce.output.cluster.quorum";
 
+    public static final String INDEX_DISABLED_TIMESTAMP_VALUE = "phoenix.mr.index.disableTimestamp";
+
+    public static final String INDEX_MAINTAINERS = "phoenix.mr.index.maintainers";
+    
+    public static final String DISABLED_INDEXES = "phoenix.mr.index.disabledIndexes";
+
     public enum SchemaType {
         TABLE,
         QUERY;
@@ -428,4 +436,27 @@ public final class PhoenixConfigurationUtil {
     
         return ReflectionUtils.newInstance(processorClass, conf);
     }
+
+    public static byte[] getIndexMaintainers(final Configuration configuration){
+        Preconditions.checkNotNull(configuration);
+        return Base64.decode(configuration.get(INDEX_MAINTAINERS));
+    }
+    
+    public static void setIndexMaintainers(final Configuration configuration,
+            final ImmutableBytesWritable indexMetaDataPtr) {
+        Preconditions.checkNotNull(configuration);
+        Preconditions.checkNotNull(indexMetaDataPtr);
+        configuration.set(INDEX_MAINTAINERS, Base64.encodeBytes(indexMetaDataPtr.get()));
+    }
+    
+    public static void setDisableIndexes(Configuration configuration, String indexName) {
+        Preconditions.checkNotNull(configuration);
+        Preconditions.checkNotNull(indexName);
+        configuration.set(DISABLED_INDEXES, indexName);
+    }
+    
+    public static String getDisableIndexes(Configuration configuration) {
+        Preconditions.checkNotNull(configuration);
+        return configuration.get(DISABLED_INDEXES);
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/83827cd8/phoenix-core/src/main/java/org/apache/phoenix/parse/AlterIndexStatement.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/AlterIndexStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/AlterIndexStatement.java
index fcf817a..11328c2 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/AlterIndexStatement.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/AlterIndexStatement.java
@@ -23,12 +23,14 @@ public class AlterIndexStatement extends SingleTableStatement {
     private final String dataTableName;
     private final boolean ifExists;
     private final PIndexState indexState;
+    private boolean async;
 
-    public AlterIndexStatement(NamedTableNode indexTableNode, String dataTableName, boolean ifExists, PIndexState indexState) {
+    public AlterIndexStatement(NamedTableNode indexTableNode, String dataTableName, boolean ifExists, PIndexState indexState, boolean async) {
         super(indexTableNode,0);
         this.dataTableName = dataTableName;
         this.ifExists = ifExists;
         this.indexState = indexState;
+        this.async = async;
     }
 
     public String getTableName() {
@@ -48,4 +50,8 @@ public class AlterIndexStatement extends SingleTableStatement {
         return indexState;
     }
 
+    public boolean isAsync() {
+        return async;
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/83827cd8/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java
index 0c60e39..4b65c6a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java
@@ -374,8 +374,12 @@ public class ParseNodeFactory {
         return new DropIndexStatement(indexName, tableName, ifExists);
     }
 
+    public AlterIndexStatement alterIndex(NamedTableNode indexTableNode, String dataTableName, boolean ifExists, PIndexState state, boolean async) {
+        return new AlterIndexStatement(indexTableNode, dataTableName, ifExists, state, async);
+    }
+    
     public AlterIndexStatement alterIndex(NamedTableNode indexTableNode, String dataTableName, boolean ifExists, PIndexState state) {
-        return new AlterIndexStatement(indexTableNode, dataTableName, ifExists, state);
+        return new AlterIndexStatement(indexTableNode, dataTableName, ifExists, state, false);
     }
 
     public TraceStatement trace(boolean isTraceOn, double samplingRate) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/83827cd8/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
index 4a565be..1e002d2 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
@@ -127,10 +127,13 @@ public interface QueryServices extends SQLCloseable {
 
     // A master switch if to enable auto rebuild an index which failed to be updated previously
     public static final String INDEX_FAILURE_HANDLING_REBUILD_ATTRIB = "phoenix.index.failure.handling.rebuild";
+    public static final String INDEX_FAILURE_HANDLING_REBUILD_PERIOD = "phoenix.index.failure.handling.rebuild.period";
 
     // Time interval to check if there is an index needs to be rebuild
     public static final String INDEX_FAILURE_HANDLING_REBUILD_INTERVAL_ATTRIB =
         "phoenix.index.failure.handling.rebuild.interval";
+    
+    public static final String INDEX_FAILURE_HANDLING_REBUILD_NUMBER_OF_BATCHES_PER_TABLE = "phoenix.index.rebuild.batch.perTable";
 
     // A master switch if to block writes when index build failed
     public static final String INDEX_FAILURE_BLOCK_WRITE = "phoenix.index.failure.block.write";
@@ -227,6 +230,7 @@ public interface QueryServices extends SQLCloseable {
     
     public static final String CLIENT_CACHE_ENCODING = "phoenix.table.client.cache.encoding";
     public static final String AUTO_UPGRADE_ENABLED = "phoenix.autoupgrade.enabled";
+	
     /**
      * Get executor service used for parallel scans
      */

http://git-wip-us.apache.org/repos/asf/phoenix/blob/83827cd8/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
index 7f183e9..a3b9b32 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
@@ -27,6 +27,7 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.APPEND_ONLY_SCHEMA
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ARG_POSITION;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ARRAY_SIZE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ASYNC_CREATED_DATE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ASYNC_REBUILD_TIMESTAMP;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.AUTO_PARTITION_SEQ;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.BASE_COLUMN_COUNT;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.CLASS_NAME;
@@ -304,16 +305,27 @@ public class MetaDataClient {
                     TENANT_ID + "," +
                     TABLE_SCHEM + "," +
                     TABLE_NAME + "," +
-                    INDEX_STATE +
+                    INDEX_STATE + "," +
+                    ASYNC_REBUILD_TIMESTAMP + " " + PLong.INSTANCE.getSqlTypeName() +
+                    ") VALUES (?, ?, ?, ?, ?)";
+    
+    private static final String UPDATE_INDEX_REBUILD_ASYNC_STATE =
+            "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    ASYNC_REBUILD_TIMESTAMP + " " + PLong.INSTANCE.getSqlTypeName() +
                     ") VALUES (?, ?, ?, ?)";
+    
     private static final String UPDATE_INDEX_STATE_TO_ACTIVE =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
                     TENANT_ID + "," +
                     TABLE_SCHEM + "," +
                     TABLE_NAME + "," +
                     INDEX_STATE + "," +
-                    INDEX_DISABLE_TIMESTAMP +
-                    ") VALUES (?, ?, ?, ?, ?)";
+                    INDEX_DISABLE_TIMESTAMP +","+
+                    ASYNC_REBUILD_TIMESTAMP + " " + PLong.INSTANCE.getSqlTypeName() +
+                    ") VALUES (?, ?, ?, ?, ?, ?)";
     //TODO: merge INSERT_COLUMN_CREATE_TABLE and INSERT_COLUMN_ALTER_TABLE column when
     // the new major release is out.
     private static final String INSERT_COLUMN_CREATE_TABLE =
@@ -3469,7 +3481,13 @@ public class MetaDataClient {
             String dataTableName = statement.getTableName();
             String schemaName = statement.getTable().getName().getSchemaName();
             String indexName = statement.getTable().getName().getTableName();
+            boolean isAsync = statement.isAsync();
             PIndexState newIndexState = statement.getIndexState();
+            if (isAsync && newIndexState != PIndexState.REBUILD) { throw new SQLExceptionInfo.Builder(
+                    SQLExceptionCode.ASYNC_NOT_ALLOWED)
+                            .setMessage(" ASYNC building of index is allowed only with REBUILD index state")
+                            .setSchemaName(schemaName).setTableName(indexName).build().buildException(); }
+
             if (newIndexState == PIndexState.REBUILD) {
                 newIndexState = PIndexState.BUILDING;
             }
@@ -3480,15 +3498,16 @@ public class MetaDataClient {
             try {
                 if(newIndexState == PIndexState.ACTIVE){
                     tableUpsert = connection.prepareStatement(UPDATE_INDEX_STATE_TO_ACTIVE);
-                } else {
+                }else{
                     tableUpsert = connection.prepareStatement(UPDATE_INDEX_STATE);
                 }
                 tableUpsert.setString(1, connection.getTenantId() == null ? null : connection.getTenantId().getString());
                 tableUpsert.setString(2, schemaName);
                 tableUpsert.setString(3, indexName);
                 tableUpsert.setString(4, newIndexState.getSerializedValue());
+                tableUpsert.setLong(5, 0);
                 if(newIndexState == PIndexState.ACTIVE){
-                    tableUpsert.setLong(5, 0);
+                    tableUpsert.setLong(6, 0);
                 }
                 tableUpsert.execute();
             } finally {
@@ -3515,9 +3534,25 @@ public class MetaDataClient {
                     addTableToCache(result);
                     // Set so that we get the table below with the potentially modified rowKeyOrderOptimizable flag set
                     indexRef.setTable(result.getTable());
+                    if (newIndexState == PIndexState.BUILDING && isAsync) {
+                        try {
+                            tableUpsert = connection.prepareStatement(UPDATE_INDEX_REBUILD_ASYNC_STATE);
+                            tableUpsert.setString(1,
+                                    connection.getTenantId() == null ? null : connection.getTenantId().getString());
+                            tableUpsert.setString(2, schemaName);
+                            tableUpsert.setString(3, indexName);
+                            tableUpsert.setLong(4, result.getTable().getTimeStamp());
+                            tableUpsert.execute();
+                            connection.commit();
+                        } finally {
+                            if (tableUpsert != null) {
+                                tableUpsert.close();
+                            }
+                        }
+                    }
                 }
             }
-            if (newIndexState == PIndexState.BUILDING) {
+            if (newIndexState == PIndexState.BUILDING && !isAsync) {
                 PTable index = indexRef.getTable();
                 // First delete any existing rows of the index
                 Long scn = connection.getSCN();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/83827cd8/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
index 9622880..2a0f3b9 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
@@ -42,9 +42,13 @@ import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.coprocessor.Batch;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.ipc.BlockingRpcCallback;
+import org.apache.hadoop.hbase.ipc.ServerRpcController;
+import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.io.WritableUtils;
@@ -54,6 +58,10 @@ import org.apache.phoenix.compile.IndexStatementRewriter;
 import org.apache.phoenix.compile.StatementContext;
 import org.apache.phoenix.compile.WhereCompiler;
 import org.apache.phoenix.coprocessor.BaseScannerRegionObserver;
+import org.apache.phoenix.coprocessor.MetaDataProtocol.MetaDataMutationResult;
+import org.apache.phoenix.coprocessor.generated.MetaDataProtos.MetaDataResponse;
+import org.apache.phoenix.coprocessor.generated.MetaDataProtos.MetaDataService;
+import org.apache.phoenix.coprocessor.generated.MetaDataProtos.UpdateIndexStateRequest;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.exception.SQLExceptionInfo;
 import org.apache.phoenix.execute.TupleProjector;
@@ -67,10 +75,12 @@ import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.hbase.index.util.KeyValueBuilder;
 import org.apache.phoenix.index.IndexMaintainer;
 import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.jdbc.PhoenixStatement;
 import org.apache.phoenix.parse.ParseNode;
 import org.apache.phoenix.parse.SQLParser;
 import org.apache.phoenix.parse.SelectStatement;
+import org.apache.phoenix.protobuf.ProtobufUtil;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.ColumnFamilyNotFoundException;
 import org.apache.phoenix.schema.ColumnNotFoundException;
@@ -78,6 +88,7 @@ import org.apache.phoenix.schema.ColumnRef;
 import org.apache.phoenix.schema.KeyValueSchema;
 import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PColumnFamily;
+import org.apache.phoenix.schema.PIndexState;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.TableNotFoundException;
@@ -87,12 +98,13 @@ import org.apache.phoenix.schema.tuple.Tuple;
 import org.apache.phoenix.schema.types.PBinary;
 import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.types.PDecimal;
+import org.apache.phoenix.schema.types.PLong;
 import org.apache.phoenix.schema.types.PVarbinary;
 import org.apache.phoenix.schema.types.PVarchar;
+import org.apache.tephra.TxConstants;
 
 import com.google.common.collect.Lists;
-
-import org.apache.tephra.TxConstants;
+import com.google.protobuf.ServiceException;
 
 public class IndexUtil {
     public static final String INDEX_COLUMN_NAME_SEP = ":";
@@ -673,13 +685,48 @@ public class IndexUtil {
             HConstants.NO_NONCE, HConstants.NO_NONCE);
     }
 
+    public static MetaDataMutationResult disableIndexWithTimestamp(String indexTableName, long minTimeStamp,
+            HTableInterface metaTable, boolean blockWriteRebuildIndex) throws ServiceException, Throwable {
+        byte[] indexTableKey = SchemaUtil.getTableKeyFromFullName(indexTableName);
+        // Mimic the Put that gets generated by the client on an update of the index state
+        Put put = new Put(indexTableKey);
+        if (blockWriteRebuildIndex)
+            put.add(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES, PhoenixDatabaseMetaData.INDEX_STATE_BYTES,
+                    PIndexState.ACTIVE.getSerializedBytes());
+        else
+            put.add(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES, PhoenixDatabaseMetaData.INDEX_STATE_BYTES,
+                    PIndexState.DISABLE.getSerializedBytes());
+        put.add(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES, PhoenixDatabaseMetaData.INDEX_DISABLE_TIMESTAMP_BYTES,
+                PLong.INSTANCE.toBytes(minTimeStamp));
+        put.add(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES, PhoenixDatabaseMetaData.ASYNC_REBUILD_TIMESTAMP_BYTES,
+                PLong.INSTANCE.toBytes(0));
+        final List<Mutation> tableMetadata = Collections.<Mutation> singletonList(put);
+
+        final Map<byte[], MetaDataResponse> results = metaTable.coprocessorService(MetaDataService.class, indexTableKey,
+                indexTableKey, new Batch.Call<MetaDataService, MetaDataResponse>() {
+                    @Override
+                    public MetaDataResponse call(MetaDataService instance) throws IOException {
+                        ServerRpcController controller = new ServerRpcController();
+                        BlockingRpcCallback<MetaDataResponse> rpcCallback = new BlockingRpcCallback<MetaDataResponse>();
+                        UpdateIndexStateRequest.Builder builder = UpdateIndexStateRequest.newBuilder();
+                        for (Mutation m : tableMetadata) {
+                            MutationProto mp = ProtobufUtil.toProto(m);
+                            builder.addTableMetadataMutations(mp.toByteString());
+                        }
+                        instance.updateIndexState(controller, builder.build(), rpcCallback);
+                        if (controller.getFailedOn() != null) { throw controller.getFailedOn(); }
+                        return rpcCallback.get();
+                    }
+                });
+        if (results.isEmpty()) { throw new IOException("Didn't get expected result size"); }
+        MetaDataResponse tmpResponse = results.values().iterator().next();
+        return MetaDataMutationResult.constructFromProto(tmpResponse);
+    }
+
     public static boolean matchingSplitKeys(byte[][] splitKeys1, byte[][] splitKeys2) throws IOException {
-        if (splitKeys1 != null && splitKeys2 != null
-                && splitKeys1.length == splitKeys2.length) {
+        if (splitKeys1 != null && splitKeys2 != null && splitKeys1.length == splitKeys2.length) {
             for (int i = 0; i < splitKeys1.length; i++) {
-                if (Bytes.compareTo(splitKeys1[i], splitKeys2[i]) != 0) {
-                    return false;
-                }
+                if (Bytes.compareTo(splitKeys1[i], splitKeys2[i]) != 0) { return false; }
             }
         } else {
             return false;


[08/26] phoenix git commit: PHOENIX-3544 Fix UNSIGNED_DATE and UNSIGNED_TIME type in phoenix-spark

Posted by ma...@apache.org.
PHOENIX-3544 Fix UNSIGNED_DATE and UNSIGNED_TIME type in phoenix-spark


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

Branch: refs/heads/calcite
Commit: b32699505d18b5b93b4072e0b146df6b8ddaa7fb
Parents: e906841
Author: Ankit Singhal <an...@gmail.com>
Authored: Mon Dec 26 16:54:13 2016 +0530
Committer: Ankit Singhal <an...@gmail.com>
Committed: Mon Dec 26 16:54:13 2016 +0530

----------------------------------------------------------------------
 phoenix-spark/src/it/resources/globalSetup.sql               | 5 ++++-
 .../it/scala/org/apache/phoenix/spark/PhoenixSparkIT.scala   | 8 ++++++++
 .../src/main/scala/org/apache/phoenix/spark/PhoenixRDD.scala | 5 ++---
 3 files changed, 14 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/b3269950/phoenix-spark/src/it/resources/globalSetup.sql
----------------------------------------------------------------------
diff --git a/phoenix-spark/src/it/resources/globalSetup.sql b/phoenix-spark/src/it/resources/globalSetup.sql
index 72f8620..28eb0f7 100644
--- a/phoenix-spark/src/it/resources/globalSetup.sql
+++ b/phoenix-spark/src/it/resources/globalSetup.sql
@@ -57,4 +57,7 @@ UPSERT INTO "small" VALUES ('key1', 'foo', 10000)
 UPSERT INTO "small" VALUES ('key2', 'bar', 20000)
 UPSERT INTO "small" VALUES ('key3', 'xyz', 30000)
 
-CREATE TABLE MULTITENANT_TEST_TABLE (TENANT_ID VARCHAR NOT NULL, ORGANIZATION_ID VARCHAR, GLOBAL_COL1 VARCHAR  CONSTRAINT pk PRIMARY KEY (TENANT_ID, ORGANIZATION_ID)) MULTI_TENANT=true
\ No newline at end of file
+CREATE TABLE MULTITENANT_TEST_TABLE (TENANT_ID VARCHAR NOT NULL, ORGANIZATION_ID VARCHAR, GLOBAL_COL1 VARCHAR  CONSTRAINT pk PRIMARY KEY (TENANT_ID, ORGANIZATION_ID)) MULTI_TENANT=true
+CREATE TABLE IF NOT EXISTS GIGANTIC_TABLE (ID INTEGER PRIMARY KEY,unsig_id UNSIGNED_INT,big_id BIGINT,unsig_long_id UNSIGNED_LONG,tiny_id TINYINT,unsig_tiny_id UNSIGNED_TINYINT,small_id SMALLINT,unsig_small_id UNSIGNED_SMALLINT,float_id FLOAT,unsig_float_id UNSIGNED_FLOAT,double_id DOUBLE,unsig_double_id UNSIGNED_DOUBLE,decimal_id DECIMAL,boolean_id BOOLEAN,time_id TIME,date_id DATE,timestamp_id TIMESTAMP,unsig_time_id UNSIGNED_TIME,unsig_date_id UNSIGNED_DATE,unsig_timestamp_id UNSIGNED_TIMESTAMP,varchar_id VARCHAR (30),char_id CHAR (30),binary_id BINARY (100),varbinary_id VARBINARY (100))
+ CREATE TABLE IF NOT EXISTS OUTPUT_GIGANTIC_TABLE (ID INTEGER PRIMARY KEY,unsig_id UNSIGNED_INT,big_id BIGINT,unsig_long_id UNSIGNED_LONG,tiny_id TINYINT,unsig_tiny_id UNSIGNED_TINYINT,small_id SMALLINT,unsig_small_id UNSIGNED_SMALLINT,float_id FLOAT,unsig_float_id UNSIGNED_FLOAT,double_id DOUBLE,unsig_double_id UNSIGNED_DOUBLE,decimal_id DECIMAL,boolean_id BOOLEAN,time_id TIME,date_id DATE,timestamp_id TIMESTAMP,unsig_time_id UNSIGNED_TIME,unsig_date_id UNSIGNED_DATE,unsig_timestamp_id UNSIGNED_TIMESTAMP,varchar_id VARCHAR (30),char_id CHAR (30),binary_id BINARY (100),varbinary_id VARBINARY (100))
+ upsert into GIGANTIC_TABLE values(0,2,3,4,-5,6,7,8,9.3,10.4,11.5,12.6,13.7,true,CURRENT_TIME(),CURRENT_DATE(),CURRENT_TIME(),CURRENT_TIME(),CURRENT_DATE(),CURRENT_TIME(),'This is random textA','a','a','a')

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b3269950/phoenix-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkIT.scala
----------------------------------------------------------------------
diff --git a/phoenix-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkIT.scala b/phoenix-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkIT.scala
index dbcc4f1..9def354 100644
--- a/phoenix-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkIT.scala
+++ b/phoenix-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkIT.scala
@@ -632,5 +632,13 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT {
     val epoch = new Date().getTime
     assert(Math.abs(epoch - time) < 86400000)
   }
+  test("can read all Phoenix data types") {
+    val sqlContext = new SQLContext(sc)
+    val df = sqlContext.load("org.apache.phoenix.spark", Map("table" -> "GIGANTIC_TABLE",
+      "zkUrl" -> quorumAddress))
+    df.save("org.apache.phoenix.spark",SaveMode.Overwrite, Map("table" -> "OUTPUT_GIGANTIC_TABLE",
+     "zkUrl" -> quorumAddress))
+    df.count() shouldEqual 1
+  }
 
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b3269950/phoenix-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRDD.scala
----------------------------------------------------------------------
diff --git a/phoenix-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRDD.scala b/phoenix-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRDD.scala
index 204a7ef..63289a0 100644
--- a/phoenix-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRDD.scala
+++ b/phoenix-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRDD.scala
@@ -129,11 +129,10 @@ class PhoenixRDD(sc: SparkContext, table: String, columns: Seq[String],
       // Create a sequence of column data
       val rowSeq = columns.map { case (name, sqlType) =>
         val res = pr.resultMap(name)
-
           // Special handling for data types
-          if (dateAsTimestamp && sqlType == 91) { // 91 is the defined type for Date
+          if (dateAsTimestamp && (sqlType == 91 || sqlType == 19)) { // 91 is the defined type for Date and 19 for UNSIGNED_DATE
             new java.sql.Timestamp(res.asInstanceOf[java.sql.Date].getTime)
-          } else if (sqlType == 92) { // 92 is the defined type for Time
+          } else if (sqlType == 92 || sqlType == 18) { // 92 is the defined type for Time and 18 for UNSIGNED_TIME
             new java.sql.Timestamp(res.asInstanceOf[java.sql.Time].getTime)
           } else {
             res


[11/26] phoenix git commit: PHOENIX-3557 Provide a good error message when an invalid test pattern was provided

Posted by ma...@apache.org.
PHOENIX-3557 Provide a good error message when an invalid test pattern was provided


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

Branch: refs/heads/calcite
Commit: e4c6f5cde940931bb4be82c7aa7959de58857ad8
Parents: a0e5efc
Author: Josh Elser <el...@apache.org>
Authored: Thu Jan 5 13:37:59 2017 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Thu Jan 5 22:19:13 2017 -0500

----------------------------------------------------------------------
 .../org/apache/phoenix/end2end/End2EndTestDriver.java   | 12 +++++++++++-
 1 file changed, 11 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/e4c6f5cd/phoenix-core/src/it/java/org/apache/phoenix/end2end/End2EndTestDriver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/End2EndTestDriver.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/End2EndTestDriver.java
index a505c1e..feb506f 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/End2EndTestDriver.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/End2EndTestDriver.java
@@ -25,6 +25,7 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 import java.util.regex.Pattern;
+import java.util.regex.PatternSyntaxException;
 
 import org.apache.commons.cli.CommandLine;
 import org.apache.hadoop.hbase.ClassFinder;
@@ -38,6 +39,8 @@ import org.junit.runner.Description;
 import org.junit.runner.JUnitCore;
 import org.junit.runner.Result;
 import org.junit.runner.notification.Failure;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * This class drives the End2End tests suite execution against an
@@ -45,6 +48,7 @@ import org.junit.runner.notification.Failure;
  */
 public class End2EndTestDriver extends AbstractHBaseTool {
     
+    private static final Logger LOG = LoggerFactory.getLogger(End2EndTestDriver.class);
     private static final String SHORT_REGEX_ARG = "r";
     private static final String SKIP_TESTS = "n";
     
@@ -73,7 +77,13 @@ public class End2EndTestDriver extends AbstractHBaseTool {
       }
 
       public void setPattern(String pattern) {
-        testFilterRe = Pattern.compile(pattern);
+        try {
+          testFilterRe = Pattern.compile(pattern);
+        } catch (PatternSyntaxException e) {
+          LOG.error("Failed to find tests using pattern '" + pattern
+              + "'. Is it a valid Java regular expression?", e);
+          throw e;
+        }
       }
 
       @Override


[18/26] phoenix git commit: PHOENIX-3468 Double quote SYSTEM, USER, DATE keywords in IT tests(Rajeshbabu)

Posted by ma...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/895cb1df/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertValuesIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertValuesIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertValuesIT.java
index eb81ae3..64935d2 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertValuesIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertValuesIT.java
@@ -57,7 +57,7 @@ public class UpsertValuesIT extends BaseClientManagedTimeIT {
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 10));
         Connection conn = DriverManager.getConnection(getUrl(), props);
         conn.setAutoCommit(true);
-        PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " + TestUtil.PTSDB_NAME + " (inst,host,date) VALUES(?,'b',CURRENT_DATE())");
+        PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " + TestUtil.PTSDB_NAME + " (inst,host,\"DATE\") VALUES(?,'b',CURRENT_DATE())");
         stmt.setString(1, "a");
         stmt.execute();
         stmt.execute();
@@ -316,8 +316,7 @@ public class UpsertValuesIT extends BaseClientManagedTimeIT {
         PreparedStatement stmt = null;
         try {
             conn = DriverManager.getConnection(getUrl(), props);
-            stmt = conn.prepareStatement("create table UpsertTimestamp (a integer NOT NULL, t timestamp NOT NULL CONSTRAINT pk PRIMARY KEY (a, t))");
-            stmt.execute();
+            conn.createStatement().execute("create table UpsertTimestamp (a integer NOT NULL, t timestamp NOT NULL CONSTRAINT pk PRIMARY KEY (a, t))");
         } finally {
             closeStmtAndConn(stmt, conn);
         }
@@ -357,8 +356,7 @@ public class UpsertValuesIT extends BaseClientManagedTimeIT {
         PreparedStatement stmt = null;
         try {
             conn = DriverManager.getConnection(getUrl(), props);
-            stmt = conn.prepareStatement("create table UpsertTimestamp (a integer NOT NULL, t timestamp NOT NULL CONSTRAINT pk PRIMARY KEY (a, t))");
-            stmt.execute();
+            conn.createStatement().execute("create table UpsertTimestamp (a integer NOT NULL, t timestamp NOT NULL CONSTRAINT pk PRIMARY KEY (a, t))");
         } finally {
             closeStmtAndConn(stmt, conn);
         }
@@ -455,8 +453,7 @@ public class UpsertValuesIT extends BaseClientManagedTimeIT {
         PreparedStatement stmt = null;
         try {
             conn = DriverManager.getConnection(getUrl(), props);
-            stmt = conn.prepareStatement("create table UpsertFloat (k varchar primary key, v float)");
-            stmt.execute();
+            conn.createStatement().execute("create table UpsertFloat (k varchar primary key, v float)");
         } finally {
             closeStmtAndConn(stmt, conn);
         }
@@ -496,8 +493,7 @@ public class UpsertValuesIT extends BaseClientManagedTimeIT {
         String tableName = BaseTest.generateUniqueName();
         try {
             conn = DriverManager.getConnection(getUrl(), props);
-            pstmt = conn.prepareStatement("create table " + tableName + " (k varchar primary key, v integer)");
-            pstmt.execute();
+            conn.createStatement().execute("create table " + tableName + " (k varchar primary key, v integer)");
         } finally {
             closeStmtAndConn(pstmt, conn);
         }
@@ -587,8 +583,7 @@ public class UpsertValuesIT extends BaseClientManagedTimeIT {
         PreparedStatement stmt = null;
         try {
             conn = DriverManager.getConnection(getUrl(), props);
-            stmt = conn.prepareStatement("create table UpsertTimestamp (k varchar, v unsigned_date not null, constraint pk primary key (k,v desc))");
-            stmt.execute();
+            conn.createStatement().execute("create table UpsertTimestamp (k varchar, v unsigned_date not null, constraint pk primary key (k,v desc))");
         } finally {
             closeStmtAndConn(stmt, conn);
         }
@@ -629,9 +624,8 @@ public class UpsertValuesIT extends BaseClientManagedTimeIT {
         PreparedStatement stmt = null;
         try {
             conn = DriverManager.getConnection(getUrl(), props);
-            stmt = conn.prepareStatement("create table UpsertDateVal (k varchar, v date not null, t timestamp" +
+            conn.createStatement().execute("create table UpsertDateVal (k varchar, v date not null, t timestamp" +
                     ", tt time constraint pk primary key (k,v desc))");
-            stmt.execute();
         } finally {
             closeStmtAndConn(stmt, conn);
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/895cb1df/phoenix-core/src/test/java/org/apache/phoenix/compile/JoinQueryCompilerTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/compile/JoinQueryCompilerTest.java b/phoenix-core/src/test/java/org/apache/phoenix/compile/JoinQueryCompilerTest.java
index f62e716..8c1f536 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/compile/JoinQueryCompilerTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/compile/JoinQueryCompilerTest.java
@@ -58,14 +58,14 @@ public class JoinQueryCompilerTest extends BaseConnectionlessQueryTest {
                     "    \"item_id\" varchar(10), " +
                     "    price integer, " +
                     "    quantity integer, " +
-                    "    date timestamp)");
+                    "    \"date\" timestamp)");
             conn.createStatement().execute("create table " + JOIN_CUSTOMER_TABLE_FULL_NAME +
                     "   (\"customer_id\" varchar(10) not null primary key, " +
                     "    name varchar, " +
                     "    phone varchar(12), " +
                     "    address varchar, " +
                     "    loc_id varchar(5), " +
-                    "    date date)");
+                    "    \"date\" date)");
             conn.createStatement().execute("create table " + JOIN_ITEM_TABLE_FULL_NAME +
                     "   (\"item_id\" varchar(10) not null primary key, " +
                     "    name varchar, " +
@@ -86,7 +86,7 @@ public class JoinQueryCompilerTest extends BaseConnectionlessQueryTest {
     @Test
     public void testExplainPlan() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        String query = "EXPLAIN SELECT s.\"supplier_id\", \"order_id\", c.name, i.name, quantity, o.\"DATE\" FROM " + JOIN_ORDER_TABLE_FULL_NAME + " o LEFT JOIN "
+        String query = "EXPLAIN SELECT s.\"supplier_id\", \"order_id\", c.name, i.name, quantity, o.\"date\" FROM " + JOIN_ORDER_TABLE_FULL_NAME + " o LEFT JOIN "
     	+ JOIN_CUSTOMER_TABLE_FULL_NAME + " c ON o.\"customer_id\" = c.\"customer_id\" AND c.name LIKE 'C%' LEFT JOIN " 
     	+ JOIN_ITEM_TABLE_FULL_NAME + " i ON o.\"item_id\" = i.\"item_id\" RIGHT JOIN " 
     	+ JOIN_SUPPLIER_TABLE_FULL_NAME + " s ON s.\"supplier_id\" = i.\"supplier_id\" WHERE i.name LIKE 'T%'";

http://git-wip-us.apache.org/repos/asf/phoenix/blob/895cb1df/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java b/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java
index 8daefc0..04cd93f 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java
@@ -17,6 +17,8 @@
  */
 package org.apache.phoenix.compile;
 
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_STATS_TABLE;
 import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
 import static org.apache.phoenix.util.TestUtil.assertDegenerate;
 import static org.junit.Assert.assertArrayEquals;
@@ -500,7 +502,7 @@ public class QueryCompilerTest extends BaseConnectionlessQueryTest {
             "SELECT count(1) FROM atable GROUP BY organization_id,substr(entity_id,1,3),entity_id",
             "SELECT count(1) FROM atable GROUP BY entity_id,organization_id",
             "SELECT count(1) FROM atable GROUP BY substr(entity_id,1,3),organization_id",
-            "SELECT count(1) FROM ptsdb GROUP BY host,inst,round(date,'HOUR')",
+            "SELECT count(1) FROM ptsdb GROUP BY host,inst,round(\"DATE\",'HOUR')",
             "SELECT count(1) FROM atable GROUP BY organization_id",
         };
         List<Object> binds = Collections.emptyList();
@@ -1539,7 +1541,7 @@ public class QueryCompilerTest extends BaseConnectionlessQueryTest {
         try{
             Statement statement = conn1.createStatement();
             statement.execute("create table example (id integer not null,fn varchar,"
-                    + "ln varchar constraint pk primary key(id)) DEFAULT_COLUMN_FAMILY='F'");
+                    + "\"ln\" varchar constraint pk primary key(id)) DEFAULT_COLUMN_FAMILY='F'");
             try {
                 statement.execute("create local index my_idx on example (fn) DEFAULT_COLUMN_FAMILY='F'");
                 fail();
@@ -1701,7 +1703,7 @@ public class QueryCompilerTest extends BaseConnectionlessQueryTest {
             Integer.MAX_VALUE + "," + Long.MAX_VALUE + "," +
             (Integer.MAX_VALUE - 1) + "," + (Long.MAX_VALUE - 1) + "," +
             ((long)Integer.MAX_VALUE + 1) + "," + oneMoreThanMaxLong +
-        " FROM " + PhoenixDatabaseMetaData.SYSTEM_STATS_NAME + " LIMIT 1";
+        " FROM " + "\""+ SYSTEM_CATALOG_SCHEMA + "\".\"" + SYSTEM_STATS_TABLE + "\"" + " LIMIT 1";
         List<Object> binds = Collections.emptyList();
         QueryPlan plan = getQueryPlan(query, binds);
         RowProjector p = plan.getProjector();


[25/26] phoenix git commit: PHOENIX-3614 Use unique name for index in TransactionIT#testNonTxToTxTable

Posted by ma...@apache.org.
PHOENIX-3614 Use unique name for index in TransactionIT#testNonTxToTxTable


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

Branch: refs/heads/calcite
Commit: 069c371aec23a6b87519615eda7eafb895cb1af0
Parents: a44d317
Author: Samarth <sa...@salesforce.com>
Authored: Thu Jan 26 15:49:33 2017 -0800
Committer: Samarth <sa...@salesforce.com>
Committed: Thu Jan 26 15:49:33 2017 -0800

----------------------------------------------------------------------
 .../src/it/java/org/apache/phoenix/tx/TransactionIT.java      | 7 ++++---
 1 file changed, 4 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/069c371a/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java
index bde5cc8..1399f6c 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java
@@ -255,6 +255,7 @@ public class TransactionIT extends ParallelStatsDisabledIT {
     @Test
     public void testNonTxToTxTable() throws Exception {
         String nonTxTableName = generateUniqueName();
+        String indexName = generateUniqueName() + "_IDX";
 
         Connection conn = DriverManager.getConnection(getUrl());
         conn.createStatement().execute("CREATE TABLE " + nonTxTableName + "(k INTEGER PRIMARY KEY, v VARCHAR)");
@@ -263,7 +264,7 @@ public class TransactionIT extends ParallelStatsDisabledIT {
         conn.createStatement().execute("UPSERT INTO " + nonTxTableName + " VALUES (3, 'b')");
         conn.commit();
         
-        conn.createStatement().execute("CREATE INDEX IDX ON " + nonTxTableName + "(v)");
+        conn.createStatement().execute("CREATE INDEX " + indexName + " ON " + nonTxTableName + "(v)");
         // Reset empty column value to an empty value like it is pre-transactions
         HTableInterface htable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes( nonTxTableName));
         List<Put>puts = Lists.newArrayList(new Put(PInteger.INSTANCE.toBytes(1)), new Put(PInteger.INSTANCE.toBytes(2)), new Put(PInteger.INSTANCE.toBytes(3)));
@@ -276,7 +277,7 @@ public class TransactionIT extends ParallelStatsDisabledIT {
         
         htable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes( nonTxTableName));
         assertTrue(htable.getTableDescriptor().getCoprocessors().contains(PhoenixTransactionalProcessor.class.getName()));
-        htable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes("IDX"));
+        htable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes(indexName));
         assertTrue(htable.getTableDescriptor().getCoprocessors().contains(PhoenixTransactionalProcessor.class.getName()));
 
         conn.createStatement().execute("UPSERT INTO " + nonTxTableName + " VALUES (4, 'c')");
@@ -289,7 +290,7 @@ public class TransactionIT extends ParallelStatsDisabledIT {
         
         conn.createStatement().execute("UPSERT INTO " + nonTxTableName + " VALUES (5, 'd')");
         rs = conn.createStatement().executeQuery("SELECT k FROM " + nonTxTableName);
-        assertTrue(conn.unwrap(PhoenixConnection.class).getTable(new PTableKey(null, "IDX")).isTransactional());
+        assertTrue(conn.unwrap(PhoenixConnection.class).getTable(new PTableKey(null, indexName)).isTransactional());
         assertTrue(rs.next());
         assertEquals(1,rs.getInt(1));
         assertTrue(rs.next());


[19/26] phoenix git commit: PHOENIX-3468 Double quote SYSTEM, USER, DATE keywords in IT tests(Rajeshbabu)

Posted by ma...@apache.org.
PHOENIX-3468 Double quote SYSTEM, USER,DATE keywords in IT tests(Rajeshbabu)


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

Branch: refs/heads/calcite
Commit: 895cb1dfca01a9cfffe7d25eaa524f944cb53952
Parents: a675211
Author: Rajeshbabu Chintaguntla <ra...@apache.org>
Authored: Fri Jan 20 18:39:39 2017 +0530
Committer: Rajeshbabu Chintaguntla <ra...@apache.org>
Committed: Fri Jan 20 18:39:39 2017 +0530

----------------------------------------------------------------------
 .../AlterMultiTenantTableWithViewsIT.java       |  2 +-
 .../apache/phoenix/end2end/AlterTableIT.java    | 16 ++---
 .../phoenix/end2end/AlterTableWithViewsIT.java  |  2 +-
 .../phoenix/end2end/ArithmeticQueryIT.java      |  2 +-
 .../phoenix/end2end/ArrayFillFunctionIT.java    | 28 ++++-----
 .../end2end/BaseTenantSpecificTablesIT.java     |  4 +-
 .../phoenix/end2end/CoalesceFunctionIT.java     | 60 +++++++++---------
 .../org/apache/phoenix/end2end/DateTimeIT.java  |  6 +-
 .../org/apache/phoenix/end2end/DeleteIT.java    | 26 ++++----
 .../phoenix/end2end/EvaluationOfORIT.java       |  5 +-
 .../apache/phoenix/end2end/GroupByCaseIT.java   | 10 +--
 .../org/apache/phoenix/end2end/GroupByIT.java   |  6 +-
 .../apache/phoenix/end2end/HashJoinMoreIT.java  | 47 +++++++-------
 .../phoenix/end2end/SortMergeJoinMoreIT.java    | 37 +++++------
 .../org/apache/phoenix/end2end/SortOrderIT.java |  8 +--
 .../end2end/TenantSpecificTablesDDLIT.java      | 12 ++--
 .../end2end/TenantSpecificTablesDMLIT.java      | 66 ++++++++++----------
 .../apache/phoenix/end2end/UpsertValuesIT.java  | 20 +++---
 .../phoenix/compile/JoinQueryCompilerTest.java  |  6 +-
 .../phoenix/compile/QueryCompilerTest.java      |  8 ++-
 20 files changed, 178 insertions(+), 193 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/895cb1df/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterMultiTenantTableWithViewsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterMultiTenantTableWithViewsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterMultiTenantTableWithViewsIT.java
index adadca7..d6f3a7f 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterMultiTenantTableWithViewsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterMultiTenantTableWithViewsIT.java
@@ -656,7 +656,7 @@ public class AlterMultiTenantTableWithViewsIT extends ParallelStatsDisabledIT {
     }
 
     public static void assertTableDefinition(Connection conn, String tableName, PTableType tableType, String parentTableName, int sequenceNumber, int columnCount, int baseColumnCount, String... columnName) throws Exception {
-        PreparedStatement p = conn.prepareStatement("SELECT * FROM SYSTEM.CATALOG WHERE TABLE_NAME=? AND TABLE_TYPE=?");
+        PreparedStatement p = conn.prepareStatement("SELECT * FROM \"SYSTEM\".\"CATALOG\" WHERE TABLE_NAME=? AND TABLE_TYPE=?");
         p.setString(1, tableName);
         p.setString(2, tableType.getSerializedValue());
         ResultSet rs = p.executeQuery();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/895cb1df/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java
index f51c3cb..fee7316 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java
@@ -996,7 +996,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
         assertEquals("COL3",rs.getString(4));
         assertFalse(rs.next());
         
-        rs = conn1.createStatement().executeQuery("SELECT COLUMN_COUNT FROM SYSTEM.CATALOG\n"
+        rs = conn1.createStatement().executeQuery("SELECT COLUMN_COUNT FROM \"SYSTEM\".\"CATALOG\"\n"
                 + "WHERE TENANT_ID IS NULL AND\n"
                 + "(TABLE_SCHEM, TABLE_NAME) = ('" + schemaName + "','"+ dataTableName + "') AND\n"
                 + "COLUMN_FAMILY IS NULL AND COLUMN_NAME IS NULL");
@@ -1004,7 +1004,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
         assertEquals(4,rs.getInt(1));
         assertFalse(rs.next());
 
-        rs = conn1.createStatement().executeQuery("SELECT COLUMN_COUNT FROM SYSTEM.CATALOG\n"
+        rs = conn1.createStatement().executeQuery("SELECT COLUMN_COUNT FROM \"SYSTEM\".\"CATALOG\"\n"
                 + "WHERE TENANT_ID IS NULL AND\n"
                 + "(TABLE_SCHEM, TABLE_NAME) = ('" + schemaName + "','"+ indexTableName + "') AND\n"
                 + "COLUMN_FAMILY IS NULL AND COLUMN_NAME IS NULL");
@@ -1051,7 +1051,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
         assertEquals("COL4",rs.getString(4));
         assertFalse(rs.next());
         
-        rs = conn1.createStatement().executeQuery("SELECT COLUMN_COUNT FROM SYSTEM.CATALOG\n"
+        rs = conn1.createStatement().executeQuery("SELECT COLUMN_COUNT FROM \"SYSTEM\".\"CATALOG\"\n"
                 + "WHERE TENANT_ID IS NULL AND\n"
                 + "(TABLE_SCHEM, TABLE_NAME) = ('" + schemaName + "','"+ dataTableName + "') AND\n"
                 + "COLUMN_FAMILY IS NULL AND COLUMN_NAME IS NULL");
@@ -1059,7 +1059,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
         assertEquals(3,rs.getInt(1));
         assertFalse(rs.next());
 
-        rs = conn1.createStatement().executeQuery("SELECT COLUMN_COUNT FROM SYSTEM.CATALOG\n"
+        rs = conn1.createStatement().executeQuery("SELECT COLUMN_COUNT FROM \"SYSTEM\".\"CATALOG\"\n"
                 + "WHERE TENANT_ID IS NULL AND\n"
                 + "(TABLE_SCHEM, TABLE_NAME) = ('" + schemaName + "','"+ indexTableName + "') AND\n"
                 + "COLUMN_FAMILY IS NULL AND COLUMN_NAME IS NULL");
@@ -1977,7 +1977,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
         Statement stmt = conn.createStatement();
         stmt.execute("CREATE TABLE " + dataTableFullName + " (id SMALLINT PRIMARY KEY, name VARCHAR)");
 
-        ResultSet rs = stmt.executeQuery("SELECT STORE_NULLS FROM SYSTEM.CATALOG " +
+        ResultSet rs = stmt.executeQuery("SELECT STORE_NULLS FROM \"SYSTEM\".\"CATALOG\" " +
  "WHERE table_name = '"
                 + dataTableFullName + "' AND STORE_NULLS IS NOT NULL");
         assertTrue(rs.next());
@@ -1987,7 +1987,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
 
         stmt.execute("ALTER TABLE " + dataTableFullName + " SET STORE_NULLS = true");
 
-        rs = stmt.executeQuery("SELECT STORE_NULLS FROM SYSTEM.CATALOG " +
+        rs = stmt.executeQuery("SELECT STORE_NULLS FROM \"SYSTEM\".\"CATALOG\" " +
  "WHERE table_name = '" + dataTableFullName
                 + "' AND STORE_NULLS IS NOT NULL");
         assertTrue(rs.next());
@@ -2102,7 +2102,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
             assertTrue(t.isMultiTenant());
             
             // check table metadata updated server side
-            ResultSet rs = conn.createStatement().executeQuery("SELECT DISABLE_WAL, MULTI_TENANT FROM SYSTEM.CATALOG "
+            ResultSet rs = conn.createStatement().executeQuery("SELECT DISABLE_WAL, MULTI_TENANT FROM \"SYSTEM\".\"CATALOG\""
                             + "WHERE table_name = '"
                             + dataTableFullName + "' AND DISABLE_WAL IS NOT NULL AND MULTI_TENANT IS NOT NULL");
             assertTrue(rs.next());
@@ -2162,7 +2162,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
     }
     
     private void assertIsRowTimestampSet(String schemaName, String tableName, String columnName) throws SQLException {
-        String sql = "SELECT IS_ROW_TIMESTAMP FROM SYSTEM.CATALOG WHERE "
+        String sql = "SELECT IS_ROW_TIMESTAMP FROM \"SYSTEM\".\"CATALOG\" WHERE "
                 + "(TABLE_SCHEM, TABLE_NAME) = ('" + schemaName + "','"+ tableName + "') AND\n"
                 + "COLUMN_FAMILY IS NULL AND COLUMN_NAME = ?";
         try(Connection conn = DriverManager.getConnection(getUrl())) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/895cb1df/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
index 52dae00..211145e 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
@@ -585,7 +585,7 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
     public static String getSystemCatalogEntriesForTable(Connection conn, String tableName, String message) throws Exception {
         StringBuilder sb = new StringBuilder(message);
         sb.append("\n\n\n");
-        ResultSet rs = conn.createStatement().executeQuery("SELECT * FROM SYSTEM.CATALOG WHERE TABLE_NAME='"+ tableName +"'");
+        ResultSet rs = conn.createStatement().executeQuery("SELECT * FROM \"SYSTEM\".\"CATALOG\" WHERE TABLE_NAME='"+ tableName +"'");
         ResultSetMetaData metaData = rs.getMetaData();
         int rowNum = 0;
         while (rs.next()) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/895cb1df/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArithmeticQueryIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArithmeticQueryIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArithmeticQueryIT.java
index c297441..6126746 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArithmeticQueryIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArithmeticQueryIT.java
@@ -1050,7 +1050,7 @@ public class ArithmeticQueryIT extends ParallelStatsDisabledIT {
         conn.createStatement().execute(dml);
         conn.commit();
 
-        ResultSet rs = conn.createStatement().executeQuery("SELECT 1.2E3 FROM SYSTEM.CATALOG LIMIT 1");
+        ResultSet rs = conn.createStatement().executeQuery("SELECT 1.2E3 FROM \"SYSTEM\".\"CATALOG\" LIMIT 1");
         assertTrue(rs.next());
         assertTrue(rs.getObject(1) instanceof Double);
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/895cb1df/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayFillFunctionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayFillFunctionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayFillFunctionIT.java
index dfa0beb..5133e5d 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayFillFunctionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayFillFunctionIT.java
@@ -44,10 +44,10 @@ public class ArrayFillFunctionIT extends ParallelStatsDisabledIT {
         tableName = generateUniqueName();
         Connection conn = DriverManager.getConnection(getUrl());
         String ddl = "CREATE TABLE " + tableName
-            + " (region_name VARCHAR PRIMARY KEY,length1 INTEGER, length2 INTEGER,\"DATE\" DATE,time TIME,timestamp TIMESTAMP,varchar VARCHAR,integer INTEGER,double DOUBLE,bigint BIGINT,char CHAR(15),double1 DOUBLE,char1 CHAR(17),nullcheck INTEGER,chars2 CHAR(15)[], varchars2 VARCHAR[])";
+            + " (region_name VARCHAR PRIMARY KEY,length1 INTEGER, length2 INTEGER,\"DATE\" DATE,\"time\" TIME,\"timestamp\" TIMESTAMP,\"varchar\" VARCHAR,\"integer\" INTEGER,\"double\" DOUBLE,\"bigint\" BIGINT,\"char\" CHAR(15),double1 DOUBLE,char1 CHAR(17),nullcheck INTEGER,chars2 CHAR(15)[], varchars2 VARCHAR[])";
         conn.createStatement().execute(ddl);
         String dml = "UPSERT INTO " + tableName
-            + "(region_name,length1,length2,\"DATE\",time,timestamp,varchar,integer,double,bigint,char,double1,char1,nullcheck,chars2,varchars2) VALUES('SF Bay Area',"
+            + "(region_name,length1,length2,\"DATE\",\"time\",\"timestamp\",\"varchar\",\"integer\",\"double\",\"bigint\",\"char\",double1,char1,nullcheck,chars2,varchars2) VALUES('SF Bay Area',"
             +
                 "0," +
                 "-3," +
@@ -76,7 +76,7 @@ public class ArrayFillFunctionIT extends ParallelStatsDisabledIT {
 
         ResultSet rs;
         rs = conn.createStatement().executeQuery(
-            "SELECT ARRAY_FILL(varchar,5) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
+            "SELECT ARRAY_FILL(\"varchar\",5) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         String[] strings = new String[]{"foo", "foo", "foo", "foo", "foo"};
@@ -93,7 +93,7 @@ public class ArrayFillFunctionIT extends ParallelStatsDisabledIT {
 
         ResultSet rs;
         rs = conn.createStatement().executeQuery(
-            "SELECT ARRAY_FILL(integer,4) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
+            "SELECT ARRAY_FILL(\"integer\",4) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         Object[] objects = new Object[]{34, 34, 34, 34};
@@ -111,7 +111,7 @@ public class ArrayFillFunctionIT extends ParallelStatsDisabledIT {
 
         ResultSet rs;
         rs = conn.createStatement().executeQuery(
-            "SELECT ARRAY_FILL(double,4) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
+            "SELECT ARRAY_FILL(\"double\",4) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         Object[] objects = new Object[]{23.45, 23.45, 23.45, 23.45};
@@ -129,7 +129,7 @@ public class ArrayFillFunctionIT extends ParallelStatsDisabledIT {
 
         ResultSet rs;
         rs = conn.createStatement().executeQuery(
-            "SELECT ARRAY_FILL(bigint,4) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
+            "SELECT ARRAY_FILL(\"bigint\",4) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         Object[] objects = new Object[]{34567l, 34567l, 34567l, 34567l};
@@ -147,7 +147,7 @@ public class ArrayFillFunctionIT extends ParallelStatsDisabledIT {
 
         ResultSet rs;
         rs = conn.createStatement().executeQuery(
-            "SELECT ARRAY_FILL(char,4) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
+            "SELECT ARRAY_FILL(\"char\",4) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         Object[] objects = new Object[]{"foo", "foo", "foo", "foo"};
@@ -164,7 +164,7 @@ public class ArrayFillFunctionIT extends ParallelStatsDisabledIT {
 
         ResultSet rs;
         rs = conn.createStatement().executeQuery(
-            "SELECT ARRAY_FILL(varchar,4) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
+            "SELECT ARRAY_FILL(\"varchar\",4) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         Object[] objects = new Object[]{"foo", "foo", "foo", "foo"};
@@ -198,7 +198,7 @@ public class ArrayFillFunctionIT extends ParallelStatsDisabledIT {
 
         ResultSet rs;
         rs = conn.createStatement().executeQuery(
-            "SELECT ARRAY_FILL(time,3) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
+            "SELECT ARRAY_FILL(\"time\",3) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         Object[] objects = new Object[]{new Time(1432102334184l), new Time(1432102334184l), new Time(1432102334184l)};
@@ -215,7 +215,7 @@ public class ArrayFillFunctionIT extends ParallelStatsDisabledIT {
 
         ResultSet rs;
         rs = conn.createStatement().executeQuery(
-            "SELECT ARRAY_FILL(timestamp,3) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
+            "SELECT ARRAY_FILL(\"timestamp\",3) FROM " + tableName + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
         Object[] objects = new Object[]{new Timestamp(1432102334184l), new Timestamp(1432102334184l), new Timestamp(1432102334184l)};
@@ -232,7 +232,7 @@ public class ArrayFillFunctionIT extends ParallelStatsDisabledIT {
 
         ResultSet rs;
         rs = conn.createStatement().executeQuery(
-            "SELECT ARRAY_FILL(timestamp,length2) FROM " + tableName
+            "SELECT ARRAY_FILL(\"timestamp\",length2) FROM " + tableName
                 + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
@@ -250,7 +250,7 @@ public class ArrayFillFunctionIT extends ParallelStatsDisabledIT {
 
         ResultSet rs;
         rs = conn.createStatement().executeQuery(
-            "SELECT ARRAY_FILL(timestamp,length1) FROM " + tableName
+            "SELECT ARRAY_FILL(\"timestamp\",length1) FROM " + tableName
                 + " WHERE region_name = 'SF Bay Area'");
         assertTrue(rs.next());
 
@@ -513,7 +513,7 @@ public class ArrayFillFunctionIT extends ParallelStatsDisabledIT {
 
         ResultSet rs;
         rs = conn.createStatement().executeQuery(
-            "SELECT region_name FROM " + tableName + " WHERE varchar=ANY(ARRAY_FILL('foo',3))");
+            "SELECT region_name FROM " + tableName + " WHERE \"varchar\"=ANY(ARRAY_FILL('foo',3))");
         assertTrue(rs.next());
 
         assertEquals("SF Bay Area", rs.getString(1));
@@ -555,7 +555,7 @@ public class ArrayFillFunctionIT extends ParallelStatsDisabledIT {
 
         ResultSet rs;
         rs = conn.createStatement().executeQuery("SELECT region_name FROM " + tableName
-            + " WHERE ARRAY['foo','foo','foo','foo','foo']=ARRAY_FILL(varchar,5)");
+            + " WHERE ARRAY['foo','foo','foo','foo','foo']=ARRAY_FILL(\"varchar\",5)");
         assertTrue(rs.next());
 
         assertEquals("SF Bay Area", rs.getString(1));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/895cb1df/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseTenantSpecificTablesIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseTenantSpecificTablesIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseTenantSpecificTablesIT.java
index 0749a20..5e9208b 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseTenantSpecificTablesIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseTenantSpecificTablesIT.java
@@ -56,7 +56,7 @@ public abstract class BaseTenantSpecificTablesIT extends ParallelStatsEnabledIT
         PARENT_TABLE_NAME_NO_TENANT_TYPE_ID = "P_" + BaseTest.generateUniqueName();
         TENANT_TABLE_NAME_NO_TENANT_TYPE_ID = "V_" + BaseTest.generateUniqueName();
         PARENT_TABLE_DDL = "CREATE TABLE " + PARENT_TABLE_NAME + " ( \n" + 
-                "                user VARCHAR ,\n" + 
+                "                \"user\" VARCHAR ,\n" + 
                 "                tenant_id VARCHAR NOT NULL,\n" + 
                 "                tenant_type_id VARCHAR(3) NOT NULL, \n" + 
                 "                id INTEGER NOT NULL\n" + 
@@ -65,7 +65,7 @@ public abstract class BaseTenantSpecificTablesIT extends ParallelStatsEnabledIT
                 "                tenant_col VARCHAR) AS SELECT *\n" + 
                 "                FROM " + PARENT_TABLE_NAME + " WHERE tenant_type_id= '" + TENANT_TYPE_ID + "'";
         PARENT_TABLE_DDL_NO_TENANT_TYPE_ID = "CREATE TABLE " + PARENT_TABLE_NAME_NO_TENANT_TYPE_ID + " ( \n" + 
-                "                user VARCHAR ,\n" + 
+                "                \"user\" VARCHAR ,\n" + 
                 "                tenant_id VARCHAR NOT NULL,\n" + 
                 "                id INTEGER NOT NULL,\n" + 
                 "                CONSTRAINT pk PRIMARY KEY (tenant_id, id)) MULTI_TENANT=true, IMMUTABLE_ROWS=true";

http://git-wip-us.apache.org/repos/asf/phoenix/blob/895cb1df/phoenix-core/src/it/java/org/apache/phoenix/end2end/CoalesceFunctionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CoalesceFunctionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CoalesceFunctionIT.java
index 0666f77..828051f 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CoalesceFunctionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CoalesceFunctionIT.java
@@ -69,16 +69,16 @@ public class CoalesceFunctionIT extends ParallelStatsDisabledIT {
         String tableName = generateUniqueName();
         String ddl = "CREATE TABLE " + tableName + "("
                 + "    ID BIGINT NOT NULL, "
-                + "    COUNT BIGINT "
+                + "    \"COUNT\" BIGINT "
                 + "    CONSTRAINT pk PRIMARY KEY(ID))";
         conn.createStatement().execute(ddl);
 
-        conn.createStatement().execute("UPSERT INTO " + tableName + "(ID, COUNT) VALUES(2, null)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + "(ID, \"COUNT\") VALUES(2, null)");
         conn.commit();
 
         ResultSet rs = conn.createStatement().executeQuery(
                 "SELECT "
-                + "COALESCE(SUM(COUNT), CAST(0 AS BIGINT)) " //explicitly def long
+                + "COALESCE(SUM(\"COUNT\"), CAST(0 AS BIGINT)) " //explicitly def long
                 + "FROM  " + tableName
                 + " GROUP BY ID");
 
@@ -93,16 +93,16 @@ public class CoalesceFunctionIT extends ParallelStatsDisabledIT {
         String tableName = generateUniqueName();
         String ddl = "CREATE TABLE " + tableName + "("
                 + "    ID BIGINT NOT NULL, "
-                + "    COUNT BIGINT "
+                + "    \"COUNT\" BIGINT "
                 + "    CONSTRAINT pk PRIMARY KEY(ID))";
         conn.createStatement().execute(ddl);
 
-        conn.createStatement().execute("UPSERT INTO " + tableName + "(ID, COUNT) VALUES(2, null)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + "(ID, \"COUNT\") VALUES(2, null)");
         conn.commit();
 
         ResultSet rs = conn.createStatement().executeQuery(
                 "SELECT "
-                + "COALESCE(SUM(COUNT), 0) " // no long def
+                + "COALESCE(SUM(\"COUNT\"), 0) " // no long def
                 + "FROM " + tableName
                 + " GROUP BY ID");
 
@@ -117,16 +117,16 @@ public class CoalesceFunctionIT extends ParallelStatsDisabledIT {
         String tableName = generateUniqueName();
         String ddl = "CREATE TABLE " + tableName + "("
                 + "    ID BIGINT NOT NULL, "
-                + "    COUNT BIGINT "
+                + "    \"COUNT\" BIGINT "
                 + "    CONSTRAINT pk PRIMARY KEY(ID))";
         conn.createStatement().execute(ddl);
 
-        conn.createStatement().execute("UPSERT INTO " + tableName + "(ID, COUNT) VALUES(2, null)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + "(ID, \"COUNT\") VALUES(2, null)");
         conn.commit();
 
         ResultSet rs = conn.createStatement().executeQuery(
                 "SELECT "
-                + "COALESCE(SUM(COUNT), SUM(ID)) " // second param as expression
+                + "COALESCE(SUM(\"COUNT\"), SUM(ID)) " // second param as expression
                 + "FROM " + tableName
                 + " GROUP BY ID");
 
@@ -142,16 +142,16 @@ public class CoalesceFunctionIT extends ParallelStatsDisabledIT {
 
         String ddl = "CREATE TABLE " + tableName + "("
                 + "    ID BIGINT NOT NULL, "
-                + "    COUNT BIGINT " //first parameter to coalesce
+                + "    \"COUNT\" BIGINT " //first parameter to coalesce
                 + "    CONSTRAINT pk PRIMARY KEY(ID))";
         conn.createStatement().execute(ddl);
 
-        conn.createStatement().execute("UPSERT INTO "  + tableName + "(ID, COUNT) VALUES(2, null)");
+        conn.createStatement().execute("UPSERT INTO "  + tableName + "(ID, \"COUNT\") VALUES(2, null)");
         conn.commit();
 
         ResultSet rs = conn.createStatement().executeQuery(
                 "SELECT "
-                + "COALESCE(NTH_VALUE(COUNT, 100) WITHIN GROUP (ORDER BY COUNT DESC), 0) " //second param is int
+                + "COALESCE(NTH_VALUE(\"COUNT\", 100) WITHIN GROUP (ORDER BY \"COUNT\" DESC), 0) " //second param is int
                 + "FROM " + tableName
                 + " GROUP BY ID");
 
@@ -166,17 +166,17 @@ public class CoalesceFunctionIT extends ParallelStatsDisabledIT {
         String tableName = generateUniqueName();
         String ddl = "CREATE TABLE " + tableName + "("
                 + "    ID BIGINT NOT NULL, "
-                + "    COUNT UNSIGNED_INT " //first parameter to coalesce
+                + "    \"COUNT\" UNSIGNED_INT " //first parameter to coalesce
                 + "    CONSTRAINT pk PRIMARY KEY(ID))";
         conn.createStatement().execute(ddl);
 
-        conn.createStatement().execute("UPSERT INTO " + tableName + " (ID, COUNT) VALUES(2, null)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (ID, \"COUNT\") VALUES(2, null)");
         conn.commit();
 
         //second param to coalesce is signed int
         ResultSet rs = conn.createStatement().executeQuery(
                 "SELECT "
-                + " COALESCE(NTH_VALUE(COUNT, 100) WITHIN GROUP (ORDER BY COUNT DESC), 1) "
+                + " COALESCE(NTH_VALUE(\"COUNT\", 100) WITHIN GROUP (ORDER BY COUNT DESC), 1) "
                 + " FROM " + tableName
                 + " GROUP BY ID");
 
@@ -193,21 +193,21 @@ public class CoalesceFunctionIT extends ParallelStatsDisabledIT {
         String ddl = "CREATE TABLE " + tableName + "("
                 + "    ID BIGINT NOT NULL, "
                 + "    DATE TIMESTAMP NOT NULL, "
-                + "    COUNT BIGINT "
+                + "    \"COUNT\" BIGINT "
                 + "    CONSTRAINT pk PRIMARY KEY(ID, DATE))";
         conn.createStatement().execute(ddl);
 
-        conn.createStatement().execute("UPSERT INTO " + tableName + "(ID, DATE, COUNT) VALUES(1, CURRENT_TIME(), 1)");
-        conn.createStatement().execute("UPSERT INTO " + tableName + "(ID, DATE, COUNT) VALUES(1, CURRENT_TIME(), 2)");
-        conn.createStatement().execute("UPSERT INTO " + tableName + "(ID, DATE, COUNT) VALUES(2, CURRENT_TIME(), 1)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + "(ID, DATE, \"COUNT\") VALUES(1, CURRENT_TIME(), 1)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + "(ID, DATE, \"COUNT\") VALUES(1, CURRENT_TIME(), 2)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + "(ID, DATE, \"COUNT\") VALUES(2, CURRENT_TIME(), 1)");
         conn.commit();
 
         //second param to coalesce is signed int
         ResultSet rs = conn.createStatement().executeQuery(
                 "SELECT "
                 + " COALESCE("
-                + "            NTH_VALUE(COUNT, 2000)" // should evaluate null
-                + "            WITHIN GROUP (ORDER BY COUNT DESC),"
+                + "            NTH_VALUE(\"COUNT\", 2000)" // should evaluate null
+                + "            WITHIN GROUP (ORDER BY \"COUNT\" DESC),"
                 + "       0)"
                 + "FROM  " + tableName
                 + " GROUP BY ID");
@@ -224,17 +224,17 @@ public class CoalesceFunctionIT extends ParallelStatsDisabledIT {
 
         String ddl = "CREATE TABLE " + tableName + "("
                 + "    ID UNSIGNED_INT NOT NULL, "
-                + "    COUNT UNSIGNED_INT "
+                + "    \"COUNT\" UNSIGNED_INT "
                 + "    CONSTRAINT pk PRIMARY KEY(ID))";
         conn.createStatement().execute(ddl);
 
-        conn.createStatement().execute("UPSERT INTO " + tableName + "(ID, COUNT) VALUES(2, null)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + "(ID, \"COUNT\") VALUES(2, null)");
         conn.commit();
 
         try {
             conn.createStatement().executeQuery(
                     "SELECT "
-                    + "COALESCE(MIN(COUNT), -1) " // invalid value for UNSIGNED_INT
+                    + "COALESCE(MIN(\"COUNT\"), -1) " // invalid value for UNSIGNED_INT
                     + "FROM " + tableName
                     + " GROUP BY ID");
 
@@ -251,17 +251,17 @@ public class CoalesceFunctionIT extends ParallelStatsDisabledIT {
 
         String ddl = "CREATE TABLE " + tableName + "("
                 + "    ID INTEGER NOT NULL, "
-                + "    COUNT UNSIGNED_INT " //first parameter to coalesce
+                + "    \"COUNT\" UNSIGNED_INT " //first parameter to coalesce
                 + "    CONSTRAINT pk PRIMARY KEY(ID))";
         conn.createStatement().execute(ddl);
 
-        conn.createStatement().execute("UPSERT INTO " + tableName + "(ID, COUNT) VALUES(-2, null)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + "(ID, \"COUNT\") VALUES(-2, null)");
         conn.commit();
 
         try {
             ResultSet rs = conn.createStatement().executeQuery(
                     "SELECT "
-                    + "COALESCE(MIN(COUNT), ID) "
+                    + "COALESCE(MIN(\"COUNT\"), ID) "
                     + "FROM " + tableName
                     + " GROUP BY ID");
 
@@ -280,16 +280,16 @@ public class CoalesceFunctionIT extends ParallelStatsDisabledIT {
 
         String ddl = "CREATE TABLE " + tableName + "("
                 + "    ID DOUBLE NOT NULL, "
-                + "    COUNT INTEGER " //first parameter to coalesce
+                + "    \"COUNT\" INTEGER " //first parameter to coalesce
                 + "    CONSTRAINT pk PRIMARY KEY(ID))";
         conn.createStatement().execute(ddl);
 
-        conn.createStatement().execute("UPSERT INTO " + tableName + "(ID, COUNT) VALUES(2.0, null)");
+        conn.createStatement().execute("UPSERT INTO " + tableName + "(ID, \"COUNT\") VALUES(2.0, null)");
         conn.commit();
 
         ResultSet rs = conn.createStatement().executeQuery(
                 "SELECT "
-                + "COALESCE(MIN(COUNT), ID) "
+                + "COALESCE(MIN(\"COUNT\"), ID) "
                 + "FROM " + tableName
                 + " GROUP BY ID");
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/895cb1df/phoenix-core/src/it/java/org/apache/phoenix/end2end/DateTimeIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DateTimeIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DateTimeIT.java
index 74cc068..ad45d5e 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DateTimeIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DateTimeIT.java
@@ -395,7 +395,7 @@ public class DateTimeIT extends ParallelStatsDisabledIT {
         Statement stmt = conn.createStatement();
         ResultSet rs =
                 stmt.executeQuery(String
-                    .format("SELECT %s FROM SYSTEM.CATALOG LIMIT 1", invocation));
+                    .format("SELECT %s FROM \"SYSTEM\".\"CATALOG\" LIMIT 1", invocation));
         assertTrue(rs.next());
         int returnValue = rs.getInt(1);
         assertFalse(rs.next());
@@ -815,7 +815,7 @@ public class DateTimeIT extends ParallelStatsDisabledIT {
     public void testLiteralDateTimestampComparison() throws Exception {
         ResultSet rs =
                 conn.createStatement().executeQuery(
-                    "select DATE '2016-05-10 00:00:00' > TIMESTAMP '2016-05-11 00:00:00'");
+                    "select \"DATE\" '2016-05-10 00:00:00' > \"TIMESTAMP\" '2016-05-11 00:00:00'");
 
         assertTrue(rs.next());
         assertEquals(false, rs.getBoolean(1));
@@ -826,7 +826,7 @@ public class DateTimeIT extends ParallelStatsDisabledIT {
     public void testLiteralDateTimestampComparison2() throws Exception {
         ResultSet rs =
                 conn.createStatement().executeQuery(
-                    "select TIMESTAMP '2016-05-10 00:00:00' > DATE '2016-05-11 00:00:00'");
+                    "select \"TIMESTAMP\" '2016-05-10 00:00:00' > \"DATE\" '2016-05-11 00:00:00'");
 
         assertTrue(rs.next());
         assertEquals(false, rs.getBoolean(1));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/895cb1df/phoenix-core/src/it/java/org/apache/phoenix/end2end/DeleteIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DeleteIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DeleteIT.java
index 9f514b3..09e1021 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DeleteIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DeleteIT.java
@@ -299,11 +299,11 @@ public class DeleteIT extends ParallelStatsDisabledIT {
                     "HOST CHAR(2) NOT NULL," +
                     "DOMAIN VARCHAR NOT NULL, " +
                     "FEATURE VARCHAR NOT NULL, " +
-                    "DATE DATE NOT NULL, \n" + 
+                    "\"DATE\" DATE NOT NULL, \n" + 
                     "USAGE.CORE BIGINT," +
                     "USAGE.DB BIGINT," +
                     "STATS.ACTIVE_VISITOR INTEGER " +
-                    "CONSTRAINT PK PRIMARY KEY (HOST, DOMAIN, FEATURE, DATE))" + (isSalted ? " SALT_BUCKETS=3" : "");
+                    "CONSTRAINT PK PRIMARY KEY (HOST, DOMAIN, FEATURE, \"DATE\"))" + (isSalted ? " SALT_BUCKETS=3" : "");
             stm.execute(s);
             String localIndexName = generateUniqueName();
             String indexName = generateUniqueName();
@@ -315,7 +315,7 @@ public class DeleteIT extends ParallelStatsDisabledIT {
             stm.close();
 
             PreparedStatement psInsert = con
-                    .prepareStatement("UPSERT INTO " + tableName + "(HOST, DOMAIN, FEATURE, DATE, CORE, DB, ACTIVE_VISITOR) VALUES(?,?, ? , ?, ?, ?, ?)");
+                    .prepareStatement("UPSERT INTO " + tableName + "(HOST, DOMAIN, FEATURE, \"DATE\", CORE, DB, ACTIVE_VISITOR) VALUES(?,?, ? , ?, ?, ?, ?)");
             psInsert.setString(1, "AA");
             psInsert.setString(2, "BB");
             psInsert.setString(3, "CC");
@@ -380,18 +380,18 @@ public class DeleteIT extends ParallelStatsDisabledIT {
                     "HOST CHAR(2) NOT NULL," +
                     "DOMAIN VARCHAR NOT NULL, " +
                     "FEATURE VARCHAR NOT NULL, " +
-                    "DATE DATE NOT NULL, \n" + 
+                    "\"DATE\" DATE NOT NULL, \n" + 
                     "USAGE.CORE BIGINT," +
                     "USAGE.DB BIGINT," +
                     "STATS.ACTIVE_VISITOR INTEGER " +
-                    "CONSTRAINT PK PRIMARY KEY (HOST, DOMAIN, FEATURE, DATE)) IMMUTABLE_ROWS=true");
-            stm.execute("CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName1 + " ON " + tableName + " (DATE, FEATURE)");
-            stm.execute("CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName2 + " ON " + tableName + " (DATE, FEATURE, USAGE.DB)");
+                    "CONSTRAINT PK PRIMARY KEY (HOST, DOMAIN, FEATURE, \"DATE\")) IMMUTABLE_ROWS=true");
+            stm.execute("CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName1 + " ON " + tableName + " (\"DATE\", FEATURE)");
+            stm.execute("CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName2 + " ON " + tableName + " (\"DATE\", FEATURE, USAGE.DB)");
             stm.close();
 
             Date date = new Date(0);
             PreparedStatement psInsert = con
-                    .prepareStatement("UPSERT INTO " + tableName + "(HOST, DOMAIN, FEATURE, DATE, CORE, DB, ACTIVE_VISITOR) VALUES(?,?, ? , ?, ?, ?, ?)");
+                    .prepareStatement("UPSERT INTO " + tableName + "(HOST, DOMAIN, FEATURE, \"DATE\", CORE, DB, ACTIVE_VISITOR) VALUES(?,?, ? , ?, ?, ?, ?)");
             psInsert.setString(1, "AA");
             psInsert.setString(2, "BB");
             psInsert.setString(3, "CC");
@@ -405,7 +405,7 @@ public class DeleteIT extends ParallelStatsDisabledIT {
                 con.commit();
             }
             
-            psInsert = con.prepareStatement("DELETE FROM " + tableName + " WHERE (HOST, DOMAIN, FEATURE, DATE) = (?,?,?,?)");
+            psInsert = con.prepareStatement("DELETE FROM " + tableName + " WHERE (HOST, DOMAIN, FEATURE, \"DATE\") = (?,?,?,?)");
             psInsert.setString(1, "AA");
             psInsert.setString(2, "BB");
             psInsert.setString(3, "CC");
@@ -427,7 +427,7 @@ public class DeleteIT extends ParallelStatsDisabledIT {
             stm.execute("DROP INDEX " + indexName2 + " ON " + tableName);
 
             stm.execute("CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName1 + " ON " + tableName + " (USAGE.DB)");
-            stm.execute("CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName2 + " ON " + tableName + " (USAGE.DB, DATE)");
+            stm.execute("CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName2 + " ON " + tableName + " (USAGE.DB, \"DATE\")");
             try{
                 psInsert = con.prepareStatement("DELETE FROM " + tableName + " WHERE  USAGE.DB=2");
             } catch(Exception e) {
@@ -465,15 +465,15 @@ public class DeleteIT extends ParallelStatsDisabledIT {
                     " HOST CHAR(2) NOT NULL," +
                     "DOMAIN VARCHAR NOT NULL, " +
                     "FEATURE VARCHAR NOT NULL, " +
-                    "DATE DATE NOT NULL, \n" + 
+                    "\"DATE\" DATE NOT NULL, \n" + 
                     "USAGE.CORE BIGINT," +
                     "USAGE.DB BIGINT," +
                     "STATS.ACTIVE_VISITOR INTEGER " +
-                    "CONSTRAINT PK PRIMARY KEY (HOST, DOMAIN, FEATURE, DATE))");
+                    "CONSTRAINT PK PRIMARY KEY (HOST, DOMAIN, FEATURE, \"DATE\"))");
             stm.close();
 
             PreparedStatement psInsert = con
-                    .prepareStatement("UPSERT INTO " + tableName + "(HOST, DOMAIN, FEATURE, DATE, CORE, DB, ACTIVE_VISITOR) VALUES(?,?, ? , ?, ?, ?, ?)");
+                    .prepareStatement("UPSERT INTO " + tableName + "(HOST, DOMAIN, FEATURE, \"DATE\", CORE, DB, ACTIVE_VISITOR) VALUES(?,?, ? , ?, ?, ?, ?)");
             psInsert.setString(1, "AA");
             psInsert.setString(2, "BB");
             psInsert.setString(3, "CC");

http://git-wip-us.apache.org/repos/asf/phoenix/blob/895cb1df/phoenix-core/src/it/java/org/apache/phoenix/end2end/EvaluationOfORIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/EvaluationOfORIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/EvaluationOfORIT.java
index 1e450de..8edbaa6 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/EvaluationOfORIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/EvaluationOfORIT.java
@@ -39,7 +39,7 @@ public class EvaluationOfORIT extends ParallelStatsDisabledIT{
     public void testFalseOrFalse() throws SQLException {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
-        ResultSet rs = conn.createStatement().executeQuery("SELECT (FALSE OR FALSE) AS B FROM SYSTEM.CATALOG LIMIT 1");
+        ResultSet rs = conn.createStatement().executeQuery("SELECT (FALSE OR FALSE) AS B FROM \"SYSTEM\".\"CATALOG\" LIMIT 1");
         assertTrue(rs.next());
         assertFalse(rs.getBoolean(1));
         conn.close();
@@ -53,8 +53,7 @@ public class EvaluationOfORIT extends ParallelStatsDisabledIT{
 	    conn.setAutoCommit(false);
 	    
             String create = "CREATE TABLE " + tableName + " ( ID INTEGER NOT NULL PRIMARY KEY,NAME VARCHAR(50))";
-            PreparedStatement createStmt = conn.prepareStatement(create);
-            createStmt.execute();
+            conn.createStatement().execute(create);
             PreparedStatement stmt = conn.prepareStatement(
                     "upsert into " +
                      tableName + " VALUES (?, ?)");

http://git-wip-us.apache.org/repos/asf/phoenix/blob/895cb1df/phoenix-core/src/it/java/org/apache/phoenix/end2end/GroupByCaseIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/GroupByCaseIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/GroupByCaseIT.java
index 56a94b3..5ceb412 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/GroupByCaseIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/GroupByCaseIT.java
@@ -412,21 +412,21 @@ public class GroupByCaseIT extends ParallelStatsDisabledIT {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         String tableName = generateUniqueName();
-        ResultSet rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + PhoenixDatabaseMetaData.SYSTEM_STATS_NAME + " WHERE " + PhoenixDatabaseMetaData.PHYSICAL_NAME + " ='" + tableName + "'");
+        ResultSet rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + "\"SYSTEM\".\"STATS\"" + " WHERE " + PhoenixDatabaseMetaData.PHYSICAL_NAME + " ='" + tableName + "'");
         assertTrue(rs.next());
         assertEquals(0,rs.getInt(1));
         initAvgGroupTable(conn, tableName, PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH + "=20 ");
         testAvgGroupByOrderPreserving(conn, tableName, 13);
-        rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + PhoenixDatabaseMetaData.SYSTEM_STATS_NAME + " WHERE " + PhoenixDatabaseMetaData.PHYSICAL_NAME + " ='" + tableName + "'");
+        rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + "\"SYSTEM\".\"STATS\"" + " WHERE " + PhoenixDatabaseMetaData.PHYSICAL_NAME + " ='" + tableName + "'");
         assertTrue(rs.next());
         assertEquals(13,rs.getInt(1));
         conn.setAutoCommit(true);
-        conn.createStatement().execute("DELETE FROM " + PhoenixDatabaseMetaData.SYSTEM_STATS_NAME);
-        rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + PhoenixDatabaseMetaData.SYSTEM_STATS_NAME + " WHERE " + PhoenixDatabaseMetaData.PHYSICAL_NAME + " ='" + tableName + "'");
+        conn.createStatement().execute("DELETE FROM " + "\"SYSTEM\".\"STATS\"");
+        rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + "\"SYSTEM\".\"STATS\"" + " WHERE " + PhoenixDatabaseMetaData.PHYSICAL_NAME + " ='" + tableName + "'");
         assertTrue(rs.next());
         assertEquals(0,rs.getInt(1));
         TestUtil.doMajorCompaction(conn, tableName);
-        rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + PhoenixDatabaseMetaData.SYSTEM_STATS_NAME + " WHERE " + PhoenixDatabaseMetaData.PHYSICAL_NAME + " ='" + tableName + "'");
+        rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + "\"SYSTEM\".\"STATS\"" + " WHERE " + PhoenixDatabaseMetaData.PHYSICAL_NAME + " ='" + tableName + "'");
         assertTrue(rs.next());
         assertEquals(13,rs.getInt(1));
         testAvgGroupByOrderPreserving(conn, tableName, 13);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/895cb1df/phoenix-core/src/it/java/org/apache/phoenix/end2end/GroupByIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/GroupByIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/GroupByIT.java
index f0c781f..b9fa15b 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/GroupByIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/GroupByIT.java
@@ -576,7 +576,7 @@ public class GroupByIT extends BaseQueryIT {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 10));
         Connection conn = DriverManager.getConnection(getUrl(), props);
-        String ddl = "create table test1(time integer not null, hostname varchar not null,usage float,period integer constraint pk PRIMARY KEY(time, hostname))";
+        String ddl = "create table test1(\"time\" integer not null, hostname varchar not null,usage float,period integer constraint pk PRIMARY KEY(\"time\", hostname))";
         conn.createStatement().execute(ddl);
         conn.close();
 
@@ -596,7 +596,7 @@ public class GroupByIT extends BaseQueryIT {
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 40));
         conn = DriverManager.getConnection(getUrl(), props);
         ResultSet rs;
-        stmt = conn.prepareStatement("select time/10 as tm, hostname, avg(usage) from test1 group by hostname, tm");
+        stmt = conn.prepareStatement("select \"time\"/10 as tm, hostname, avg(usage) from test1 group by hostname, tm");
         rs = stmt.executeQuery();
         assertTrue(rs.next());
         assertEquals(143985345, rs.getInt(1));
@@ -612,7 +612,7 @@ public class GroupByIT extends BaseQueryIT {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 10));
         Connection conn = DriverManager.getConnection(getUrl(), props);
-        String ddl = "create table test1(time integer not null, hostname varchar not null,usage float,period integer constraint pk PRIMARY KEY(time, hostname))";
+        String ddl = "create table test1(\"time\" integer not null, hostname varchar not null,usage float,period integer constraint pk PRIMARY KEY(\"time\", hostname))";
         conn.createStatement().execute(ddl);
         conn.close();
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/895cb1df/phoenix-core/src/it/java/org/apache/phoenix/end2end/HashJoinMoreIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/HashJoinMoreIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/HashJoinMoreIT.java
index 52e8006..c76eb67 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/HashJoinMoreIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/HashJoinMoreIT.java
@@ -236,13 +236,8 @@ public class HashJoinMoreIT extends ParallelStatsDisabledIT {
                     + "        colA2 VARCHAR " + "CONSTRAINT PK PRIMARY KEY" + "(pkA)" + ")";
 
             String ddlB = "CREATE TABLE " + tableB + "   (pkB INTEGER NOT NULL PRIMARY KEY, " + "    colB INTEGER)";
-            stmt = conn.prepareStatement(ddlA);
-            stmt.execute();
-            stmt.close();
-
-            stmt = conn.prepareStatement(ddlB);
-            stmt.execute();
-            stmt.close();
+            conn.createStatement().execute(ddlA);
+            conn.createStatement().execute(ddlB);
 
             String upsertA = "UPSERT INTO " + tableA + " (pkA, colA1, colA2) VALUES(?, ?, ?)";
             stmt = conn.prepareStatement(upsertA);
@@ -498,15 +493,15 @@ public class HashJoinMoreIT extends ParallelStatsDisabledIT {
                             " TENANTID UNSIGNED_INT NOT NULL" +
                             ",ID UNSIGNED_INT NOT NULL" +
                             ",FOO UNSIGNED_INT NOT NULL" +
-                            ",TIMESTAMP UNSIGNED_LONG NOT NULL" +
+                            ",\"TIMESTAMP\"  UNSIGNED_LONG NOT NULL" +
                             ",CODES INTEGER ARRAY[] NOT NULL" +
                             ",V UNSIGNED_LONG" +
-                            " CONSTRAINT pk PRIMARY KEY (TENANTID, ID, FOO, TIMESTAMP, CODES))" +
+                            " CONSTRAINT pk PRIMARY KEY (TENANTID, ID, FOO, \"TIMESTAMP\" , CODES))" +
                             " DEFAULT_COLUMN_FAMILY ='E'," +
                             " MULTI_TENANT=true");
             PreparedStatement upsertStmt = conn.prepareStatement(
                     "upsert into INVENTORY "
-                    + "(tenantid, id, foo, timestamp, codes) "
+                    + "(tenantid, id, foo, \"TIMESTAMP\" , codes) "
                     + "values (?, ?, ?, ?, ?)");
             upsertStmt.setInt(1, 15);
             upsertStmt.setInt(2, 5);
@@ -672,16 +667,16 @@ public class HashJoinMoreIT extends ParallelStatsDisabledIT {
                     "CREATE TABLE IF NOT EXISTS EVENT_COUNT (\n" +
                     "        BUCKET VARCHAR,\n" +
                     "        TIMESTAMP_DATE TIMESTAMP,\n" +
-                    "        TIMESTAMP UNSIGNED_LONG NOT NULL,\n" +
+                    "        \"TIMESTAMP\" UNSIGNED_LONG NOT NULL,\n" +
                     "        LOCATION VARCHAR,\n" +
                     "        A VARCHAR,\n" +
                     "        B VARCHAR,\n" +
                     "        C VARCHAR,\n" +
                     "        D UNSIGNED_LONG,\n" +
                     "        E FLOAT\n" +
-                    "    CONSTRAINT pk PRIMARY KEY (BUCKET, TIMESTAMP DESC, LOCATION, A, B, C)\n" +
+                    "    CONSTRAINT pk PRIMARY KEY (BUCKET, \"TIMESTAMP\" DESC, LOCATION, A, B, C)\n" +
                     ") SALT_BUCKETS=2, COMPRESSION='GZ', TTL=31622400");
-            PreparedStatement stmt = conn.prepareStatement("UPSERT INTO EVENT_COUNT(BUCKET, TIMESTAMP, LOCATION, A, B, C) VALUES(?,?,?,?,?,?)");
+            PreparedStatement stmt = conn.prepareStatement("UPSERT INTO EVENT_COUNT(BUCKET, \"TIMESTAMP\", LOCATION, A, B, C) VALUES(?,?,?,?,?,?)");
             stmt.setString(1, "5SEC");
             stmt.setString(3, "Tr/Bal");
             stmt.setString(4, "A1");
@@ -733,7 +728,7 @@ public class HashJoinMoreIT extends ParallelStatsDisabledIT {
                         "CREATE TABLE IF NOT EXISTS " + t[i] + " (\n" +
                                 "        BUCKET VARCHAR,\n" +
                                 "        TIMESTAMP_DATE TIMESTAMP,\n" +
-                                "        TIMESTAMP UNSIGNED_LONG NOT NULL,\n" +
+                                "        \"TIMESTAMP\" UNSIGNED_LONG NOT NULL,\n" +
                                 "        SRC_LOCATION VARCHAR,\n" +
                                 "        DST_LOCATION VARCHAR,\n" +
                                 "        B VARCHAR,\n" +
@@ -742,9 +737,9 @@ public class HashJoinMoreIT extends ParallelStatsDisabledIT {
                                 "        G UNSIGNED_LONG,\n" +
                                 "        H UNSIGNED_LONG,\n" +
                                 "        I UNSIGNED_LONG\n" +
-                                "    CONSTRAINT pk PRIMARY KEY (BUCKET, TIMESTAMP" + (i == 0 ? " DESC" : "") + ", SRC_LOCATION, DST_LOCATION, B, C)\n" +
+                                "    CONSTRAINT pk PRIMARY KEY (BUCKET, \"TIMESTAMP\"" + (i == 0 ? " DESC" : "") + ", SRC_LOCATION, DST_LOCATION, B, C)\n" +
                         ") SALT_BUCKETS=2, COMPRESSION='GZ', TTL=31622400");
-                stmt = conn.prepareStatement("UPSERT INTO " + t[i] + "(BUCKET, TIMESTAMP, SRC_LOCATION, DST_LOCATION, B, C) VALUES(?,?,?,?,?,?)");
+                stmt = conn.prepareStatement("UPSERT INTO " + t[i] + "(BUCKET, \"TIMESTAMP\", SRC_LOCATION, DST_LOCATION, B, C) VALUES(?,?,?,?,?,?)");
                 stmt.setString(1, "5SEC");
                 stmt.setString(3, "Tr/Bal");
                 stmt.setString(4, "Tr/Bal");
@@ -772,25 +767,25 @@ public class HashJoinMoreIT extends ParallelStatsDisabledIT {
                 stmt.execute();
                 
                 String q =
-                        "SELECT C.BUCKET, C.TIMESTAMP FROM (\n" +
-                        "     SELECT E.BUCKET as BUCKET, L.BUCKET as LBUCKET, E.TIMESTAMP as TIMESTAMP, L.TIMESTAMP as LTIMESTAMP FROM\n" +
-                        "        (SELECT BUCKET, TIMESTAMP FROM EVENT_COUNT\n" +
+                        "SELECT C.BUCKET, C.\"TIMESTAMP\" FROM (\n" +
+                        "     SELECT E.BUCKET as BUCKET, L.BUCKET as LBUCKET, E.\"TIMESTAMP\" as TIMESTAMP, L.\"TIMESTAMP\" as LTIMESTAMP FROM\n" +
+                        "        (SELECT BUCKET, \"TIMESTAMP\"  FROM EVENT_COUNT\n" +
                         "             WHERE BUCKET = '5SEC' AND LOCATION = 'Tr/Bal'\n" +
-                        "                 AND TIMESTAMP <= 1462993520000000000 AND TIMESTAMP > 1462993420000000000\n" +
+                        "                 AND \"TIMESTAMP\"  <= 1462993520000000000 AND \"TIMESTAMP\"  > 1462993420000000000\n" +
                         "        ) E\n" +
                         "        JOIN\n" +
-                        "         (SELECT BUCKET, TIMESTAMP FROM "+ t[i] +"\n" +
+                        "         (SELECT BUCKET, \"TIMESTAMP\"  FROM "+ t[i] +"\n" +
                         "             WHERE BUCKET = '5SEC' AND SRC_LOCATION = 'Tr/Bal' AND SRC_LOCATION = DST_LOCATION\n" +
-                        "                 AND TIMESTAMP <= 1462993520000000000 AND TIMESTAMP > 1462993420000000000\n" +
+                        "                 AND \"TIMESTAMP\"  <= 1462993520000000000 AND \"TIMESTAMP\"  > 1462993420000000000\n" +
                         "         ) L\n" +
-                        "     ON L.BUCKET = E.BUCKET AND L.TIMESTAMP = E.TIMESTAMP\n" +
+                        "     ON L.BUCKET = E.BUCKET AND L.\"TIMESTAMP\"  = E.\"TIMESTAMP\"\n" +
                         " ) C\n" +
-                        " GROUP BY C.BUCKET, C.TIMESTAMP";
+                        " GROUP BY C.BUCKET, C.\"TIMESTAMP\"";
                     
                 String p = i == 0 ?
                         "CLIENT PARALLEL 2-WAY SKIP SCAN ON 2 RANGES OVER EVENT_COUNT [0,'5SEC',~1462993520000000000,'Tr/Bal'] - [1,'5SEC',~1462993420000000000,'Tr/Bal']\n" +
                         "    SERVER FILTER BY FIRST KEY ONLY\n" +
-                        "    SERVER AGGREGATE INTO DISTINCT ROWS BY [E.BUCKET, E.TIMESTAMP]\n" +
+                        "    SERVER AGGREGATE INTO DISTINCT ROWS BY [E.BUCKET, \"E.TIMESTAMP\"]\n" +
                         "CLIENT MERGE SORT\n" +
                         "    PARALLEL INNER-JOIN TABLE 0 (SKIP MERGE)\n" +
                         "        CLIENT PARALLEL 2-WAY SKIP SCAN ON 2 RANGES OVER " + t[i] + " [0,'5SEC',~1462993520000000000,'Tr/Bal'] - [1,'5SEC',~1462993420000000000,'Tr/Bal']\n" +
@@ -799,7 +794,7 @@ public class HashJoinMoreIT extends ParallelStatsDisabledIT {
                         :
                         "CLIENT PARALLEL 2-WAY SKIP SCAN ON 2 RANGES OVER EVENT_COUNT [0,'5SEC',~1462993520000000000,'Tr/Bal'] - [1,'5SEC',~1462993420000000000,'Tr/Bal']\n" +
                         "    SERVER FILTER BY FIRST KEY ONLY\n" +
-                        "    SERVER AGGREGATE INTO DISTINCT ROWS BY [E.BUCKET, E.TIMESTAMP]\n" +
+                        "    SERVER AGGREGATE INTO DISTINCT ROWS BY [E.BUCKET, \"E.TIMESTAMP\"]\n" +
                         "CLIENT MERGE SORT\n" +
                         "    PARALLEL INNER-JOIN TABLE 0 (SKIP MERGE)\n" +
                         "        CLIENT PARALLEL 2-WAY SKIP SCAN ON 2 RANGES OVER " + t[i] + " [0,'5SEC',1462993420000000001,'Tr/Bal'] - [1,'5SEC',1462993520000000000,'Tr/Bal']\n" +

http://git-wip-us.apache.org/repos/asf/phoenix/blob/895cb1df/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortMergeJoinMoreIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortMergeJoinMoreIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortMergeJoinMoreIT.java
index a08ba39..992e55f 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortMergeJoinMoreIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortMergeJoinMoreIT.java
@@ -186,13 +186,8 @@ public class SortMergeJoinMoreIT extends ParallelStatsDisabledIT {
                     + "        colA2 VARCHAR " + "CONSTRAINT PK PRIMARY KEY" + "(pkA)" + ")";
 
             String ddlB = "CREATE TABLE " + tableB + "   (pkB INTEGER NOT NULL PRIMARY KEY, " + "    colB INTEGER)";
-            stmt = conn.prepareStatement(ddlA);
-            stmt.execute();
-            stmt.close();
-
-            stmt = conn.prepareStatement(ddlB);
-            stmt.execute();
-            stmt.close();
+            conn.createStatement().execute(ddlA);
+            conn.createStatement().execute(ddlB);
 
             String upsertA = "UPSERT INTO " + tableA + " (pkA, colA1, colA2) VALUES(?, ?, ?)";
             stmt = conn.prepareStatement(upsertA);
@@ -313,16 +308,16 @@ public class SortMergeJoinMoreIT extends ParallelStatsDisabledIT {
                     "CREATE TABLE IF NOT EXISTS " + eventCountTableName + " (\n" +
                     "        BUCKET VARCHAR,\n" +
                     "        TIMESTAMP_DATE TIMESTAMP,\n" +
-                    "        TIMESTAMP UNSIGNED_LONG NOT NULL,\n" +
+                    "        \"TIMESTAMP\" UNSIGNED_LONG NOT NULL,\n" +
                     "        LOCATION VARCHAR,\n" +
                     "        A VARCHAR,\n" +
                     "        B VARCHAR,\n" +
                     "        C VARCHAR,\n" +
                     "        D UNSIGNED_LONG,\n" +
                     "        E FLOAT\n" +
-                    "    CONSTRAINT pk PRIMARY KEY (BUCKET, TIMESTAMP DESC, LOCATION, A, B, C)\n" +
+                    "    CONSTRAINT pk PRIMARY KEY (BUCKET, \"TIMESTAMP\" DESC, LOCATION, A, B, C)\n" +
                     ") SALT_BUCKETS=2, COMPRESSION='GZ', TTL=31622400");
-            PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " + eventCountTableName + "(BUCKET, TIMESTAMP, LOCATION, A, B, C) VALUES(?,?,?,?,?,?)");
+            PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " + eventCountTableName + "(BUCKET, \"TIMESTAMP\", LOCATION, A, B, C) VALUES(?,?,?,?,?,?)");
             stmt.setString(1, "5SEC");
             stmt.setString(3, "Tr/Bal");
             stmt.setString(4, "A1");
@@ -374,7 +369,7 @@ public class SortMergeJoinMoreIT extends ParallelStatsDisabledIT {
                         "CREATE TABLE IF NOT EXISTS " + t[i] + " (\n" +
                                 "        BUCKET VARCHAR,\n" +
                                 "        TIMESTAMP_DATE TIMESTAMP,\n" +
-                                "        TIMESTAMP UNSIGNED_LONG NOT NULL,\n" +
+                                "        \"TIMESTAMP\" UNSIGNED_LONG NOT NULL,\n" +
                                 "        SRC_LOCATION VARCHAR,\n" +
                                 "        DST_LOCATION VARCHAR,\n" +
                                 "        B VARCHAR,\n" +
@@ -383,9 +378,9 @@ public class SortMergeJoinMoreIT extends ParallelStatsDisabledIT {
                                 "        G UNSIGNED_LONG,\n" +
                                 "        H UNSIGNED_LONG,\n" +
                                 "        I UNSIGNED_LONG\n" +
-                                "    CONSTRAINT pk PRIMARY KEY (BUCKET, TIMESTAMP" + (i == 0 ? " DESC" : "") + ", SRC_LOCATION, DST_LOCATION, B, C)\n" +
+                                "    CONSTRAINT pk PRIMARY KEY (BUCKET, \"TIMESTAMP\"" + (i == 0 ? " DESC" : "") + ", SRC_LOCATION, DST_LOCATION, B, C)\n" +
                         ") SALT_BUCKETS=2, COMPRESSION='GZ', TTL=31622400");
-                stmt = conn.prepareStatement("UPSERT INTO " + t[i] + "(BUCKET, TIMESTAMP, SRC_LOCATION, DST_LOCATION, B, C) VALUES(?,?,?,?,?,?)");
+                stmt = conn.prepareStatement("UPSERT INTO " + t[i] + "(BUCKET, \"TIMESTAMP\", SRC_LOCATION, DST_LOCATION, B, C) VALUES(?,?,?,?,?,?)");
                 stmt.setString(1, "5SEC");
                 stmt.setString(3, "Tr/Bal");
                 stmt.setString(4, "Tr/Bal");
@@ -421,10 +416,10 @@ public class SortMergeJoinMoreIT extends ParallelStatsDisabledIT {
                         "             GROUP BY BUCKET, TIMESTAMP, LOCATION\n" +
                         "        ) E\n" +
                         "        JOIN\n" +
-                        "         (SELECT BUCKET, TIMESTAMP FROM "+ t[i] +"\n" +
+                        "         (SELECT BUCKET, \"TIMESTAMP\" FROM "+ t[i] +"\n" +
                         "             WHERE BUCKET = '5SEC' AND SRC_LOCATION = 'Tr/Bal' AND SRC_LOCATION = DST_LOCATION\n" +
-                        "                 AND TIMESTAMP <= 1462993520000000000 AND TIMESTAMP > 1462993420000000000\n" +
-                        "             GROUP BY BUCKET, TIMESTAMP, SRC_LOCATION, DST_LOCATION\n" +
+                        "                 AND \"TIMESTAMP\" <= 1462993520000000000 AND \"TIMESTAMP\" > 1462993420000000000\n" +
+                        "             GROUP BY BUCKET, \"TIMESTAMP\", SRC_LOCATION, DST_LOCATION\n" +
                         "         ) L\n" +
                         "     ON L.BUCKET = E.BUCKET AND L.TIMESTAMP = E.TIMESTAMP\n" +
                         " ) C\n" +
@@ -441,10 +436,10 @@ public class SortMergeJoinMoreIT extends ParallelStatsDisabledIT {
                         "AND (SKIP MERGE)\n" +
                         "    CLIENT PARALLEL 2-WAY SKIP SCAN ON 2 RANGES OVER " + t[i] + " [0,'5SEC',~1462993520000000000,'Tr/Bal'] - [1,'5SEC',~1462993420000000000,'Tr/Bal']\n" +
                         "        SERVER FILTER BY FIRST KEY ONLY AND SRC_LOCATION = DST_LOCATION\n" +
-                        "        SERVER DISTINCT PREFIX FILTER OVER [BUCKET, TIMESTAMP, SRC_LOCATION, DST_LOCATION]\n" +
-                        "        SERVER AGGREGATE INTO ORDERED DISTINCT ROWS BY [BUCKET, TIMESTAMP, SRC_LOCATION, DST_LOCATION]\n" +
+                        "        SERVER DISTINCT PREFIX FILTER OVER [BUCKET, \"TIMESTAMP\", SRC_LOCATION, DST_LOCATION]\n" +
+                        "        SERVER AGGREGATE INTO ORDERED DISTINCT ROWS BY [BUCKET, \"TIMESTAMP\", SRC_LOCATION, DST_LOCATION]\n" +
                         "    CLIENT MERGE SORT\n" +
-                        "    CLIENT SORTED BY [BUCKET, TIMESTAMP]\n" +
+                        "    CLIENT SORTED BY [BUCKET, \"TIMESTAMP\"]\n" +
                         "CLIENT SORTED BY [E.BUCKET, E.TIMESTAMP]\n" +
                         "CLIENT AGGREGATE INTO DISTINCT ROWS BY [E.BUCKET, E.TIMESTAMP]"
                         :
@@ -458,8 +453,8 @@ public class SortMergeJoinMoreIT extends ParallelStatsDisabledIT {
                         "AND (SKIP MERGE)\n" +
                         "    CLIENT PARALLEL 2-WAY SKIP SCAN ON 2 RANGES OVER " + t[i] + " [0,'5SEC',1462993420000000001,'Tr/Bal'] - [1,'5SEC',1462993520000000000,'Tr/Bal']\n" +
                         "        SERVER FILTER BY FIRST KEY ONLY AND SRC_LOCATION = DST_LOCATION\n" +
-                        "        SERVER DISTINCT PREFIX FILTER OVER [BUCKET, TIMESTAMP, SRC_LOCATION, DST_LOCATION]\n" +
-                        "        SERVER AGGREGATE INTO ORDERED DISTINCT ROWS BY [BUCKET, TIMESTAMP, SRC_LOCATION, DST_LOCATION]\n" +
+                        "        SERVER DISTINCT PREFIX FILTER OVER [BUCKET, \"TIMESTAMP\", SRC_LOCATION, DST_LOCATION]\n" +
+                        "        SERVER AGGREGATE INTO ORDERED DISTINCT ROWS BY [BUCKET, \"TIMESTAMP\", SRC_LOCATION, DST_LOCATION]\n" +
                         "    CLIENT MERGE SORT\n" +
                         "CLIENT SORTED BY [E.BUCKET, E.TIMESTAMP]\n" +
                         "CLIENT AGGREGATE INTO DISTINCT ROWS BY [E.BUCKET, E.TIMESTAMP]";

http://git-wip-us.apache.org/repos/asf/phoenix/blob/895cb1df/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortOrderIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortOrderIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortOrderIT.java
index c810844..655dbb1 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortOrderIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortOrderIT.java
@@ -370,7 +370,7 @@ public class SortOrderIT extends ParallelStatsDisabledIT {
     @Test
     public void queryDescDateWithExplicitOrderBy() throws Exception {
         String table = generateUniqueName();
-        String ddl = "CREATE table " + table + " (c1 CHAR(1) NOT NULL, c2 CHAR(1) NOT NULL, d1 DATE NOT NULL, c3 CHAR(1) NOT NULL " +
+        String ddl = "CREATE table " + table + " (c1 CHAR(1) NOT NULL, c2 CHAR(1) NOT NULL, d1 \"DATE\" NOT NULL, c3 CHAR(1) NOT NULL " +
             "constraint pk primary key (c1, c2, d1 DESC, c3))";
         Object[] row1 = {"1", "2", date(10, 11, 2001), "3"};
         Object[] row2 = {"1", "2", date(10, 11, 2003), "3"};
@@ -408,7 +408,7 @@ public class SortOrderIT extends ParallelStatsDisabledIT {
     @Test
     public void lessThanLeadingDescCompositePK() throws Exception {
         String table = generateUniqueName();
-        String ddl = "CREATE table " + table + " (id INTEGER NOT NULL, \"DATE\" DATE NOT NULL constraint pk primary key (id DESC, date))";
+        String ddl = "CREATE table " + table + " (id INTEGER NOT NULL, \"DATE\" DATE NOT NULL constraint pk primary key (id DESC, \"DATE\"))";
         Object[][] insertedRows = new Object[][]{{1, date(1, 1, 2012)}, {3, date(1, 1, 2013)}, {2, date(1, 1, 2011)}};
         Object[][] expectedRows = new Object[][]{{1, date(1, 1, 2012)}};
         runQueryTest(ddl, upsert("id", "date"), insertedRows, expectedRows, new WhereCondition("id", "<", "2"),
@@ -418,10 +418,10 @@ public class SortOrderIT extends ParallelStatsDisabledIT {
     @Test
     public void lessThanTrailingDescCompositePK() throws Exception {
         String table = generateUniqueName();
-        String ddl = "CREATE table " + table + " (id INTEGER NOT NULL, \"DATE\" DATE NOT NULL constraint pk primary key (id DESC, date))";
+        String ddl = "CREATE table " + table + " (id INTEGER NOT NULL, \"DATE\" DATE NOT NULL constraint pk primary key (id DESC, \"DATE\"))";
         Object[][] insertedRows = new Object[][]{{1, date(1, 1, 2002)}, {3, date(1, 1, 2003)}, {2, date(1, 1, 2001)}};
         Object[][] expectedRows = new Object[][]{{2, date(1, 1, 2001)}};
-        runQueryTest(ddl, upsert("id", "DATE"), insertedRows, expectedRows, new WhereCondition("DATE", "<", "TO_DATE('02-02-2001','mm-dd-yyyy')"),
+        runQueryTest(ddl, upsert("id", "\"DATE\""), insertedRows, expectedRows, new WhereCondition("\"DATE\"", "<", "TO_DATE('02-02-2001','mm-dd-yyyy')"),
             table);
     }
     

http://git-wip-us.apache.org/repos/asf/phoenix/blob/895cb1df/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java
index b8f8177..e4479ac 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java
@@ -152,7 +152,7 @@ public class TenantSpecificTablesDDLIT extends BaseTenantSpecificTablesIT {
     public void testCreationOfParentTableFailsOnTenantSpecificConnection() throws Exception {
         try {
             createTestTable(PHOENIX_JDBC_TENANT_SPECIFIC_URL, "CREATE TABLE " + generateUniqueName() + "( \n" + 
-                    "                user VARCHAR ,\n" + 
+                    "                \"user\" VARCHAR ,\n" + 
                     "                id INTEGER not null primary key desc\n" + 
                     "                ) ");
             fail();
@@ -178,7 +178,7 @@ public class TenantSpecificTablesDDLIT extends BaseTenantSpecificTablesIT {
         }
         String newDDL =
         "CREATE TABLE DIFFSCHEMA." + PARENT_TABLE_NAME + " ( \n" + 
-        "                user VARCHAR ,\n" + 
+        "                \"user\" VARCHAR ,\n" + 
         "                tenant_id VARCHAR(5) NOT NULL,\n" + 
         "                tenant_type_id VARCHAR(3) NOT NULL, \n" + 
         "                id INTEGER NOT NULL\n" + 
@@ -199,7 +199,7 @@ public class TenantSpecificTablesDDLIT extends BaseTenantSpecificTablesIT {
     @Test(expected=ColumnAlreadyExistsException.class)
     public void testTenantSpecificTableCannotOverrideParentCol() throws SQLException {
         createTestTable(PHOENIX_JDBC_TENANT_SPECIFIC_URL, "CREATE VIEW " + generateUniqueName() + " ( \n" + 
-                "                user INTEGER) AS SELECT *\n" + 
+                "                \"user\" INTEGER) AS SELECT *\n" + 
                 "                FROM " + PARENT_TABLE_NAME);
     }
     
@@ -285,7 +285,7 @@ public class TenantSpecificTablesDDLIT extends BaseTenantSpecificTablesIT {
             
             // try removing a non-PK col, which is allowed
             try {
-                conn.createStatement().execute("alter table " + PARENT_TABLE_NAME + " drop column user");
+                conn.createStatement().execute("alter table " + PARENT_TABLE_NAME + " drop column \"user\"");
             }
             catch (SQLException expected) {
                 fail("We should be able to drop a non pk base table column");
@@ -525,7 +525,7 @@ public class TenantSpecificTablesDDLIT extends BaseTenantSpecificTablesIT {
             // make sure tenants see parent table's columns and their own
             rs = meta.getColumns(null, null, StringUtil.escapeLike(TENANT_TABLE_NAME), null);
             assertTrue(rs.next());
-            assertColumnMetaData(rs, null, TENANT_TABLE_NAME, "user", 1);
+            assertColumnMetaData(rs, null, TENANT_TABLE_NAME, "\"user\"", 1);
             assertTrue(rs.next());
             // (tenant_id column is not visible in tenant-specific connection)
             assertColumnMetaData(rs, null, TENANT_TABLE_NAME, "tenant_type_id", 2);
@@ -538,7 +538,7 @@ public class TenantSpecificTablesDDLIT extends BaseTenantSpecificTablesIT {
             
             rs = meta.getColumns(null, null, StringUtil.escapeLike(TENANT_TABLE_NAME_NO_TENANT_TYPE_ID), null);
             assertTrue(rs.next());
-            assertColumnMetaData(rs, null, TENANT_TABLE_NAME_NO_TENANT_TYPE_ID, "user", 1);
+            assertColumnMetaData(rs, null, TENANT_TABLE_NAME_NO_TENANT_TYPE_ID, "\"user\"", 1);
             assertTrue(rs.next());
             // (tenant_id column is not visible in tenant-specific connection)
             assertColumnMetaData(rs, null, TENANT_TABLE_NAME_NO_TENANT_TYPE_ID, "id", 2);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/895cb1df/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDMLIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDMLIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDMLIT.java
index 76625c0..55d38fd 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDMLIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDMLIT.java
@@ -174,12 +174,12 @@ public class TenantSpecificTablesDMLIT extends BaseTenantSpecificTablesIT {
         try {
             conn.setAutoCommit(true);
             conn.createStatement().executeUpdate("delete from " + PARENT_TABLE_NAME);
-            conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, user) values ('AC/DC', 'abc', 1, 'Bon Scott')");
-            conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, user) values ('" + TENANT_ID + "', '" + TENANT_TYPE_ID + "', 1, 'Billy Gibbons')");
-            conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, user) values ('" + TENANT_ID + "', 'def', 1, 'Billy Gibbons')");
+            conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, \"user\") values ('AC/DC', 'abc', 1, 'Bon Scott')");
+            conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, \"user\") values ('" + TENANT_ID + "', '" + TENANT_TYPE_ID + "', 1, 'Billy Gibbons')");
+            conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, \"user\") values ('" + TENANT_ID + "', 'def', 1, 'Billy Gibbons')");
             
             conn = DriverManager.getConnection(PHOENIX_JDBC_TENANT_SPECIFIC_URL, props);
-            ResultSet rs = conn.createStatement().executeQuery("select user from " + TENANT_TABLE_NAME);
+            ResultSet rs = conn.createStatement().executeQuery("select \"user\" from " + TENANT_TABLE_NAME);
             assertTrue("Expected 1 row in result set", rs.next());
             assertEquals("Billy Gibbons", rs.getString(1));
             assertFalse("Expected 1 row in result set", rs.next());
@@ -202,9 +202,9 @@ public class TenantSpecificTablesDMLIT extends BaseTenantSpecificTablesIT {
         try {
             conn.setAutoCommit(true);
             conn.createStatement().executeUpdate("delete from " + PARENT_TABLE_NAME);
-            conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, user) values ('AC/DC', 'abc', 1, 'Bon Scott')");
-            conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, user) values ('" + TENANT_ID + "', '" + TENANT_TYPE_ID + "', 1, 'Billy Gibbons')");
-            conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, user) values ('" + TENANT_ID + "', 'def', 1, 'Billy Gibbons')");
+            conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, \"user\") values ('AC/DC', 'abc', 1, 'Bon Scott')");
+            conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, \"user\") values ('" + TENANT_ID + "', '" + TENANT_TYPE_ID + "', 1, 'Billy Gibbons')");
+            conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, \"user\") values ('" + TENANT_ID + "', 'def', 1, 'Billy Gibbons')");
             
             conn = DriverManager.getConnection(PHOENIX_JDBC_TENANT_SPECIFIC_URL, props);
             conn.setAutoCommit(true);
@@ -231,14 +231,14 @@ public class TenantSpecificTablesDMLIT extends BaseTenantSpecificTablesIT {
         try {
             conn.setAutoCommit(true);
             conn.createStatement().executeUpdate("delete from " + PARENT_TABLE_NAME);
-            conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, user) values ('AC/DC', 'abc', 1, 'Bon Scott')");
-            conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, user) values ('" + TENANT_ID + "', '" + TENANT_TYPE_ID + "', 1, 'Billy Gibbons')");
-            conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, user) values ('" + TENANT_ID + "', 'def', 1, 'Billy Gibbons')");
+            conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, \"user\") values ('AC/DC', 'abc', 1, 'Bon Scott')");
+            conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, \"user\") values ('" + TENANT_ID + "', '" + TENANT_TYPE_ID + "', 1, 'Billy Gibbons')");
+            conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, \"user\") values ('" + TENANT_ID + "', 'def', 1, 'Billy Gibbons')");
             
             Connection tsConn = DriverManager.getConnection(PHOENIX_JDBC_TENANT_SPECIFIC_URL, props);
             tsConn.setAutoCommit(true);
-            tsConn.createStatement().executeUpdate("create index idx1 on " + TENANT_TABLE_NAME + "(user)");
-            int count = tsConn.createStatement().executeUpdate("delete from " + TENANT_TABLE_NAME + " where user='Billy Gibbons'");
+            tsConn.createStatement().executeUpdate("create index idx1 on " + TENANT_TABLE_NAME + "(\"user\")");
+            int count = tsConn.createStatement().executeUpdate("delete from " + TENANT_TABLE_NAME + " where \"user\"='Billy Gibbons'");
             assertEquals("Expected 1 row have been deleted", 1, count);
             ResultSet rs = tsConn.createStatement().executeQuery("select * from " + TENANT_TABLE_NAME);
             assertFalse("Expected no rows in result set", rs.next());
@@ -261,9 +261,9 @@ public class TenantSpecificTablesDMLIT extends BaseTenantSpecificTablesIT {
         try {
             conn.setAutoCommit(true);
             conn.createStatement().executeUpdate("delete from " + PARENT_TABLE_NAME_NO_TENANT_TYPE_ID);
-            conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME_NO_TENANT_TYPE_ID + " (tenant_id, id, user) values ('AC/DC', 1, 'Bon Scott')");
-            conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME_NO_TENANT_TYPE_ID + " (tenant_id, id, user) values ('" + TENANT_ID + "', 1, 'Billy Gibbons')");
-            conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME_NO_TENANT_TYPE_ID + " (tenant_id, id, user) values ('" + TENANT_ID + "', 2, 'Billy Gibbons')");
+            conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME_NO_TENANT_TYPE_ID + " (tenant_id, id, \"user\") values ('AC/DC', 1, 'Bon Scott')");
+            conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME_NO_TENANT_TYPE_ID + " (tenant_id, id, \"user\") values ('" + TENANT_ID + "', 1, 'Billy Gibbons')");
+            conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME_NO_TENANT_TYPE_ID + " (tenant_id, id, \"user\") values ('" + TENANT_ID + "', 2, 'Billy Gibbons')");
             
             Connection tsConn = DriverManager.getConnection(PHOENIX_JDBC_TENANT_SPECIFIC_URL, props);
             tsConn.setAutoCommit(true);
@@ -289,9 +289,9 @@ public class TenantSpecificTablesDMLIT extends BaseTenantSpecificTablesIT {
         try {
             conn.setAutoCommit(true);
             conn.createStatement().executeUpdate("delete from " + PARENT_TABLE_NAME);
-            conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, user) values ('AC/DC', 'abc', 1, 'Bon Scott')");
-            conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, user) values ('" + TENANT_ID + "', '" + TENANT_TYPE_ID + "', 1, 'Billy Gibbons')");
-            conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, user) values ('" + TENANT_ID + "', 'def', 1, 'Billy Gibbons')");
+            conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, \"user\") values ('AC/DC', 'abc', 1, 'Bon Scott')");
+            conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, \"user\") values ('" + TENANT_ID + "', '" + TENANT_TYPE_ID + "', 1, 'Billy Gibbons')");
+            conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, \"user\") values ('" + TENANT_ID + "', 'def', 1, 'Billy Gibbons')");
             
             analyzeTable(tsConn, PARENT_TABLE_NAME);
             tsConn.createStatement().execute("delete from " + TENANT_TABLE_NAME);
@@ -315,9 +315,9 @@ public class TenantSpecificTablesDMLIT extends BaseTenantSpecificTablesIT {
         try {
             conn.setAutoCommit(true);
             conn.createStatement().executeUpdate("delete from " + PARENT_TABLE_NAME_NO_TENANT_TYPE_ID);
-            conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME_NO_TENANT_TYPE_ID + " (tenant_id, id, user) values ('AC/DC', 1, 'Bon Scott')");
-            conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME_NO_TENANT_TYPE_ID + " (tenant_id, id, user) values ('" + TENANT_ID + "', 1, 'Billy Gibbons')");
-            conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME_NO_TENANT_TYPE_ID + " (tenant_id, id, user) values ('" + TENANT_ID + "', 2, 'Billy Gibbons')");
+            conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME_NO_TENANT_TYPE_ID + " (tenant_id, id, \"user\") values ('AC/DC', 1, 'Bon Scott')");
+            conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME_NO_TENANT_TYPE_ID + " (tenant_id, id, \"user\") values ('" + TENANT_ID + "', 1, 'Billy Gibbons')");
+            conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME_NO_TENANT_TYPE_ID + " (tenant_id, id, \"user\") values ('" + TENANT_ID + "', 2, 'Billy Gibbons')");
             
             tsConn.createStatement().execute("drop view " + TENANT_TABLE_NAME_NO_TENANT_TYPE_ID);
             
@@ -340,12 +340,12 @@ public class TenantSpecificTablesDMLIT extends BaseTenantSpecificTablesIT {
         try {
             conn.setAutoCommit(true);
             conn.createStatement().executeUpdate("delete from " + PARENT_TABLE_NAME);
-            conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, user) values ('AC/DC', 'aaa', 1, 'Bon Scott')");
-            conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, user) values ('" + TENANT_ID + "', '" + TENANT_TYPE_ID + "', 1, 'Billy Gibbons')");
-            conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, user) values ('" + TENANT_ID + "', 'def', 2, 'Billy Gibbons')");
+            conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, \"user\") values ('AC/DC', 'aaa', 1, 'Bon Scott')");
+            conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, \"user\") values ('" + TENANT_ID + "', '" + TENANT_TYPE_ID + "', 1, 'Billy Gibbons')");
+            conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, \"user\") values ('" + TENANT_ID + "', 'def', 2, 'Billy Gibbons')");
             
             analyzeTable(tsConn, TENANT_TABLE_NAME);
-            int count = tsConn.createStatement().executeUpdate("upsert into " + TENANT_TABLE_NAME + "(id, user) select id+100, user from " + TENANT_TABLE_NAME);
+            int count = tsConn.createStatement().executeUpdate("upsert into " + TENANT_TABLE_NAME + "(id, \"user\") select id+100, \"user\" from " + TENANT_TABLE_NAME);
             tsConn.commit();
             assertEquals("Expected 1 row to have been inserted", 1, count);
             
@@ -371,14 +371,14 @@ public class TenantSpecificTablesDMLIT extends BaseTenantSpecificTablesIT {
         try {
             conn.setAutoCommit(true);
             conn.createStatement().executeUpdate("delete from " + PARENT_TABLE_NAME);
-            conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, user) values ('AC/DC', 'aaa', 1, 'Bon Scott')");
-            conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, user) values ('" + TENANT_ID + "', '" + TENANT_TYPE_ID + "', 1, 'Billy Gibbons')");
-            conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, user) values ('" + TENANT_ID + "', 'def', 2, 'Billy Gibbons')");
+            conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, \"user\") values ('AC/DC', 'aaa', 1, 'Bon Scott')");
+            conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, \"user\") values ('" + TENANT_ID + "', '" + TENANT_TYPE_ID + "', 1, 'Billy Gibbons')");
+            conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_id, tenant_type_id, id, \"user\") values ('" + TENANT_ID + "', 'def', 2, 'Billy Gibbons')");
             
             analyzeTable(tsConn, TENANT_TABLE_NAME);
             tsConn.setAutoCommit(true);
-            int count = tsConn.createStatement().executeUpdate("upsert into " + TENANT_TABLE_NAME + "(id, user)"
-                    + "select id+100, user from " + anotherTableName + " where id=2");
+            int count = tsConn.createStatement().executeUpdate("upsert into " + TENANT_TABLE_NAME + "(id, \"user\")"
+                    + "select id+100, \"user\" from " + anotherTableName + " where id=2");
             assertEquals("Expected 1 row to have been inserted", 1, count);
             ResultSet rs = tsConn.createStatement().executeQuery("select count(*) from " + TENANT_TABLE_NAME);
             rs.next();
@@ -395,7 +395,7 @@ public class TenantSpecificTablesDMLIT extends BaseTenantSpecificTablesIT {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(PHOENIX_JDBC_TENANT_SPECIFIC_URL, props);
         try {
-            int count = conn.createStatement().executeUpdate("upsert into " + TENANT_TABLE_NAME + " (id, user) values (1, 'Bon Scott')");
+            int count = conn.createStatement().executeUpdate("upsert into " + TENANT_TABLE_NAME + " (id, \"user\") values (1, 'Bon Scott')");
             conn.commit();
             assertEquals("Expected 1 row to have been inserted", 1, count);
             ResultSet rs = conn.createStatement().executeQuery("select count(*) from " + TENANT_TABLE_NAME);
@@ -414,10 +414,10 @@ public class TenantSpecificTablesDMLIT extends BaseTenantSpecificTablesIT {
         try {
             ResultSet rs = conn.createStatement().executeQuery("select * from " + PARENT_TABLE_NAME);
             assertFalse(rs.next());
-            conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_type_id, id, user) values ('" + TENANT_TYPE_ID + "', 1, 'Billy Gibbons')");
+            conn.createStatement().executeUpdate("upsert into " + PARENT_TABLE_NAME + " (tenant_type_id, id, \"user\") values ('" + TENANT_TYPE_ID + "', 1, 'Billy Gibbons')");
             conn.commit();
             analyzeTable(conn, PARENT_TABLE_NAME);
-            rs = conn.createStatement().executeQuery("select user from " + PARENT_TABLE_NAME);
+            rs = conn.createStatement().executeQuery("select \"user\" from " + PARENT_TABLE_NAME);
             assertTrue(rs.next());
             assertEquals(rs.getString(1),"Billy Gibbons");
             assertFalse(rs.next());


[20/26] phoenix git commit: PHOENIX-3608 KeyRange interset should return EMPTY_RANGE when one of it is NULL_RANGE(Rajeshbabu)

Posted by ma...@apache.org.
PHOENIX-3608 KeyRange interset should return EMPTY_RANGE when one of it is NULL_RANGE(Rajeshbabu)


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

Branch: refs/heads/calcite
Commit: e7ef25eca2468e6d0a154b5e3539219f07748f22
Parents: 895cb1d
Author: Rajeshbabu Chintaguntla <ra...@apache.org>
Authored: Fri Jan 20 19:05:06 2017 +0530
Committer: Rajeshbabu Chintaguntla <ra...@apache.org>
Committed: Fri Jan 20 19:05:06 2017 +0530

----------------------------------------------------------------------
 .../src/main/java/org/apache/phoenix/query/KeyRange.java        | 5 ++---
 1 file changed, 2 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/e7ef25ec/phoenix-core/src/main/java/org/apache/phoenix/query/KeyRange.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/KeyRange.java b/phoenix-core/src/main/java/org/apache/phoenix/query/KeyRange.java
index f4bf793..babce9d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/KeyRange.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/KeyRange.java
@@ -356,10 +356,9 @@ public class KeyRange implements Writable {
         boolean newUpperInclusive;
         // Special case for null, is it is never included another range
         // except for null itself.
-        if (this == IS_NULL_RANGE) {
-            if (range == IS_NULL_RANGE) {
+        if (this == IS_NULL_RANGE && range == IS_NULL_RANGE) {
                 return IS_NULL_RANGE;
-            }
+        } else if(this == IS_NULL_RANGE || range == IS_NULL_RANGE) {
             return EMPTY_RANGE;
         }
         if (lowerUnbound()) {


[22/26] phoenix git commit: PHOENIX-3611 Cache for client connections will expire (and close) entries in LRU fashion.

Posted by ma...@apache.org.
PHOENIX-3611 Cache for client connections will expire (and close) entries in LRU fashion.

Signed-off-by: Andrew Purtell <ap...@apache.org>


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

Branch: refs/heads/calcite
Commit: 9b7f3ca5b2e6bbf0757d99fbe48b3065e79d6066
Parents: 4e4f7dd
Author: Geoffrey <gj...@salesforce.com>
Authored: Thu Jan 19 16:08:20 2017 -0800
Committer: Andrew Purtell <ap...@apache.org>
Committed: Fri Jan 20 16:16:50 2017 -0800

----------------------------------------------------------------------
 .../org/apache/phoenix/jdbc/PhoenixDriver.java  | 99 +++++++++++++-------
 .../org/apache/phoenix/query/QueryServices.java |  5 +-
 .../phoenix/query/QueryServicesOptions.java     |  2 +
 3 files changed, 70 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/9b7f3ca5/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDriver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDriver.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDriver.java
index fa31dd9..b2acacf 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDriver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDriver.java
@@ -23,20 +23,13 @@ import java.sql.Connection;
 import java.sql.DriverManager;
 import java.sql.SQLException;
 import java.util.Properties;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
+import java.util.concurrent.*;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 import javax.annotation.concurrent.GuardedBy;
 
+import com.google.common.cache.*;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.exception.SQLExceptionInfo;
@@ -50,7 +43,6 @@ import org.apache.phoenix.query.QueryServicesOptions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.annotations.VisibleForTesting;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 
 
@@ -147,13 +139,43 @@ public final class PhoenixDriver extends PhoenixEmbeddedDriver {
     }
 
     // One entry per cluster here
-    private final ConcurrentMap<ConnectionInfo,ConnectionQueryServices> connectionQueryServicesMap = new ConcurrentHashMap<ConnectionInfo,ConnectionQueryServices>(3);
+    private final Cache<ConnectionInfo, ConnectionQueryServices> connectionQueryServicesCache =
+        initializeConnectionCache();
 
     public PhoenixDriver() { // for Squirrel
         // Use production services implementation
         super();
     }
-    
+
+    private Cache<ConnectionInfo, ConnectionQueryServices> initializeConnectionCache() {
+        Configuration config = HBaseFactoryProvider.getConfigurationFactory().getConfiguration();
+        int maxCacheSize = config.getInt(QueryServices.CLIENT_CONNECTION_CACHE_MAX_SIZE,
+            QueryServicesOptions.DEFAULT_CLIENT_CONNECTION_CACHE_MAX_SIZE);
+        int maxCacheDuration = config.getInt(QueryServices.CLIENT_CONNECTION_CACHE_MAX_DURATION_MILLISECONDS,
+            QueryServicesOptions.DEFAULT_CLIENT_CONNECTION_CACHE_MAX_DURATION);
+        RemovalListener<ConnectionInfo, ConnectionQueryServices> cacheRemovalListener =
+            new RemovalListener<ConnectionInfo, ConnectionQueryServices>() {
+                @Override
+                public void onRemoval(RemovalNotification<ConnectionInfo, ConnectionQueryServices> notification) {
+                    String connInfoIdentifier = notification.getKey().toString();
+                    logger.debug("Expiring " + connInfoIdentifier + " because of "
+                        + notification.getCause().name());
+
+                    try {
+                        notification.getValue().close();
+                    }
+                    catch (SQLException se) {
+                        logger.error("Error while closing expired cache connection " + connInfoIdentifier, se);
+                    }
+                }
+            };
+        return CacheBuilder.newBuilder()
+            .maximumSize(maxCacheSize)
+            .expireAfterAccess(maxCacheDuration, TimeUnit.MILLISECONDS)
+            .removalListener(cacheRemovalListener)
+            .build();
+    }
+
     // writes guarded by "this"
     private volatile QueryServices services;
     
@@ -206,38 +228,49 @@ public final class PhoenixDriver extends PhoenixEmbeddedDriver {
     }
     
     @Override
-    protected ConnectionQueryServices getConnectionQueryServices(String url, Properties info) throws SQLException {
+    protected ConnectionQueryServices getConnectionQueryServices(String url, final Properties info) throws SQLException {
         try {
             lockInterruptibly(LockMode.READ);
             checkClosed();
             ConnectionInfo connInfo = ConnectionInfo.create(url);
-            QueryServices services = getQueryServices();
-            // Also performs the Kerberos login if the URL/properties request this
-            ConnectionInfo normalizedConnInfo = connInfo.normalize(services.getProps(), info);
-            ConnectionQueryServices connectionQueryServices = connectionQueryServicesMap.get(normalizedConnInfo);
-            if (connectionQueryServices == null) {
-                if (normalizedConnInfo.isConnectionless()) {
-                    connectionQueryServices = new ConnectionlessQueryServicesImpl(services, normalizedConnInfo, info);
-                } else {
-                    connectionQueryServices = new ConnectionQueryServicesImpl(services, normalizedConnInfo, info);
-                }
-                ConnectionQueryServices prevValue = connectionQueryServicesMap.putIfAbsent(normalizedConnInfo, connectionQueryServices);
-                if (prevValue != null) {
-                    connectionQueryServices = prevValue;
-                }
-            }
-            boolean success = false;
             SQLException sqlE = null;
+            boolean success = false;
+            final QueryServices services = getQueryServices();
+            ConnectionQueryServices connectionQueryServices = null;
+            // Also performs the Kerberos login if the URL/properties request this
+            final ConnectionInfo normalizedConnInfo = connInfo.normalize(services.getProps(), info);
             try {
+                connectionQueryServices =
+                    connectionQueryServicesCache.get(normalizedConnInfo, new Callable<ConnectionQueryServices>() {
+                        @Override
+                        public ConnectionQueryServices call() throws Exception {
+                            ConnectionQueryServices connectionQueryServices;
+                            if (normalizedConnInfo.isConnectionless()) {
+                                connectionQueryServices = new ConnectionlessQueryServicesImpl(services, normalizedConnInfo, info);
+                            } else {
+                                connectionQueryServices = new ConnectionQueryServicesImpl(services, normalizedConnInfo, info);
+                            }
+
+                            return connectionQueryServices;
+                        }
+                    });
+
                 connectionQueryServices.init(url, info);
                 success = true;
-            } catch (SQLException e) {
+            } catch (ExecutionException ee){
+                if (ee.getCause() instanceof  SQLException) {
+                    sqlE = (SQLException) ee.getCause();
+                } else {
+                    throw new SQLException(ee);
+                }
+            }
+              catch (SQLException e) {
                 sqlE = e;
             }
             finally {
                 if (!success) {
                     // Remove from map, as initialization failed
-                    connectionQueryServicesMap.remove(normalizedConnInfo);
+                    connectionQueryServicesCache.invalidate(normalizedConnInfo);
                     if (sqlE != null) {
                         throw sqlE;
                     }
@@ -319,8 +352,4 @@ public final class PhoenixDriver extends PhoenixEmbeddedDriver {
         }
     }
 
-    @VisibleForTesting
-    protected ConcurrentMap<ConnectionInfo,ConnectionQueryServices> getCachedConnections() {
-        return this.connectionQueryServicesMap;
-    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/9b7f3ca5/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
index 14d9887..233007f 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
@@ -233,7 +233,10 @@ public interface QueryServices extends SQLCloseable {
     
     public static final String CLIENT_CACHE_ENCODING = "phoenix.table.client.cache.encoding";
     public static final String AUTO_UPGRADE_ENABLED = "phoenix.autoupgrade.enabled";
-	
+
+    public static final String CLIENT_CONNECTION_CACHE_MAX_SIZE = "phoenix.client.connection.cache.max.size";
+    public static final String CLIENT_CONNECTION_CACHE_MAX_DURATION_MILLISECONDS =
+        "phoenix.client.connection.max.duration";
     /**
      * Get executor service used for parallel scans
      */

http://git-wip-us.apache.org/repos/asf/phoenix/blob/9b7f3ca5/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
index 0e8b9d5..df203b5 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
@@ -262,6 +262,8 @@ public class QueryServicesOptions {
     
     public static final String DEFAULT_CLIENT_CACHE_ENCODING = PTableRefFactory.Encoding.OBJECT.toString();
     public static final boolean DEFAULT_AUTO_UPGRADE_ENABLED = true;
+    public static final int DEFAULT_CLIENT_CONNECTION_CACHE_MAX_SIZE = 100;
+    public static final int DEFAULT_CLIENT_CONNECTION_CACHE_MAX_DURATION = 86400000;
 
     @SuppressWarnings("serial")
     public static final Set<String> DEFAULT_QUERY_SERVER_SKIP_WORDS = new HashSet<String>() {


[13/26] phoenix git commit: PHOENIX-3563 Ensure we release ZooKeeper resources allocated by the Tephra client embedded in the Phoenix connection

Posted by ma...@apache.org.
PHOENIX-3563 Ensure we release ZooKeeper resources allocated by the Tephra client embedded in the Phoenix connection


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

Branch: refs/heads/calcite
Commit: d8f4594989c0b73945aaffec5649a0b62ac59724
Parents: a164f03
Author: Andrew Purtell <ap...@apache.org>
Authored: Wed Jan 4 16:48:44 2017 -0800
Committer: Andrew Purtell <ap...@apache.org>
Committed: Sat Jan 7 10:03:23 2017 -0800

----------------------------------------------------------------------
 .../query/ConnectionQueryServicesImpl.java      | 37 ++++++++++++--------
 1 file changed, 23 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/d8f45949/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
index f66b358..be34f66 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
@@ -254,6 +254,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     private final boolean returnSequenceValues ;
 
     private HConnection connection;
+    private ZKClientService txZKClientService;
     private TransactionServiceClient txServiceClient;
     private volatile boolean initialized;
     private volatile int nSequenceSaltBuckets;
@@ -370,15 +371,16 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
 
         int timeOut = props.getInt(HConstants.ZK_SESSION_TIMEOUT, HConstants.DEFAULT_ZK_SESSION_TIMEOUT);
         // Create instance of the tephra zookeeper client
-        ZKClientService tephraZKClientService = new TephraZKClientService(zkQuorumServersString, timeOut, null, ArrayListMultimap.<String, byte[]>create());
-
-        ZKClientService zkClientService = ZKClientServices.delegate(
-                ZKClients.reWatchOnExpire(
-                        ZKClients.retryOnFailure(tephraZKClientService, RetryStrategies.exponentialDelay(500, 2000, TimeUnit.MILLISECONDS))
-                        )
+        txZKClientService = ZKClientServices.delegate(
+            ZKClients.reWatchOnExpire(
+                ZKClients.retryOnFailure(
+                     new TephraZKClientService(zkQuorumServersString, timeOut, null,
+                             ArrayListMultimap.<String, byte[]>create()), 
+                         RetryStrategies.exponentialDelay(500, 2000, TimeUnit.MILLISECONDS))
+                     )
                 );
-        zkClientService.startAndWait();
-        ZKDiscoveryService zkDiscoveryService = new ZKDiscoveryService(zkClientService);
+        txZKClientService.startAndWait();
+        ZKDiscoveryService zkDiscoveryService = new ZKDiscoveryService(txZKClientService);
         PooledClientProvider pooledClientProvider = new PooledClientProvider(
                 config, zkDiscoveryService);
         this.txServiceClient = new TransactionServiceClient(config,pooledClientProvider);
@@ -389,11 +391,12 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             boolean transactionsEnabled = props.getBoolean(
                     QueryServices.TRANSACTIONS_ENABLED,
                     QueryServicesOptions.DEFAULT_TRANSACTIONS_ENABLED);
-            // only initialize the tx service client if needed
+            this.connection = HBaseFactoryProvider.getHConnectionFactory().createConnection(this.config);
+            // only initialize the tx service client if needed and if we succeeded in getting a connection
+            // to HBase
             if (transactionsEnabled) {
                 initTxServiceClient();
             }
-            this.connection = HBaseFactoryProvider.getHConnectionFactory().createConnection(this.config);
         } catch (IOException e) {
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_ESTABLISH_CONNECTION)
             .setRootCause(e).build().buildException();
@@ -463,14 +466,20 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             } finally {
                 try {
                     childServices.clear();
-                    if (renewLeaseExecutor != null) {
-                        renewLeaseExecutor.shutdownNow();
-                    }
                     synchronized (latestMetaDataLock) {
                         latestMetaData = null;
                         latestMetaDataLock.notifyAll();
                     }
-                    if (connection != null) connection.close();
+                    try {
+                        // close the HBase connection
+                        if (connection != null) connection.close();
+                    } finally {
+                        if (renewLeaseExecutor != null) {
+                            renewLeaseExecutor.shutdownNow();
+                        }
+                        // shut down the tx client service if we created one to support transactions
+                        if (this.txZKClientService != null) this.txZKClientService.stopAndWait();
+                    }
                 } catch (IOException e) {
                     if (sqlE == null) {
                         sqlE = ServerUtil.parseServerException(e);


[17/26] phoenix git commit: PHOENIX-3134 varbinary fields bulk load difference between MR/psql and upserts

Posted by ma...@apache.org.
PHOENIX-3134 varbinary fields bulk load difference between MR/psql and upserts


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

Branch: refs/heads/calcite
Commit: a675211909415ca376e432d25f8a8822aadf5712
Parents: 7332fc3
Author: Ankit Singhal <an...@gmail.com>
Authored: Wed Jan 18 13:27:17 2017 +0530
Committer: Ankit Singhal <an...@gmail.com>
Committed: Wed Jan 18 13:27:17 2017 +0530

----------------------------------------------------------------------
 .../expression/function/EncodeFormat.java       |  4 +-
 .../phoenix/mapreduce/CsvBulkImportUtil.java    |  7 +-
 .../phoenix/mapreduce/CsvBulkLoadTool.java      | 12 ++-
 .../org/apache/phoenix/query/QueryServices.java |  3 +
 .../phoenix/query/QueryServicesOptions.java     |  9 ++-
 .../apache/phoenix/schema/types/PBinary.java    |  6 +-
 .../apache/phoenix/schema/types/PVarbinary.java |  5 +-
 .../org/apache/phoenix/util/PhoenixRuntime.java | 11 +++
 .../phoenix/util/csv/CsvUpsertExecutor.java     | 25 ++++++
 .../phoenix/util/json/JsonUpsertExecutor.java   | 44 +++++++++++
 .../mapreduce/CsvBulkImportUtilTest.java        |  8 +-
 .../util/AbstractUpsertExecutorTest.java        | 82 +++++++++++++++++---
 .../phoenix/util/csv/CsvUpsertExecutorTest.java | 26 +++----
 .../util/json/JsonUpsertExecutorTest.java       |  6 ++
 14 files changed, 207 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/a6752119/phoenix-core/src/main/java/org/apache/phoenix/expression/function/EncodeFormat.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/EncodeFormat.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/EncodeFormat.java
index ca6cb66..8130228 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/EncodeFormat.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/EncodeFormat.java
@@ -20,5 +20,7 @@ package org.apache.phoenix.expression.function;
 public enum EncodeFormat {
 
 	HEX, //format for encoding HEX value to bytes
-	BASE62 //format for encoding a base 10 long value to base 62 string
+	BASE62, //format for encoding a base 10 long value to base 62 string
+	BASE64, //format for encoding a base 10 long value to base 64 string
+	ASCII // Plain Text
 };

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a6752119/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/CsvBulkImportUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/CsvBulkImportUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/CsvBulkImportUtil.java
index 9289dbf..ff9ff72 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/CsvBulkImportUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/CsvBulkImportUtil.java
@@ -22,6 +22,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.util.Base64;
 import org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil;
 import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.query.QueryServices;
 
 import com.google.common.annotations.VisibleForTesting;
 
@@ -38,15 +39,19 @@ public class CsvBulkImportUtil {
      * @param quoteChar quote character for the CSV input
      * @param escapeChar escape character for the CSV input
      * @param arrayDelimiter array delimiter character, can be null
+     * @param binaryEncoding 
      */
     public static void initCsvImportJob(Configuration conf, char fieldDelimiter, char quoteChar,
-            char escapeChar, String arrayDelimiter) {
+            char escapeChar, String arrayDelimiter, String binaryEncoding) {
         setChar(conf, CsvToKeyValueMapper.FIELD_DELIMITER_CONFKEY, fieldDelimiter);
         setChar(conf, CsvToKeyValueMapper.QUOTE_CHAR_CONFKEY, quoteChar);
         setChar(conf, CsvToKeyValueMapper.ESCAPE_CHAR_CONFKEY, escapeChar);
         if (arrayDelimiter != null) {
             conf.set(CsvToKeyValueMapper.ARRAY_DELIMITER_CONFKEY, arrayDelimiter);
         }
+        if(binaryEncoding!=null){
+            conf.set(QueryServices.UPLOAD_BINARY_DATA_TYPE_ENCODING, binaryEncoding);
+        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a6752119/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/CsvBulkLoadTool.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/CsvBulkLoadTool.java b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/CsvBulkLoadTool.java
index 8ed66b8..14b8c34 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/CsvBulkLoadTool.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/CsvBulkLoadTool.java
@@ -35,6 +35,7 @@ public class CsvBulkLoadTool extends AbstractBulkLoadTool {
     static final Option QUOTE_OPT = new Option("q", "quote", true, "Supply a custom phrase delimiter, defaults to double quote character");
     static final Option ESCAPE_OPT = new Option("e", "escape", true, "Supply a custom escape character, default is a backslash");
     static final Option ARRAY_DELIMITER_OPT = new Option("a", "array-delimiter", true, "Array element delimiter (optional)");
+    static final Option binaryEncodingOption = new Option("b", "binaryEncoding", true, "Specifies binary encoding");
 
     @Override
     protected Options getOptions() {
@@ -43,6 +44,7 @@ public class CsvBulkLoadTool extends AbstractBulkLoadTool {
         options.addOption(QUOTE_OPT);
         options.addOption(ESCAPE_OPT);
         options.addOption(ARRAY_DELIMITER_OPT);
+        options.addOption(binaryEncodingOption);
         return options;
     }
 
@@ -79,13 +81,19 @@ public class CsvBulkLoadTool extends AbstractBulkLoadTool {
             }
             escapeChar = escapeString.charAt(0);
         }
-
+        
+        String binaryEncoding = null;
+        if (cmdLine.hasOption(binaryEncodingOption.getOpt())) {
+            binaryEncoding = cmdLine.getOptionValue(binaryEncodingOption.getOpt());
+        }
+        
         CsvBulkImportUtil.initCsvImportJob(
                 conf,
                 delimiterChar,
                 quoteChar,
                 escapeChar,
-                cmdLine.getOptionValue(ARRAY_DELIMITER_OPT.getOpt()));
+                cmdLine.getOptionValue(ARRAY_DELIMITER_OPT.getOpt()),
+                binaryEncoding);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a6752119/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
index 1e002d2..14d9887 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
@@ -225,6 +225,9 @@ public interface QueryServices extends SQLCloseable {
     public static final String INDEX_POPULATION_SLEEP_TIME = "phoenix.index.population.wait.time";
     public static final String LOCAL_INDEX_CLIENT_UPGRADE_ATTRIB = "phoenix.client.localIndexUpgrade";
     public static final String LIMITED_QUERY_SERIAL_THRESHOLD = "phoenix.limited.query.serial.threshold";
+    
+    //currently BASE64 and ASCII is supported
+    public static final String UPLOAD_BINARY_DATA_TYPE_ENCODING = "phoenix.upload.binaryDataType.encoding";
 
     public static final String INDEX_ASYNC_BUILD_ENABLED = "phoenix.index.async.build.enabled";
     

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a6752119/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
index ef3ac39..0e8b9d5 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
@@ -79,6 +79,7 @@ import static org.apache.phoenix.query.QueryServices.STATS_USE_CURRENT_TIME_ATTR
 import static org.apache.phoenix.query.QueryServices.THREAD_POOL_SIZE_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.THREAD_TIMEOUT_MS_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.TRANSACTIONS_ENABLED;
+import static org.apache.phoenix.query.QueryServices.UPLOAD_BINARY_DATA_TYPE_ENCODING;
 import static org.apache.phoenix.query.QueryServices.USE_BYTE_BASED_REGEX_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.USE_INDEXES_ATTRIB;
 
@@ -98,6 +99,7 @@ import org.apache.phoenix.util.DateUtil;
 import org.apache.phoenix.util.ReadOnlyProps;
 
 
+
 /**
  * Options for {@link QueryServices}.
  *
@@ -271,6 +273,10 @@ public class QueryServicesOptions {
       }
     };
     public static final String DEFAULT_SCHEMA = null;
+    public static final String DEFAULT_UPLOAD_BINARY_DATA_TYPE_ENCODING = "BASE64"; // for backward compatibility, till
+                                                                                    // 4.10, psql and CSVBulkLoad
+                                                                                    // expects binary data to be base 64
+                                                                                    // encoded
     
     private final Configuration config;
 
@@ -338,7 +344,8 @@ public class QueryServicesOptions {
             .setIfUnset(IS_NAMESPACE_MAPPING_ENABLED, DEFAULT_IS_NAMESPACE_MAPPING_ENABLED)
             .setIfUnset(IS_SYSTEM_TABLE_MAPPED_TO_NAMESPACE, DEFAULT_IS_SYSTEM_TABLE_MAPPED_TO_NAMESPACE)
             .setIfUnset(LOCAL_INDEX_CLIENT_UPGRADE_ATTRIB, DEFAULT_LOCAL_INDEX_CLIENT_UPGRADE)
-            .setIfUnset(AUTO_UPGRADE_ENABLED, DEFAULT_AUTO_UPGRADE_ENABLED);
+            .setIfUnset(AUTO_UPGRADE_ENABLED, DEFAULT_AUTO_UPGRADE_ENABLED)
+            .setIfUnset(UPLOAD_BINARY_DATA_TYPE_ENCODING, DEFAULT_UPLOAD_BINARY_DATA_TYPE_ENCODING);
         // HBase sets this to 1, so we reset it to something more appropriate.
         // Hopefully HBase will change this, because we can't know if a user set
         // it to 1, so we'll change it.

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a6752119/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PBinary.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PBinary.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PBinary.java
index 43906f0..3a9dcc7 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PBinary.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PBinary.java
@@ -22,7 +22,6 @@ import java.text.Format;
 import java.util.Arrays;
 
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.hadoop.hbase.util.Base64;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.exception.DataExceedsCapacityException;
 import org.apache.phoenix.query.QueryConstants;
@@ -182,10 +181,7 @@ public class PBinary extends PBinaryBase {
 
     @Override
     public Object toObject(String value) {
-        if (value == null || value.length() == 0) {
-            return null;
-        }
-        return Base64.decode(value);
+        return PVarbinary.INSTANCE.toObject(value);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a6752119/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PVarbinary.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PVarbinary.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PVarbinary.java
index d96650d..b3ce57a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PVarbinary.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PVarbinary.java
@@ -131,7 +131,10 @@ public class PVarbinary extends PBinaryBase {
         if (value == null || value.length() == 0) {
             return null;
         }
-        return Base64.decode(value);
+        Object object = Base64.decode(value);
+        if (object == null) { throw newIllegalDataException(
+                "Input: [" + value + "]  is not base64 encoded"); }
+        return object;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a6752119/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java b/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java
index dbac76f..3c16d00 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java
@@ -213,6 +213,9 @@ public class PhoenixRuntime {
             if (execCmd.isLocalIndexUpgrade()) {
                 props.setProperty(QueryServices.LOCAL_INDEX_CLIENT_UPGRADE_ATTRIB, "false");
             }
+            if (execCmd.binaryEncoding != null) {
+                props.setProperty(QueryServices.UPLOAD_BINARY_DATA_TYPE_ENCODING, execCmd.binaryEncoding);
+            }
             conn = DriverManager.getConnection(jdbcUrl, props).unwrap(PhoenixConnection.class);
             conn.setRunningUpgrade(true);
             if (execCmd.isMapNamespace()) {
@@ -532,6 +535,7 @@ public class PhoenixRuntime {
         private boolean mapNamespace;
         private String srcTable;
         private boolean localIndexUpgrade;
+        private String binaryEncoding;
 
         /**
          * Factory method to build up an {@code ExecutionCommand} based on supplied parameters.
@@ -539,6 +543,8 @@ public class PhoenixRuntime {
         public static ExecutionCommand parseArgs(String[] args) {
             Option tableOption = new Option("t", "table", true,
                     "Overrides the table into which the CSV data is loaded and is case sensitive");
+            Option binaryEncodingOption = new Option("b", "binaryEncoding", true,
+                    "Specifies binary encoding");
             Option headerOption = new Option("h", "header", true, "Overrides the column names to" +
                     " which the CSV data maps and is case sensitive. A special value of " +
                     "in-line indicating that the first line of the CSV file determines the " +
@@ -588,6 +594,7 @@ public class PhoenixRuntime {
             options.addOption(bypassUpgradeOption);
             options.addOption(mapNamespaceOption);
             options.addOption(localIndexUpgradeOption);
+            options.addOption(binaryEncodingOption);
 
             CommandLineParser parser = new PosixParser();
             CommandLine cmdLine = null;
@@ -606,6 +613,10 @@ public class PhoenixRuntime {
             if (cmdLine.hasOption(tableOption.getOpt())) {
                 execCmd.tableName = cmdLine.getOptionValue(tableOption.getOpt());
             }
+            
+            if (cmdLine.hasOption(binaryEncodingOption.getOpt())) {
+                execCmd.binaryEncoding = cmdLine.getOptionValue(binaryEncodingOption.getOpt());
+            }
 
             if (cmdLine.hasOption(headerOption.getOpt())) {
                 String columnString = cmdLine.getOptionValue(headerOption.getOpt());

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a6752119/phoenix-core/src/main/java/org/apache/phoenix/util/csv/CsvUpsertExecutor.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/csv/CsvUpsertExecutor.java b/phoenix-core/src/main/java/org/apache/phoenix/util/csv/CsvUpsertExecutor.java
index 0d3e17d..cd40b44 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/csv/CsvUpsertExecutor.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/csv/CsvUpsertExecutor.java
@@ -27,12 +27,18 @@ import java.util.Properties;
 import javax.annotation.Nullable;
 
 import org.apache.commons.csv.CSVRecord;
+import org.apache.hadoop.hbase.util.Base64;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.expression.function.EncodeFormat;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
+import org.apache.phoenix.schema.IllegalDataException;
+import org.apache.phoenix.schema.types.PBinary;
 import org.apache.phoenix.schema.types.PBoolean;
 import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.types.PDataType.PDataCodec;
 import org.apache.phoenix.schema.types.PTimestamp;
+import org.apache.phoenix.schema.types.PVarbinary;
 import org.apache.phoenix.util.ColumnInfo;
 import org.apache.phoenix.util.DateUtil;
 import org.apache.phoenix.util.UpsertExecutor;
@@ -116,6 +122,7 @@ public class CsvUpsertExecutor extends UpsertExecutor<CSVRecord, String> {
         private final PDataType dataType;
         private final PDataCodec codec;
         private final DateUtil.DateTimeParser dateTimeParser;
+        private final String binaryEncoding;
 
         SimpleDatatypeConversionFunction(PDataType dataType, Connection conn) {
             Properties props;
@@ -148,6 +155,8 @@ public class CsvUpsertExecutor extends UpsertExecutor<CSVRecord, String> {
                 this.dateTimeParser = null;
             }
             this.codec = codec;
+            this.binaryEncoding = props.getProperty(QueryServices.UPLOAD_BINARY_DATA_TYPE_ENCODING,
+                            QueryServicesOptions.DEFAULT_UPLOAD_BINARY_DATA_TYPE_ENCODING);
         }
 
         @Nullable
@@ -175,6 +184,22 @@ public class CsvUpsertExecutor extends UpsertExecutor<CSVRecord, String> {
                         throw new RuntimeException("Invalid boolean value: '" + input
                                 + "', must be one of ['true','t','1','false','f','0']");
                 }
+            }else if (dataType == PVarbinary.INSTANCE || dataType == PBinary.INSTANCE){
+                EncodeFormat format = EncodeFormat.valueOf(binaryEncoding.toUpperCase());
+                Object object = null;
+                switch (format) {
+                    case BASE64:
+                        object = Base64.decode(input);
+                        if (object == null) { throw new IllegalDataException(
+                                "Input: [" + input + "]  is not base64 encoded"); }
+                        break;
+                    case ASCII:
+                        object = Bytes.toBytes(input);
+                        break;
+                    default:
+                        throw new IllegalDataException("Unsupported encoding \"" + binaryEncoding + "\"");
+                }
+                return object;
             }
             return dataType.toObject(input);
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a6752119/phoenix-core/src/main/java/org/apache/phoenix/util/json/JsonUpsertExecutor.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/json/JsonUpsertExecutor.java b/phoenix-core/src/main/java/org/apache/phoenix/util/json/JsonUpsertExecutor.java
index bbe0e30..ffa797d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/json/JsonUpsertExecutor.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/json/JsonUpsertExecutor.java
@@ -25,12 +25,20 @@ import java.sql.Types;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
+
 import javax.annotation.Nullable;
 
+import org.apache.hadoop.hbase.util.Base64;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.expression.function.EncodeFormat;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
+import org.apache.phoenix.schema.IllegalDataException;
+import org.apache.phoenix.schema.types.PBinary;
+import org.apache.phoenix.schema.types.PBoolean;
 import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.types.PTimestamp;
+import org.apache.phoenix.schema.types.PVarbinary;
 import org.apache.phoenix.util.ColumnInfo;
 import org.apache.phoenix.util.DateUtil;
 import org.apache.phoenix.util.UpsertExecutor;
@@ -137,6 +145,7 @@ public class JsonUpsertExecutor extends UpsertExecutor<Map<?, ?>, Object> {
 
         private final PDataType dataType;
         private final DateUtil.DateTimeParser dateTimeParser;
+        private final String binaryEncoding;
 
         SimpleDatatypeConversionFunction(PDataType dataType, Connection conn) {
             Properties props;
@@ -166,6 +175,8 @@ public class JsonUpsertExecutor extends UpsertExecutor<Map<?, ?>, Object> {
             } else {
                 this.dateTimeParser = null;
             }
+            this.binaryEncoding = props.getProperty(QueryServices.UPLOAD_BINARY_DATA_TYPE_ENCODING,
+                    QueryServicesOptions.DEFAULT_UPLOAD_BINARY_DATA_TYPE_ENCODING);
         }
 
         @Nullable
@@ -180,7 +191,40 @@ public class JsonUpsertExecutor extends UpsertExecutor<Map<?, ?>, Object> {
                 byte[] byteValue = new byte[dataType.getByteSize()];
                 dataType.getCodec().encodeLong(epochTime, byteValue, 0);
                 return dataType.toObject(byteValue);
+            }else if (dataType == PBoolean.INSTANCE) {
+                switch (input.toString()) {
+                case "true":
+                case "t":
+                case "T":
+                case "1":
+                    return Boolean.TRUE;
+                case "false":
+                case "f":
+                case "F":
+                case "0":
+                    return Boolean.FALSE;
+                default:
+                    throw new RuntimeException("Invalid boolean value: '" + input
+                            + "', must be one of ['true','t','1','false','f','0']");
+            }
+        }else if (dataType == PVarbinary.INSTANCE || dataType == PBinary.INSTANCE){
+            EncodeFormat format = EncodeFormat.valueOf(binaryEncoding.toUpperCase());
+            Object object = null;
+            switch (format) {
+                case BASE64:
+                    object = Base64.decode(input.toString());
+                    if (object == null) { throw new IllegalDataException(
+                            "Input: [" + input + "]  is not base64 encoded"); }
+                    break;
+                case ASCII:
+                    object = Bytes.toBytes(input.toString());
+                    break;
+                default:
+                    throw new IllegalDataException("Unsupported encoding \"" + binaryEncoding + "\"");
             }
+            return object;
+        }
+            
             return dataType.toObject(input, dataType);
         }
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a6752119/phoenix-core/src/test/java/org/apache/phoenix/mapreduce/CsvBulkImportUtilTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/mapreduce/CsvBulkImportUtilTest.java b/phoenix-core/src/test/java/org/apache/phoenix/mapreduce/CsvBulkImportUtilTest.java
index 3c6271a..33c72a8 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/mapreduce/CsvBulkImportUtilTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/mapreduce/CsvBulkImportUtilTest.java
@@ -17,6 +17,9 @@
  */
 package org.apache.phoenix.mapreduce;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
 import java.io.File;
 import java.io.FileInputStream;
 import java.io.FileOutputStream;
@@ -28,9 +31,6 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil;
 import org.junit.Test;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-
 public class CsvBulkImportUtilTest {
 
     @Test
@@ -41,7 +41,7 @@ public class CsvBulkImportUtilTest {
         char quote = '\002';
         char escape = '!';
 
-        CsvBulkImportUtil.initCsvImportJob(conf, delimiter, quote, escape, null);
+        CsvBulkImportUtil.initCsvImportJob(conf, delimiter, quote, escape, null, null);
 
         // Serialize and deserialize the config to ensure that there aren't any issues
         // with non-printable characters as delimiters

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a6752119/phoenix-core/src/test/java/org/apache/phoenix/util/AbstractUpsertExecutorTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/util/AbstractUpsertExecutorTest.java b/phoenix-core/src/test/java/org/apache/phoenix/util/AbstractUpsertExecutorTest.java
index 61b03fb..2b2544d 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/util/AbstractUpsertExecutorTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/util/AbstractUpsertExecutorTest.java
@@ -17,6 +17,12 @@
  */
 package org.apache.phoenix.util;
 
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyNoMoreInteractions;
+
 import java.io.IOException;
 import java.sql.Connection;
 import java.sql.DriverManager;
@@ -25,9 +31,14 @@ import java.sql.SQLException;
 import java.sql.Types;
 import java.util.Arrays;
 import java.util.List;
+import java.util.Properties;
 
+import org.apache.hadoop.hbase.util.Base64;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.query.BaseConnectionlessQueryTest;
+import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.schema.types.PArrayDataType;
+import org.apache.phoenix.schema.types.PBinary;
 import org.apache.phoenix.schema.types.PInteger;
 import org.apache.phoenix.schema.types.PIntegerArray;
 import org.junit.After;
@@ -36,12 +47,6 @@ import org.junit.Test;
 
 import com.google.common.collect.ImmutableList;
 
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.eq;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.verifyNoMoreInteractions;
-
 public abstract class AbstractUpsertExecutorTest<R, F> extends BaseConnectionlessQueryTest {
 
     protected Connection conn;
@@ -51,6 +56,7 @@ public abstract class AbstractUpsertExecutorTest<R, F> extends BaseConnectionles
 
     protected abstract UpsertExecutor<R, F> getUpsertExecutor();
     protected abstract R createRecord(Object... columnValues) throws IOException;
+    protected abstract UpsertExecutor<R, F> getUpsertExecutor(Connection conn);
 
     @Before
     public void setUp() throws SQLException {
@@ -59,7 +65,8 @@ public abstract class AbstractUpsertExecutorTest<R, F> extends BaseConnectionles
                 new ColumnInfo("NAME", Types.VARCHAR),
                 new ColumnInfo("AGE", Types.INTEGER),
                 new ColumnInfo("VALUES", PIntegerArray.INSTANCE.getSqlType()),
-                new ColumnInfo("BEARD", Types.BOOLEAN));
+                new ColumnInfo("BEARD", Types.BOOLEAN),
+                new ColumnInfo("PIC", Types.BINARY));
 
         preparedStatement = mock(PreparedStatement.class);
         upsertListener = mock(UpsertExecutor.UpsertListener.class);
@@ -73,8 +80,10 @@ public abstract class AbstractUpsertExecutorTest<R, F> extends BaseConnectionles
 
     @Test
     public void testExecute() throws Exception {
+        byte[] binaryData=(byte[])PBinary.INSTANCE.getSampleValue();
+        String encodedBinaryData = Base64.encodeBytes(binaryData);
         getUpsertExecutor().execute(createRecord(123L, "NameValue", 42,
-                Arrays.asList(1, 2, 3), true));
+                Arrays.asList(1, 2, 3), true, encodedBinaryData));
 
         verify(upsertListener).upsertDone(1L);
         verifyNoMoreInteractions(upsertListener);
@@ -84,6 +93,7 @@ public abstract class AbstractUpsertExecutorTest<R, F> extends BaseConnectionles
         verify(preparedStatement).setObject(3, Integer.valueOf(42));
         verify(preparedStatement).setObject(4, PArrayDataType.instantiatePhoenixArray(PInteger.INSTANCE, new Object[]{1,2,3}));
         verify(preparedStatement).setObject(5, Boolean.TRUE);
+        verify(preparedStatement).setObject(6, binaryData);
         verify(preparedStatement).execute();
         verifyNoMoreInteractions(preparedStatement);
     }
@@ -99,8 +109,10 @@ public abstract class AbstractUpsertExecutorTest<R, F> extends BaseConnectionles
 
     @Test
     public void testExecute_TooManyFields() throws Exception {
+        byte[] binaryData=(byte[])PBinary.INSTANCE.getSampleValue();
+        String encodedBinaryData = Base64.encodeBytes(binaryData);
         R recordWithTooManyFields = createRecord(123L, "NameValue", 42, Arrays.asList(1, 2, 3),
-                true, "Garbage");
+                true, encodedBinaryData, "garbage");
         getUpsertExecutor().execute(recordWithTooManyFields);
 
         verify(upsertListener).upsertDone(1L);
@@ -111,14 +123,17 @@ public abstract class AbstractUpsertExecutorTest<R, F> extends BaseConnectionles
         verify(preparedStatement).setObject(3, Integer.valueOf(42));
         verify(preparedStatement).setObject(4, PArrayDataType.instantiatePhoenixArray(PInteger.INSTANCE, new Object[]{1,2,3}));
         verify(preparedStatement).setObject(5, Boolean.TRUE);
+        verify(preparedStatement).setObject(6, binaryData);
         verify(preparedStatement).execute();
         verifyNoMoreInteractions(preparedStatement);
     }
 
     @Test
     public void testExecute_NullField() throws Exception {
+        byte[] binaryData=(byte[])PBinary.INSTANCE.getSampleValue();
+        String encodedBinaryData = Base64.encodeBytes(binaryData);
         getUpsertExecutor().execute(createRecord(123L, "NameValue", null,
-                Arrays.asList(1, 2, 3), false));
+                Arrays.asList(1, 2, 3), false, encodedBinaryData));
 
         verify(upsertListener).upsertDone(1L);
         verifyNoMoreInteractions(upsertListener);
@@ -128,17 +143,62 @@ public abstract class AbstractUpsertExecutorTest<R, F> extends BaseConnectionles
         verify(preparedStatement).setNull(3, columnInfoList.get(2).getSqlType());
         verify(preparedStatement).setObject(4, PArrayDataType.instantiatePhoenixArray(PInteger.INSTANCE, new Object[]{1,2,3}));
         verify(preparedStatement).setObject(5, Boolean.FALSE);
+        verify(preparedStatement).setObject(6, binaryData);
         verify(preparedStatement).execute();
         verifyNoMoreInteractions(preparedStatement);
     }
 
     @Test
     public void testExecute_InvalidType() throws Exception {
+        byte[] binaryData=(byte[])PBinary.INSTANCE.getSampleValue();
+        String encodedBinaryData = Base64.encodeBytes(binaryData);
         R recordWithInvalidType = createRecord(123L, "NameValue", "ThisIsNotANumber",
-                Arrays.asList(1, 2, 3), true);
+                Arrays.asList(1, 2, 3), true, encodedBinaryData);
         getUpsertExecutor().execute(recordWithInvalidType);
 
         verify(upsertListener).errorOnRecord(eq(recordWithInvalidType), any(Throwable.class));
         verifyNoMoreInteractions(upsertListener);
     }
+    
+    @Test
+    public void testExecute_InvalidBoolean() throws Exception {
+        byte[] binaryData=(byte[])PBinary.INSTANCE.getSampleValue();
+        String encodedBinaryData = Base64.encodeBytes(binaryData);
+        R csvRecordWithInvalidType = createRecord("123,NameValue,42,1:2:3,NotABoolean,"+encodedBinaryData);
+        getUpsertExecutor().execute(csvRecordWithInvalidType);
+
+        verify(upsertListener).errorOnRecord(eq(csvRecordWithInvalidType), any(Throwable.class));
+    }
+    
+    @Test
+    public void testExecute_InvalidBinary() throws Exception {
+        String notBase64Encoded="#@$df";
+        R csvRecordWithInvalidType = createRecord("123,NameValue,42,1:2:3,true,"+notBase64Encoded);
+        getUpsertExecutor().execute(csvRecordWithInvalidType);
+
+        verify(upsertListener).errorOnRecord(eq(csvRecordWithInvalidType), any(Throwable.class));
+    }
+    
+    @Test
+    public void testExecute_AsciiEncoded() throws Exception {
+        String asciiValue="#@$df";
+        Properties info=new Properties();
+        info.setProperty(QueryServices.UPLOAD_BINARY_DATA_TYPE_ENCODING,"ASCII");
+        getUpsertExecutor(DriverManager.getConnection(getUrl(),info)).execute(createRecord(123L, "NameValue", 42,
+                Arrays.asList(1, 2, 3), true, asciiValue));
+
+        verify(upsertListener).upsertDone(1L);
+        verifyNoMoreInteractions(upsertListener);
+        
+        verify(preparedStatement).setObject(1, Long.valueOf(123L));
+        verify(preparedStatement).setObject(2, "NameValue");
+        verify(preparedStatement).setObject(3, Integer.valueOf(42));
+        verify(preparedStatement).setObject(4, PArrayDataType.instantiatePhoenixArray(PInteger.INSTANCE, new Object[]{1,2,3}));
+        verify(preparedStatement).setObject(5, Boolean.TRUE);
+        verify(preparedStatement).setObject(6, Bytes.toBytes(asciiValue));
+        verify(preparedStatement).execute();
+        verifyNoMoreInteractions(preparedStatement);
+    }
+
+    
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a6752119/phoenix-core/src/test/java/org/apache/phoenix/util/csv/CsvUpsertExecutorTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/util/csv/CsvUpsertExecutorTest.java b/phoenix-core/src/test/java/org/apache/phoenix/util/csv/CsvUpsertExecutorTest.java
index c887ff7..a5ec4fa 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/util/csv/CsvUpsertExecutorTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/util/csv/CsvUpsertExecutorTest.java
@@ -18,23 +18,19 @@
 package org.apache.phoenix.util.csv;
 
 import java.io.IOException;
+import java.sql.Connection;
 import java.sql.SQLException;
 import java.util.List;
 
-import com.google.common.base.Joiner;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Iterables;
 import org.apache.commons.csv.CSVFormat;
 import org.apache.commons.csv.CSVParser;
 import org.apache.commons.csv.CSVRecord;
 import org.apache.phoenix.util.AbstractUpsertExecutorTest;
 import org.apache.phoenix.util.UpsertExecutor;
 import org.junit.Before;
-import org.junit.Test;
 
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.eq;
-import static org.mockito.Mockito.verify;
+import com.google.common.base.Joiner;
+import com.google.common.collect.Iterables;
 
 public class CsvUpsertExecutorTest extends AbstractUpsertExecutorTest<CSVRecord, String> {
 
@@ -46,7 +42,13 @@ public class CsvUpsertExecutorTest extends AbstractUpsertExecutorTest<CSVRecord,
     public UpsertExecutor<CSVRecord, String> getUpsertExecutor() {
         return upsertExecutor;
     }
-
+    
+    @Override
+    public UpsertExecutor<CSVRecord, String> getUpsertExecutor(Connection conn) {
+        return new CsvUpsertExecutor(conn, columnInfoList, preparedStatement,
+                upsertListener, ARRAY_SEP);
+    }
+    
     @Override
     public CSVRecord createRecord(Object... columnValues) throws IOException {
         for (int i = 0; i < columnValues.length; i++) {
@@ -69,11 +71,5 @@ public class CsvUpsertExecutorTest extends AbstractUpsertExecutorTest<CSVRecord,
                 upsertListener, ARRAY_SEP);
     }
 
-    @Test
-    public void testExecute_InvalidBoolean() throws Exception {
-        CSVRecord csvRecordWithInvalidType = createRecord("123,NameValue,42,1:2:3,NotABoolean");
-        upsertExecutor.execute(ImmutableList.of(csvRecordWithInvalidType));
-
-        verify(upsertListener).errorOnRecord(eq(csvRecordWithInvalidType), any(Throwable.class));
-    }
+    
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a6752119/phoenix-core/src/test/java/org/apache/phoenix/util/json/JsonUpsertExecutorTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/util/json/JsonUpsertExecutorTest.java b/phoenix-core/src/test/java/org/apache/phoenix/util/json/JsonUpsertExecutorTest.java
index c042dd4..6ac9cf9 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/util/json/JsonUpsertExecutorTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/util/json/JsonUpsertExecutorTest.java
@@ -18,6 +18,7 @@
 package org.apache.phoenix.util.json;
 
 import java.io.IOException;
+import java.sql.Connection;
 import java.sql.SQLException;
 import java.util.HashMap;
 import java.util.Map;
@@ -50,4 +51,9 @@ public class JsonUpsertExecutorTest extends AbstractUpsertExecutorTest<Map<?, ?>
         super.setUp();
         upsertExecutor = new JsonUpsertExecutor(conn, columnInfoList, preparedStatement, upsertListener);
     }
+
+    @Override
+    protected UpsertExecutor<Map<?, ?>, Object> getUpsertExecutor(Connection conn) {
+        return new JsonUpsertExecutor(conn, columnInfoList, preparedStatement, upsertListener);
+    }
 }


[09/26] phoenix git commit: PHOENIX-3516 Performance Issues with queries that have compound filters and specify phoenix.query.force.rowkeyorder=true (addendum)

Posted by ma...@apache.org.
PHOENIX-3516 Performance Issues with queries that have compound filters and specify phoenix.query.force.rowkeyorder=true (addendum)


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

Branch: refs/heads/calcite
Commit: 07f92732f9c6d2d9464012cebeb4cefc10da95d5
Parents: b326995
Author: Thomas D'Silva <td...@salesforce.com>
Authored: Thu Dec 22 17:29:35 2016 -0800
Committer: Thomas D'Silva <td...@salesforce.com>
Committed: Tue Dec 27 11:44:54 2016 -0800

----------------------------------------------------------------------
 .../main/java/org/apache/phoenix/compile/WhereOptimizer.java  | 7 ++++++-
 1 file changed, 6 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/07f92732/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereOptimizer.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereOptimizer.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereOptimizer.java
index 0681925..5e15f61 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereOptimizer.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereOptimizer.java
@@ -439,7 +439,12 @@ public class WhereOptimizer {
                 if (l.size() == 1) {
                     return l.get(0);
                 }
-                return new AndExpression(l);
+                try {
+                    return AndExpression.create(l);
+                } catch (SQLException e) {
+                    //shouldn't happen
+                    throw new RuntimeException(e);
+                }
             }
             return node;
         }


[24/26] phoenix git commit: PHOENIX-541 Make mutable batch size bytes-based instead of row-based (Geoffrey Jacoby)

Posted by ma...@apache.org.
PHOENIX-541 Make mutable batch size bytes-based instead of row-based (Geoffrey Jacoby)


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

Branch: refs/heads/calcite
Commit: a44d317e32d383c4964824448bf12dfb8ed8bfec
Parents: b9323e1
Author: Samarth <sa...@salesforce.com>
Authored: Thu Jan 26 15:31:43 2017 -0800
Committer: Samarth <sa...@salesforce.com>
Committed: Thu Jan 26 15:31:43 2017 -0800

----------------------------------------------------------------------
 .../org/apache/phoenix/end2end/QueryMoreIT.java | 35 ++++++++++++++
 .../end2end/UpsertSelectAutoCommitIT.java       |  2 +-
 .../apache/phoenix/end2end/UpsertSelectIT.java  |  2 +-
 .../phoenix/end2end/index/ImmutableIndexIT.java |  1 -
 .../org/apache/phoenix/tx/TransactionIT.java    |  2 +-
 .../org/apache/phoenix/tx/TxCheckpointIT.java   |  2 +-
 .../UngroupedAggregateRegionObserver.java       | 31 +++++++-----
 .../apache/phoenix/execute/MutationState.java   | 51 ++++++++++++++++++--
 .../apache/phoenix/jdbc/PhoenixConnection.java  | 10 +++-
 .../index/PhoenixIndexImportDirectMapper.java   | 19 +++++---
 .../org/apache/phoenix/query/QueryServices.java |  3 ++
 .../phoenix/query/QueryServicesOptions.java     |  7 ++-
 .../java/org/apache/phoenix/util/JDBCUtil.java  |  6 +++
 .../org/apache/phoenix/util/PhoenixRuntime.java | 12 ++++-
 .../apache/phoenix/jdbc/PhoenixDriverTest.java  |  9 ++++
 .../org/apache/phoenix/util/JDBCUtilTest.java   | 21 +++++++-
 16 files changed, 181 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/a44d317e/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryMoreIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryMoreIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryMoreIT.java
index 2b27f00..a2dab16 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryMoreIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryMoreIT.java
@@ -37,6 +37,8 @@ import java.util.Properties;
 
 import org.apache.hadoop.hbase.util.Base64;
 import org.apache.hadoop.hbase.util.Pair;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.TestUtil;
 import org.junit.Test;
@@ -471,4 +473,37 @@ public class QueryMoreIT extends ParallelStatsDisabledIT {
             assertFalse(rs.next());
         }
     }
+
+    @Test
+    public void testMutationBatch() throws Exception {
+        Properties connectionProperties = new Properties();
+        connectionProperties.setProperty(QueryServices.MUTATE_BATCH_SIZE_BYTES_ATTRIB, "1024");
+        PhoenixConnection connection = (PhoenixConnection) DriverManager.getConnection(getUrl(), connectionProperties);
+        String fullTableName = generateUniqueName();
+        try (Statement stmt = connection.createStatement()) {
+            stmt.execute("CREATE TABLE " + fullTableName + "(\n" +
+                "    ORGANIZATION_ID CHAR(15) NOT NULL,\n" +
+                "    SCORE DOUBLE NOT NULL,\n" +
+                "    ENTITY_ID CHAR(15) NOT NULL\n" +
+                "    CONSTRAINT PAGE_SNAPSHOT_PK PRIMARY KEY (\n" +
+                "        ORGANIZATION_ID,\n" +
+                "        SCORE DESC,\n" +
+                "        ENTITY_ID DESC\n" +
+                "    )\n" +
+                ") MULTI_TENANT=TRUE");
+        }
+        PreparedStatement stmt = connection.prepareStatement("upsert into " + fullTableName +
+            " (organization_id, entity_id, score) values (?,?,?)");
+        try {
+            for (int i = 0; i < 4; i++) {
+                stmt.setString(1, "AAAA" + i);
+                stmt.setString(2, "BBBB" + i);
+                stmt.setInt(3, 1);
+                stmt.execute();
+            }
+            connection.commit();
+        } catch (IllegalArgumentException expected) {}
+
+        assertEquals(2L, connection.getMutationState().getBatchCount());
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a44d317e/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectAutoCommitIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectAutoCommitIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectAutoCommitIT.java
index 37482de..6b781a0 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectAutoCommitIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectAutoCommitIT.java
@@ -152,7 +152,7 @@ public class UpsertSelectAutoCommitIT extends ParallelStatsDisabledIT {
     @Test
     public void testUpsertSelectDoesntSeeUpsertedData() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        props.setProperty(QueryServices.MUTATE_BATCH_SIZE_ATTRIB, Integer.toString(3));
+        props.setProperty(QueryServices.MUTATE_BATCH_SIZE_BYTES_ATTRIB, Integer.toString(512));
         props.setProperty(QueryServices.SCAN_CACHE_SIZE_ATTRIB, Integer.toString(3));
         props.setProperty(QueryServices.SCAN_RESULT_CHUNK_SIZE, Integer.toString(3));
         Connection conn = DriverManager.getConnection(getUrl(), props);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a44d317e/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectIT.java
index 763f11b..f5905ee 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectIT.java
@@ -1435,7 +1435,7 @@ public class UpsertSelectIT extends BaseClientManagedTimeIT {
         long ts = nextTimestamp();
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts));
-        props.setProperty(QueryServices.MUTATE_BATCH_SIZE_ATTRIB, Integer.toString(3));
+        props.setProperty(QueryServices.MUTATE_BATCH_SIZE_BYTES_ATTRIB, Integer.toString(512));
         props.setProperty(QueryServices.SCAN_CACHE_SIZE_ATTRIB, Integer.toString(3));
         props.setProperty(QueryServices.SCAN_RESULT_CHUNK_SIZE, Integer.toString(3));
         Connection conn = DriverManager.getConnection(getUrl(), props);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a44d317e/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexIT.java
index 3ee9721..bc301fa 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexIT.java
@@ -154,7 +154,6 @@ public class ImmutableIndexIT extends BaseUniqueNamesOwnClusterIT {
             return;
         }
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        props.setProperty(QueryServices.MUTATE_BATCH_SIZE_ATTRIB, Integer.toString(100));
         String tableName = "TBL_" + generateUniqueName();
         String indexName = "IND_" + generateUniqueName();
         String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a44d317e/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java
index 83128f1..bde5cc8 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java
@@ -603,7 +603,7 @@ public class TransactionIT extends ParallelStatsDisabledIT {
     @Test
     public void testParallelUpsertSelect() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        props.setProperty(QueryServices.MUTATE_BATCH_SIZE_ATTRIB, Integer.toString(3));
+        props.setProperty(QueryServices.MUTATE_BATCH_SIZE_BYTES_ATTRIB, Integer.toString(512));
         props.setProperty(QueryServices.SCAN_CACHE_SIZE_ATTRIB, Integer.toString(3));
         props.setProperty(QueryServices.SCAN_RESULT_CHUNK_SIZE, Integer.toString(3));
         Connection conn = DriverManager.getConnection(getUrl(), props);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a44d317e/phoenix-core/src/it/java/org/apache/phoenix/tx/TxCheckpointIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/tx/TxCheckpointIT.java b/phoenix-core/src/it/java/org/apache/phoenix/tx/TxCheckpointIT.java
index 14bcd70..246ecd4 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/tx/TxCheckpointIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/tx/TxCheckpointIT.java
@@ -80,7 +80,7 @@ public class TxCheckpointIT extends ParallelStatsDisabledIT {
         String seqName = "SEQ_" + generateUniqueName();
         String fullTableName = SchemaUtil.getTableName(tableName, tableName);
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        props.setProperty(QueryServices.MUTATE_BATCH_SIZE_ATTRIB, Integer.toString(3));
+        props.setProperty(QueryServices.MUTATE_BATCH_SIZE_BYTES_ATTRIB, Integer.toString(512));
         props.setProperty(QueryServices.SCAN_CACHE_SIZE_ATTRIB, Integer.toString(3));
         props.setProperty(QueryServices.SCAN_RESULT_CHUNK_SIZE, Integer.toString(3));
         Connection conn = getConnection(props);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a44d317e/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
index 9ee0054..a888bb2 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
@@ -22,6 +22,7 @@ import static org.apache.phoenix.query.QueryConstants.SINGLE_COLUMN;
 import static org.apache.phoenix.query.QueryConstants.SINGLE_COLUMN_FAMILY;
 import static org.apache.phoenix.query.QueryConstants.UNGROUPED_AGG_ROW_KEY;
 import static org.apache.phoenix.query.QueryServices.MUTATE_BATCH_SIZE_ATTRIB;
+import static org.apache.phoenix.query.QueryServices.MUTATE_BATCH_SIZE_BYTES_ATTRIB;
 import static org.apache.phoenix.schema.stats.StatisticsCollectionRunTracker.COMPACTION_UPDATE_STATS_ROW_COUNT;
 import static org.apache.phoenix.schema.stats.StatisticsCollectionRunTracker.CONCURRENT_UPDATE_STATS_ROW_COUNT;
 
@@ -70,6 +71,7 @@ import org.apache.hadoop.io.WritableUtils;
 import org.apache.phoenix.cache.ServerCacheClient;
 import org.apache.phoenix.coprocessor.generated.PTableProtos;
 import org.apache.phoenix.exception.DataExceedsCapacityException;
+import org.apache.phoenix.execute.MutationState;
 import org.apache.phoenix.execute.TupleProjector;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.expression.ExpressionType;
@@ -343,6 +345,7 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
         }
         
         int batchSize = 0;
+        long batchSizeBytes = 0L;
         List<Mutation> mutations = Collections.emptyList();
         boolean needToWrite = false;
         Configuration conf = c.getEnvironment().getConfiguration();
@@ -369,6 +372,8 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
             // TODO: size better
             mutations = Lists.newArrayListWithExpectedSize(1024);
             batchSize = env.getConfiguration().getInt(MUTATE_BATCH_SIZE_ATTRIB, QueryServicesOptions.DEFAULT_MUTATE_BATCH_SIZE);
+            batchSizeBytes = env.getConfiguration().getLong(MUTATE_BATCH_SIZE_BYTES_ATTRIB,
+                QueryServicesOptions.DEFAULT_MUTATE_BATCH_SIZE_BYTES);
         }
         Aggregators aggregators = ServerAggregators.deserialize(
                 scan.getAttribute(BaseScannerRegionObserver.AGGREGATORS), env.getConfiguration());
@@ -596,19 +601,23 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
                                     mutations.add(put);
                                 }
                             }
-                            // Commit in batches based on UPSERT_BATCH_SIZE_ATTRIB in config
-                            if (!mutations.isEmpty() && batchSize > 0 &&
-                                    mutations.size() % batchSize == 0) {
-                                commitBatch(region, mutations, indexUUID, blockingMemStoreSize, indexMaintainersPtr,
-                                        txState);
-                                mutations.clear();
+                            // Commit in batches based on UPSERT_BATCH_SIZE_BYTES_ATTRIB in config
+                            List<List<Mutation>> batchMutationList =
+                                MutationState.getMutationBatchList(batchSize, batchSizeBytes, mutations);
+                            for (List<Mutation> batchMutations : batchMutationList) {
+                                commitBatch(region, batchMutations, indexUUID, blockingMemStoreSize, indexMaintainersPtr,
+                                    txState);
+                                batchMutations.clear();
                             }
-                            // Commit in batches based on UPSERT_BATCH_SIZE_ATTRIB in config
-                            if (!indexMutations.isEmpty() && batchSize > 0 &&
-                                    indexMutations.size() % batchSize == 0) {
-                                commitBatch(region, indexMutations, null, blockingMemStoreSize, null, txState);
-                                indexMutations.clear();
+                            mutations.clear();
+                            // Commit in batches based on UPSERT_BATCH_SIZE_BYTES_ATTRIB in config
+                            List<List<Mutation>> batchIndexMutationList =
+                                MutationState.getMutationBatchList(batchSize, batchSizeBytes, indexMutations);
+                            for (List<Mutation> batchIndexMutations : batchIndexMutationList) {
+                                commitBatch(region, batchIndexMutations, null, blockingMemStoreSize, null, txState);
+                                batchIndexMutations.clear();
                             }
+                            indexMutations.clear();
                         }
                         aggregators.aggregate(rowAggregators, result);
                         hasAny = true;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a44d317e/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
index 31ab7c9..4775d59 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
@@ -126,6 +126,8 @@ public class MutationState implements SQLCloseable {
     
     private final PhoenixConnection connection;
     private final long maxSize;
+    private final long maxSizeBytes;
+    private long batchCount = 0L;
     private final Map<TableRef, Map<ImmutableBytesPtr,RowMutationState>> mutations;
     private final List<TransactionAware> txAwares;
     private final TransactionContext txContext;
@@ -140,7 +142,7 @@ public class MutationState implements SQLCloseable {
     
     private final MutationMetricQueue mutationMetricQueue;
     private ReadMetricQueue readMetricQueue;
-    
+
     public MutationState(long maxSize, PhoenixConnection connection) {
         this(maxSize,connection, null, null);
     }
@@ -171,6 +173,7 @@ public class MutationState implements SQLCloseable {
             Transaction tx, TransactionContext txContext) {
         this.maxSize = maxSize;
         this.connection = connection;
+        this.maxSizeBytes = connection.getMutateBatchSizeBytes();
         this.mutations = mutations;
         boolean isMetricsEnabled = connection.isRequestLevelMetricsEnabled();
         this.mutationMetricQueue = isMetricsEnabled ? new MutationMetricQueue()
@@ -743,12 +746,11 @@ public class MutationState implements SQLCloseable {
     public static long getMutationTimestamp(final Long tableTimestamp, Long scn) {
         return (tableTimestamp!=null && tableTimestamp!=QueryConstants.UNSET_TIMESTAMP) ? tableTimestamp : (scn == null ? HConstants.LATEST_TIMESTAMP : scn);
     }
-        
+
     /**
      * Validates that the meta data is valid against the server meta data if we haven't yet done so.
      * Otherwise, for every UPSERT VALUES call, we'd need to hit the server to see if the meta data
      * has changed.
-     * @param connection
      * @return the server time to use for the upsert
      * @throws SQLException if the table or any columns no longer exist
      */
@@ -842,6 +844,15 @@ public class MutationState implements SQLCloseable {
             }
         }
     }
+
+    public long getMaxSizeBytes() {
+        return maxSizeBytes;
+    }
+
+    public long getBatchCount() {
+        return batchCount;
+    }
+
     private class MetaDataAwareHTable extends DelegateHTable {
         private final TableRef tableRef;
         
@@ -1068,7 +1079,11 @@ public class MutationState implements SQLCloseable {
                         
                         long startTime = System.currentTimeMillis();
                         child.addTimelineAnnotation("Attempt " + retryCount);
-                        hTable.batch(mutationList);
+                        List<List<Mutation>> mutationBatchList = getMutationBatchList(maxSize, maxSizeBytes, mutationList);
+                        for (List<Mutation> mutationBatch : mutationBatchList) {
+                            hTable.batch(mutationBatch);
+                            batchCount++;
+                        }
                         if (logger.isDebugEnabled()) logger.debug("Sent batch of " + numMutations + " for " + Bytes.toString(htableName));
                         child.stop();
                         child.stop();
@@ -1132,6 +1147,34 @@ public class MutationState implements SQLCloseable {
         }
     }
 
+    /**
+     * Split the list of mutations into multiple lists that don't exceed row and byte thresholds
+     * @param allMutationList List of HBase mutations
+     * @return List of lists of mutations
+     */
+    public static List<List<Mutation>> getMutationBatchList(long maxSize, long maxSizeBytes, List<Mutation> allMutationList) {
+        List<List<Mutation>> mutationBatchList = Lists.newArrayList();
+        List<Mutation> currentList = Lists.newArrayList();
+        long currentBatchSizeBytes = 0L;
+        for (Mutation mutation : allMutationList) {
+            long mutationSizeBytes = mutation.heapSize();
+            if (currentList.size() == maxSize || currentBatchSizeBytes + mutationSizeBytes > maxSizeBytes) {
+                if (currentList.size() > 0) {
+                    mutationBatchList.add(currentList);
+                    currentList = Lists.newArrayList();
+                    currentBatchSizeBytes = 0L;
+                }
+            }
+            currentList.add(mutation);
+            currentBatchSizeBytes += mutationSizeBytes;
+        }
+        if (currentList.size() > 0) {
+            mutationBatchList.add(currentList);
+        }
+        return mutationBatchList;
+
+    }
+
     public byte[] encodeTransaction() throws SQLException {
         try {
             return CODEC.encode(getTransaction());

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a44d317e/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
index c060164..cb2390e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
@@ -142,6 +142,7 @@ public class PhoenixConnection implements Connection, MetaDataMutated, SQLClosea
     private final Properties info;
     private final Map<PDataType<?>, Format> formatters = new HashMap<>();
     private final int mutateBatchSize;
+    private final long mutateBatchSizeBytes;
     private final Long scn;
     private MutationState mutationState;
     private List<SQLCloseable> statements = new ArrayList<SQLCloseable>();
@@ -255,6 +256,7 @@ public class PhoenixConnection implements Connection, MetaDataMutated, SQLClosea
                 this.services.getProps().get(QueryServices.SCHEMA_ATTRIB, QueryServicesOptions.DEFAULT_SCHEMA));
         this.tenantId = tenantId;
         this.mutateBatchSize = JDBCUtil.getMutateBatchSize(url, this.info, this.services.getProps());
+        this.mutateBatchSizeBytes = JDBCUtil.getMutateBatchSizeBytes(url, this.info, this.services.getProps());
         datePattern = this.services.getProps().get(QueryServices.DATE_FORMAT_ATTRIB, DateUtil.DEFAULT_DATE_FORMAT);
         timePattern = this.services.getProps().get(QueryServices.TIME_FORMAT_ATTRIB, DateUtil.DEFAULT_TIME_FORMAT);
         timestampPattern = this.services.getProps().get(QueryServices.TIMESTAMP_FORMAT_ATTRIB, DateUtil.DEFAULT_TIMESTAMP_FORMAT);
@@ -443,7 +445,11 @@ public class PhoenixConnection implements Connection, MetaDataMutated, SQLClosea
     public int getMutateBatchSize() {
         return mutateBatchSize;
     }
-    
+
+    public long getMutateBatchSizeBytes(){
+        return mutateBatchSizeBytes;
+    }
+
     public PMetaData getMetaDataCache() {
         return metaData;
     }
@@ -457,7 +463,7 @@ public class PhoenixConnection implements Connection, MetaDataMutated, SQLClosea
     }
 
     protected MutationState newMutationState(int maxSize) {
-        return new MutationState(maxSize, this); 
+        return new MutationState(maxSize, this);
     }
     
     public MutationState getMutationState() {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a44d317e/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/PhoenixIndexImportDirectMapper.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/PhoenixIndexImportDirectMapper.java b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/PhoenixIndexImportDirectMapper.java
index 15e55dd..c1db27c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/PhoenixIndexImportDirectMapper.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/PhoenixIndexImportDirectMapper.java
@@ -65,6 +65,7 @@ public class PhoenixIndexImportDirectMapper extends
     private DirectHTableWriter writer;
 
     private int batchSize;
+    private long batchSizeBytes;
 
     private MutationState mutationState;
 
@@ -87,12 +88,16 @@ public class PhoenixIndexImportDirectMapper extends
             }
             connection = ConnectionUtil.getOutputConnection(configuration, overrideProps);
             connection.setAutoCommit(false);
-            // Get BatchSize
+            // Get BatchSize, which is in terms of rows
             ConnectionQueryServices services = ((PhoenixConnection) connection).getQueryServices();
             int maxSize =
                     services.getProps().getInt(QueryServices.MAX_MUTATION_SIZE_ATTRIB,
                         QueryServicesOptions.DEFAULT_MAX_MUTATION_SIZE);
             batchSize = Math.min(((PhoenixConnection) connection).getMutateBatchSize(), maxSize);
+
+            //Get batch size in terms of bytes
+            batchSizeBytes = ((PhoenixConnection) connection).getMutateBatchSizeBytes();
+
             LOG.info("Mutation Batch Size = " + batchSize);
 
             final String upsertQuery = PhoenixConfigurationUtil.getUpsertStatement(configuration);
@@ -107,8 +112,6 @@ public class PhoenixIndexImportDirectMapper extends
     protected void map(NullWritable key, PhoenixIndexDBWritable record, Context context)
             throws IOException, InterruptedException {
 
-        context.getCounter(PhoenixJobCounters.INPUT_RECORDS).increment(1);
-
         try {
             final List<Object> values = record.getValues();
             indxWritable.setValues(values);
@@ -119,7 +122,6 @@ public class PhoenixIndexImportDirectMapper extends
             MutationState currentMutationState = pconn.getMutationState();
             if (mutationState == null) {
                 mutationState = currentMutationState;
-                return;
             }
             // Keep accumulating Mutations till batch size
             mutationState.join(currentMutationState);
@@ -137,6 +139,7 @@ public class PhoenixIndexImportDirectMapper extends
             context.getCounter(PhoenixJobCounters.FAILED_RECORDS).increment(1);
             throw new RuntimeException(e);
         }
+        context.getCounter(PhoenixJobCounters.INPUT_RECORDS).increment(1);
     }
 
     private void writeBatch(MutationState mutationState, Context context) throws IOException,
@@ -144,8 +147,12 @@ public class PhoenixIndexImportDirectMapper extends
         final Iterator<Pair<byte[], List<Mutation>>> iterator = mutationState.toMutations(true, null);
         while (iterator.hasNext()) {
             Pair<byte[], List<Mutation>> mutationPair = iterator.next();
-
-            writer.write(mutationPair.getSecond());
+            List<Mutation> batchMutations = mutationPair.getSecond();
+            List<List<Mutation>> batchOfBatchMutations =
+                MutationState.getMutationBatchList(batchSize, batchSizeBytes, batchMutations);
+            for (List<Mutation> mutationList : batchOfBatchMutations) {
+                writer.write(mutationList);
+            }
             context.getCounter(PhoenixJobCounters.OUTPUT_RECORDS).increment(
                 mutationPair.getSecond().size());
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a44d317e/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
index 233007f..2035de8 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
@@ -86,7 +86,10 @@ public interface QueryServices extends SQLCloseable {
     public static final String CALL_QUEUE_ROUND_ROBIN_ATTRIB = "ipc.server.callqueue.roundrobin";
     public static final String SCAN_CACHE_SIZE_ATTRIB = "hbase.client.scanner.caching";
     public static final String MAX_MUTATION_SIZE_ATTRIB = "phoenix.mutate.maxSize";
+
+    @Deprecated //USE MUTATE_BATCH_SIZE_BYTES_ATTRIB instead
     public static final String MUTATE_BATCH_SIZE_ATTRIB = "phoenix.mutate.batchSize";
+    public static final String MUTATE_BATCH_SIZE_BYTES_ATTRIB = "phoenix.mutate.batchSizeBytes";
     public static final String MAX_SERVER_CACHE_TIME_TO_LIVE_MS_ATTRIB = "phoenix.coprocessor.maxServerCacheTimeToLiveMs";
     
     @Deprecated // Use FORCE_ROW_KEY_ORDER instead.

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a44d317e/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
index df203b5..de0796f 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
@@ -128,7 +128,10 @@ public class QueryServicesOptions {
     public static final boolean DEFAULT_DROP_METADATA = true; // Drop meta data also.
     public static final long DEFAULT_DRIVER_SHUTDOWN_TIMEOUT_MS = 5  * 1000; // Time to wait in ShutdownHook to exit gracefully.
 
-    public final static int DEFAULT_MUTATE_BATCH_SIZE = 1000; // Batch size for UPSERT SELECT and DELETE
+    @Deprecated //use DEFAULT_MUTATE_BATCH_SIZE_BYTES
+    public final static int DEFAULT_MUTATE_BATCH_SIZE = 100; // Batch size for UPSERT SELECT and DELETE
+    //Batch size in bytes for UPSERT, SELECT and DELETE. By default, 10MB
+    public final static long DEFAULT_MUTATE_BATCH_SIZE_BYTES = 134217728;
 	// The only downside of it being out-of-sync is that the parallelization of the scan won't be as balanced as it could be.
     public static final int DEFAULT_MAX_SERVER_CACHE_TIME_TO_LIVE_MS = 30000; // 30 sec (with no activity)
     public static final int DEFAULT_SCAN_CACHE_SIZE = 1000;
@@ -449,6 +452,7 @@ public class QueryServicesOptions {
         return set(MAX_MUTATION_SIZE_ATTRIB, maxMutateSize);
     }
 
+    @Deprecated
     public QueryServicesOptions setMutateBatchSize(int mutateBatchSize) {
         return set(MUTATE_BATCH_SIZE_ATTRIB, mutateBatchSize);
     }
@@ -513,6 +517,7 @@ public class QueryServicesOptions {
         return config.getInt(MAX_MUTATION_SIZE_ATTRIB, DEFAULT_MAX_MUTATION_SIZE);
     }
 
+    @Deprecated
     public int getMutateBatchSize() {
         return config.getInt(MUTATE_BATCH_SIZE_ATTRIB, DEFAULT_MUTATE_BATCH_SIZE);
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a44d317e/phoenix-core/src/main/java/org/apache/phoenix/util/JDBCUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/JDBCUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/JDBCUtil.java
index f835a21..2cab6fb 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/JDBCUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/JDBCUtil.java
@@ -133,11 +133,17 @@ public class JDBCUtil {
         return (scnStr == null ? null : Long.parseLong(scnStr));
     }
 
+    @Deprecated // use getMutateBatchSizeBytes
     public static int getMutateBatchSize(String url, Properties info, ReadOnlyProps props) throws SQLException {
         String batchSizeStr = findProperty(url, info, PhoenixRuntime.UPSERT_BATCH_SIZE_ATTRIB);
         return (batchSizeStr == null ? props.getInt(QueryServices.MUTATE_BATCH_SIZE_ATTRIB, QueryServicesOptions.DEFAULT_MUTATE_BATCH_SIZE) : Integer.parseInt(batchSizeStr));
     }
 
+    public static long getMutateBatchSizeBytes(String url, Properties info, ReadOnlyProps props) throws SQLException {
+        String batchSizeStr = findProperty(url, info, PhoenixRuntime.UPSERT_BATCH_SIZE_BYTES_ATTRIB);
+        return (batchSizeStr == null ? props.getLong(QueryServices.MUTATE_BATCH_SIZE_BYTES_ATTRIB, QueryServicesOptions.DEFAULT_MUTATE_BATCH_SIZE_BYTES) : Long.parseLong(batchSizeStr));
+    }
+
     public static @Nullable PName getTenantId(String url, Properties info) throws SQLException {
         String tenantId = findProperty(url, info, PhoenixRuntime.TENANT_ID_ATTRIB);
         return (tenantId == null ? null : PNameFactory.newName(tenantId));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a44d317e/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java b/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java
index 3c16d00..5bfb55d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java
@@ -147,6 +147,16 @@ public class PhoenixRuntime {
     public final static String UPSERT_BATCH_SIZE_ATTRIB = "UpsertBatchSize";
 
     /**
+     * Use this connection property to control the number of bytes that are
+     * batched together on an UPSERT INTO table1... SELECT ... FROM table2.
+     * It's only used when autoCommit is true and your source table is
+     * different than your target table or your SELECT statement has a
+     * GROUP BY clause. Overrides the value of UpsertBatchSize.
+     */
+    public final static String UPSERT_BATCH_SIZE_BYTES_ATTRIB = "UpsertBatchSizeBytes";
+
+
+    /**
      * Use this connection property to explicitly enable or disable auto-commit on a new connection.
      */
     public static final String AUTO_COMMIT_ATTRIB = "AutoCommit";
@@ -874,7 +884,7 @@ public class PhoenixRuntime {
      * Column names and family names are enclosed in double quotes to allow for case sensitivity and for presence of 
      * special characters. Salting column and view index id column are not included. If the connection is tenant specific 
      * and the table used by the query plan is multi-tenant, then the tenant id column is not included as well.
-     * @param datatypes - Initialized empty list to be filled with the corresponding data type for the columns in @param columns. 
+     * @param dataTypes - Initialized empty list to be filled with the corresponding data type for the columns in @param columns.
      * @param plan - query plan to get info for
      * @param conn - phoenix connection used to generate the query plan. Caller should take care of closing the connection appropriately.
      * @param forDataTable - if true, then column names and data types correspond to the data table even if the query plan uses

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a44d317e/phoenix-core/src/test/java/org/apache/phoenix/jdbc/PhoenixDriverTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/jdbc/PhoenixDriverTest.java b/phoenix-core/src/test/java/org/apache/phoenix/jdbc/PhoenixDriverTest.java
index 4ab75a9..c87c2db 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/jdbc/PhoenixDriverTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/jdbc/PhoenixDriverTest.java
@@ -77,6 +77,15 @@ public class PhoenixDriverTest extends BaseConnectionlessQueryTest {
             fail("Upsert should have failed since the number of upserts (200) is greater than the MAX_MUTATION_SIZE_ATTRIB (100)");
         } catch (IllegalArgumentException expected) {}
     }
+
+    @Test
+    public void testMaxMutationSizeInBytesSetCorrectly() throws Exception {
+        Properties connectionProperties = new Properties();
+        connectionProperties.setProperty(QueryServices.MUTATE_BATCH_SIZE_BYTES_ATTRIB,"100");
+        PhoenixConnection connection = (PhoenixConnection) DriverManager.getConnection(getUrl(), connectionProperties);
+        assertEquals(100L, connection.getMutateBatchSizeBytes());
+        assertEquals(100L, connection.getMutationState().getMaxSizeBytes());
+    }
     
     @Test
     public void testDisallowNegativeScn() {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a44d317e/phoenix-core/src/test/java/org/apache/phoenix/util/JDBCUtilTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/util/JDBCUtilTest.java b/phoenix-core/src/test/java/org/apache/phoenix/util/JDBCUtilTest.java
index 8c9a8a0..fae7633 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/util/JDBCUtilTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/util/JDBCUtilTest.java
@@ -27,7 +27,9 @@ import static org.junit.Assert.assertTrue;
 import java.util.Map;
 import java.util.Properties;
 
+import com.google.common.collect.Maps;
 import org.apache.hadoop.hbase.client.Consistency;
+import org.apache.phoenix.query.QueryServices;
 import org.junit.Test;
 
 public class JDBCUtilTest {
@@ -107,7 +109,7 @@ public class JDBCUtilTest {
     @Test
     public void testGetConsistency_TIMELINE_InUrl() {
         assertTrue(JDBCUtil.getConsistencyLevel("localhost;Consistency=TIMELINE", new Properties(),
-                Consistency.STRONG.toString()) == Consistency.TIMELINE);
+            Consistency.STRONG.toString()) == Consistency.TIMELINE);
     }
 
     @Test
@@ -122,6 +124,21 @@ public class JDBCUtilTest {
         Properties props = new Properties();
         props.setProperty(PhoenixRuntime.CONSISTENCY_ATTRIB, "TIMELINE");
         assertTrue(JDBCUtil.getConsistencyLevel("localhost", props, Consistency.STRONG.toString())
-                == Consistency.TIMELINE);
+            == Consistency.TIMELINE);
+    }
+
+    @Test
+    public void testGetMaxMutateBytes() throws Exception {
+        assertEquals(1000L, JDBCUtil.getMutateBatchSizeBytes("localhost;" + PhoenixRuntime.UPSERT_BATCH_SIZE_BYTES_ATTRIB +
+            "=1000", new Properties(), ReadOnlyProps.EMPTY_PROPS));
+
+        Properties props = new Properties();
+        props.setProperty(PhoenixRuntime.UPSERT_BATCH_SIZE_BYTES_ATTRIB, "2000");
+        assertEquals(2000L, JDBCUtil.getMutateBatchSizeBytes("localhost", props, ReadOnlyProps.EMPTY_PROPS));
+
+        Map<String, String> propMap = Maps.newHashMap();
+        propMap.put(QueryServices.MUTATE_BATCH_SIZE_BYTES_ATTRIB, "3000");
+        ReadOnlyProps readOnlyProps = new ReadOnlyProps(propMap);
+        assertEquals(3000L, JDBCUtil.getMutateBatchSizeBytes("localhost", new Properties(), readOnlyProps));
     }
 }


[26/26] phoenix git commit: Merge remote-tracking branch 'origin/master' into calcite

Posted by ma...@apache.org.
Merge remote-tracking branch 'origin/master' into calcite


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

Branch: refs/heads/calcite
Commit: 4054990479990f4f6551eb97fff8ffd91d15d526
Parents: 948d009 069c371
Author: maryannxue <ma...@gmail.com>
Authored: Thu Jan 26 17:14:57 2017 -0800
Committer: maryannxue <ma...@gmail.com>
Committed: Thu Jan 26 17:14:57 2017 -0800

----------------------------------------------------------------------
 bin/phoenix_utils.py                            |   5 +
 bin/sqlline-thin.py                             |  17 +-
 bin/sqlline.py                                  |  52 +--
 .../AlterMultiTenantTableWithViewsIT.java       |   2 +-
 .../apache/phoenix/end2end/AlterTableIT.java    |  16 +-
 .../phoenix/end2end/AlterTableWithViewsIT.java  |   2 +-
 .../phoenix/end2end/ArithmeticQueryIT.java      |   2 +-
 .../phoenix/end2end/ArrayFillFunctionIT.java    |  28 +-
 .../phoenix/end2end/AutomaticRebuildIT.java     | 219 +++++++++
 .../end2end/BaseTenantSpecificTablesIT.java     |   4 +-
 .../phoenix/end2end/CoalesceFunctionIT.java     |  60 +--
 .../end2end/ConvertTimezoneFunctionIT.java      |  22 +-
 .../org/apache/phoenix/end2end/DateTimeIT.java  |   6 +-
 .../org/apache/phoenix/end2end/DeleteIT.java    |  26 +-
 .../phoenix/end2end/End2EndTestDriver.java      |  12 +-
 .../phoenix/end2end/EvaluationOfORIT.java       |   5 +-
 .../apache/phoenix/end2end/GroupByCaseIT.java   |  10 +-
 .../org/apache/phoenix/end2end/GroupByIT.java   |   6 +-
 .../apache/phoenix/end2end/HashJoinMoreIT.java  |  47 +-
 .../end2end/IndexToolForPartialBuildIT.java     | 298 ++++++++++++
 ...olForPartialBuildWithNamespaceEnabledIT.java |  70 +++
 .../org/apache/phoenix/end2end/QueryMoreIT.java |  35 ++
 .../phoenix/end2end/SortMergeJoinMoreIT.java    |  37 +-
 .../org/apache/phoenix/end2end/SortOrderIT.java |   8 +-
 .../end2end/TenantSpecificTablesDDLIT.java      |  12 +-
 .../end2end/TenantSpecificTablesDMLIT.java      |  66 +--
 .../end2end/TenantSpecificViewIndexIT.java      |  47 ++
 .../end2end/UpsertSelectAutoCommitIT.java       |   2 +-
 .../apache/phoenix/end2end/UpsertSelectIT.java  |   2 +-
 .../apache/phoenix/end2end/UpsertValuesIT.java  |  20 +-
 .../phoenix/end2end/index/ImmutableIndexIT.java |   1 -
 .../phoenix/end2end/index/IndexMetadataIT.java  |  58 +++
 .../end2end/index/MutableIndexFailureIT.java    |  10 +-
 .../phoenix/monitoring/PhoenixMetricsIT.java    | 135 ++++++
 .../org/apache/phoenix/tx/TransactionIT.java    |   9 +-
 .../org/apache/phoenix/tx/TxCheckpointIT.java   |   2 +-
 phoenix-core/src/main/antlr3/PhoenixSQL.g       |   4 +-
 .../apache/phoenix/compile/UpsertCompiler.java  |  44 +-
 .../apache/phoenix/compile/WhereCompiler.java   |   3 +-
 .../apache/phoenix/compile/WhereOptimizer.java  |   7 +-
 .../coprocessor/MetaDataEndpointImpl.java       |   9 +-
 .../coprocessor/MetaDataRegionObserver.java     | 291 +++++++-----
 .../UngroupedAggregateRegionObserver.java       |  31 +-
 .../phoenix/exception/SQLExceptionCode.java     |   3 +-
 .../apache/phoenix/execute/MutationState.java   |  51 ++-
 .../function/ConvertTimezoneFunction.java       |   5 +-
 .../expression/function/EncodeFormat.java       |   4 +-
 .../index/PhoenixIndexFailurePolicy.java        |  52 +--
 .../phoenix/iterate/OrderedResultIterator.java  |   5 +-
 .../apache/phoenix/jdbc/PhoenixConnection.java  |  10 +-
 .../phoenix/jdbc/PhoenixDatabaseMetaData.java   |   2 +
 .../org/apache/phoenix/jdbc/PhoenixDriver.java  |  99 ++--
 .../apache/phoenix/jdbc/PhoenixStatement.java   |   8 +-
 .../phoenix/mapreduce/CsvBulkImportUtil.java    |   7 +-
 .../phoenix/mapreduce/CsvBulkLoadTool.java      |  12 +-
 .../phoenix/mapreduce/index/IndexTool.java      | 455 +++++++++++++------
 .../phoenix/mapreduce/index/IndexToolUtil.java  |   6 +-
 .../index/PhoenixIndexImportDirectMapper.java   |  21 +-
 .../index/PhoenixIndexPartialBuildMapper.java   | 182 ++++++++
 .../util/PhoenixConfigurationUtil.java          |  31 ++
 .../phoenix/monitoring/GlobalClientMetrics.java |   6 +-
 .../apache/phoenix/monitoring/MetricType.java   |   4 +-
 .../phoenix/parse/AlterIndexStatement.java      |   8 +-
 .../apache/phoenix/parse/ParseNodeFactory.java  |   6 +-
 .../query/ConnectionQueryServicesImpl.java      |  47 +-
 .../java/org/apache/phoenix/query/KeyRange.java |   5 +-
 .../org/apache/phoenix/query/QueryServices.java |  13 +
 .../phoenix/query/QueryServicesOptions.java     |  18 +-
 .../apache/phoenix/schema/MetaDataClient.java   |  47 +-
 .../stats/DefaultStatisticsCollector.java       |  32 +-
 .../apache/phoenix/schema/types/PBinary.java    |   6 +-
 .../apache/phoenix/schema/types/PVarbinary.java |   5 +-
 .../org/apache/phoenix/util/ExpressionUtil.java |  10 +
 .../java/org/apache/phoenix/util/IndexUtil.java |  61 ++-
 .../java/org/apache/phoenix/util/JDBCUtil.java  |   6 +
 .../org/apache/phoenix/util/PhoenixRuntime.java |  23 +-
 .../phoenix/util/csv/CsvUpsertExecutor.java     |  25 +
 .../phoenix/util/json/JsonUpsertExecutor.java   |  44 ++
 .../phoenix/compile/JoinQueryCompilerTest.java  |   6 +-
 .../phoenix/compile/QueryCompilerTest.java      |   8 +-
 .../iterate/OrderedResultIteratorTest.java      |  41 ++
 .../apache/phoenix/jdbc/PhoenixDriverTest.java  |   9 +
 .../mapreduce/CsvBulkImportUtilTest.java        |   8 +-
 .../phoenix/query/KeyRangeIntersectTest.java    |   9 +-
 .../util/AbstractUpsertExecutorTest.java        |  82 +++-
 .../org/apache/phoenix/util/JDBCUtilTest.java   |  21 +-
 .../phoenix/util/csv/CsvUpsertExecutorTest.java |  26 +-
 .../util/json/JsonUpsertExecutorTest.java       |   6 +
 phoenix-spark/src/it/resources/globalSetup.sql  |   7 +-
 .../apache/phoenix/spark/PhoenixSparkIT.scala   |  42 +-
 .../phoenix/spark/DataFrameFunctions.scala      |   6 +-
 .../org/apache/phoenix/spark/PhoenixRDD.scala   |  22 +-
 .../phoenix/spark/ProductRDDFunctions.scala     |   3 +-
 .../tracingwebapp/http/EntityFactory.java       |  19 +-
 .../tracingwebapp/http/TraceServlet.java        |  21 +-
 pom.xml                                         |  16 +-
 96 files changed, 2657 insertions(+), 756 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/40549904/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereCompiler.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/40549904/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/40549904/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/40549904/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/40549904/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/40549904/pom.xml
----------------------------------------------------------------------
diff --cc pom.xml
index 1a18ceb,d47a6b8..4005ee4
--- a/pom.xml
+++ b/pom.xml
@@@ -97,12 -97,11 +97,12 @@@
      <jodatime.version>1.6</jodatime.version>
      <joni.version>2.1.2</joni.version>
      <avatica.version>1.9.0</avatica.version>
 +    <calcite.version>1.12.0-SNAPSHOT</calcite.version>
      <jettyVersion>8.1.7.v20120910</jettyVersion>
      <tephra.version>0.9.0-incubating</tephra.version>
-     <spark.version>1.6.1</spark.version>
-     <scala.version>2.10.4</scala.version>
-     <scala.binary.version>2.10</scala.binary.version>
+     <spark.version>2.0.2</spark.version>
+     <scala.version>2.11.8</scala.version>
+     <scala.binary.version>2.11</scala.binary.version>
  
      <!-- Test Dependencies -->
      <mockito-all.version>1.8.5</mockito-all.version>


[06/26] phoenix git commit: PHOENIX-2890 Extend IndexTool to allow incremental index rebuilds

Posted by ma...@apache.org.
PHOENIX-2890 Extend IndexTool to allow incremental index rebuilds


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

Branch: refs/heads/calcite
Commit: 83827cd8c2876c6b6dccf3a5678889b40a76261b
Parents: 70dc383
Author: Ankit Singhal <an...@gmail.com>
Authored: Mon Dec 26 11:59:19 2016 +0530
Committer: Ankit Singhal <an...@gmail.com>
Committed: Mon Dec 26 11:59:19 2016 +0530

----------------------------------------------------------------------
 .../phoenix/end2end/AutomaticRebuildIT.java     | 219 +++++++++
 .../end2end/IndexToolForPartialBuildIT.java     | 298 ++++++++++++
 ...olForPartialBuildWithNamespaceEnabledIT.java |  70 +++
 .../phoenix/end2end/index/IndexMetadataIT.java  |  58 +++
 .../end2end/index/MutableIndexFailureIT.java    |  10 +-
 phoenix-core/src/main/antlr3/PhoenixSQL.g       |   4 +-
 .../coprocessor/MetaDataEndpointImpl.java       |   9 +-
 .../coprocessor/MetaDataRegionObserver.java     | 291 +++++++-----
 .../phoenix/exception/SQLExceptionCode.java     |   3 +-
 .../index/PhoenixIndexFailurePolicy.java        |  52 +--
 .../phoenix/jdbc/PhoenixDatabaseMetaData.java   |   2 +
 .../apache/phoenix/jdbc/PhoenixStatement.java   |   8 +-
 .../phoenix/mapreduce/index/IndexTool.java      | 455 +++++++++++++------
 .../phoenix/mapreduce/index/IndexToolUtil.java  |   6 +-
 .../index/PhoenixIndexImportDirectMapper.java   |   2 +-
 .../index/PhoenixIndexPartialBuildMapper.java   | 182 ++++++++
 .../util/PhoenixConfigurationUtil.java          |  31 ++
 .../phoenix/parse/AlterIndexStatement.java      |   8 +-
 .../apache/phoenix/parse/ParseNodeFactory.java  |   6 +-
 .../org/apache/phoenix/query/QueryServices.java |   4 +
 .../apache/phoenix/schema/MetaDataClient.java   |  47 +-
 .../java/org/apache/phoenix/util/IndexUtil.java |  61 ++-
 22 files changed, 1504 insertions(+), 322 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/83827cd8/phoenix-core/src/it/java/org/apache/phoenix/end2end/AutomaticRebuildIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AutomaticRebuildIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AutomaticRebuildIT.java
new file mode 100644
index 0000000..cbb7745
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AutomaticRebuildIT.java
@@ -0,0 +1,219 @@
+/*
+ * 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.end2end;
+
+import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+
+import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.HBaseIOException;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.coprocessor.SimpleRegionObserver;
+import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.query.QueryServicesOptions;
+import org.apache.phoenix.schema.PIndexState;
+import org.apache.phoenix.schema.PTableType;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.StringUtil;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+import com.google.common.collect.Maps;
+
+/**
+ * Tests for the {@link AutomaticRebuildIT}
+ */
+@RunWith(Parameterized.class)
+public class AutomaticRebuildIT extends BaseOwnClusterIT {
+
+	private final boolean localIndex;
+	protected boolean isNamespaceEnabled = false;
+	protected final String tableDDLOptions;
+
+	public AutomaticRebuildIT(boolean localIndex) {
+		this.localIndex = localIndex;
+		StringBuilder optionBuilder = new StringBuilder();
+		optionBuilder.append(" SPLIT ON(1,2)");
+		this.tableDDLOptions = optionBuilder.toString();
+	}
+
+	@BeforeClass
+	public static void doSetup() throws Exception {
+		Map<String, String> serverProps = Maps.newHashMapWithExpectedSize(7);
+		serverProps.put(QueryServices.EXTRA_JDBC_ARGUMENTS_ATTRIB, QueryServicesOptions.DEFAULT_EXTRA_JDBC_ARGUMENTS);
+		serverProps.put("hbase.coprocessor.region.classes", FailingRegionObserver.class.getName());
+		serverProps.put(" yarn.scheduler.capacity.maximum-am-resource-percent", "1.0");
+		serverProps.put(HConstants.HBASE_CLIENT_RETRIES_NUMBER, "2");
+		serverProps.put(HConstants.HBASE_RPC_TIMEOUT_KEY, "10000");
+		serverProps.put("hbase.client.pause", "5000");
+		serverProps.put(QueryServices.INDEX_FAILURE_HANDLING_REBUILD_PERIOD, "1000");
+		serverProps.put(QueryServices.INDEX_FAILURE_HANDLING_REBUILD_NUMBER_OF_BATCHES_PER_TABLE, "5");
+		Map<String, String> clientProps = Maps.newHashMapWithExpectedSize(1);
+		setUpTestDriver(new ReadOnlyProps(serverProps.entrySet().iterator()),
+				new ReadOnlyProps(clientProps.entrySet().iterator()));
+	}
+
+	@Parameters(name = "localIndex = {0}")
+	public static Collection<Boolean[]> data() {
+		return Arrays.asList(new Boolean[][] { { false }, { true } });
+	}
+
+	@Test
+	public void testSecondaryAutomaticRebuildIndex() throws Exception {
+		String schemaName = generateUniqueName();
+		String dataTableName = generateUniqueName();
+		String fullTableName = SchemaUtil.getTableName(schemaName, dataTableName);
+		final String indxTable = String.format("%s_%s", dataTableName, FailingRegionObserver.INDEX_NAME);
+		Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+		props.setProperty(QueryServices.TRANSACTIONS_ENABLED, Boolean.TRUE.toString());
+		props.setProperty(QueryServices.EXPLAIN_ROW_COUNT_ATTRIB, Boolean.FALSE.toString());
+		props.setProperty(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, Boolean.toString(isNamespaceEnabled));
+		final Connection conn = DriverManager.getConnection(getUrl(), props);
+		Statement stmt = conn.createStatement();
+		try {
+			if (isNamespaceEnabled) {
+				conn.createStatement().execute("CREATE SCHEMA IF NOT EXISTS " + schemaName);
+			}
+			stmt.execute(String.format(
+					"CREATE TABLE %s (ID BIGINT NOT NULL, NAME VARCHAR, ZIP INTEGER CONSTRAINT PK PRIMARY KEY(ID ROW_TIMESTAMP)) %s",
+					fullTableName, tableDDLOptions));
+			String upsertQuery = String.format("UPSERT INTO %s VALUES(?, ?, ?)", fullTableName);
+			PreparedStatement stmt1 = conn.prepareStatement(upsertQuery);
+			FailingRegionObserver.FAIL_WRITE = false;
+			// insert two rows
+			upsertRow(stmt1, 1000);
+			upsertRow(stmt1, 2000);
+
+			conn.commit();
+			stmt.execute(String.format("CREATE %s INDEX %s ON %s  (LPAD(UPPER(NAME),11,'x')||'_xyz') ",
+					(localIndex ? "LOCAL" : ""), indxTable, fullTableName));
+			FailingRegionObserver.FAIL_WRITE = true;
+			upsertRow(stmt1, 3000);
+			upsertRow(stmt1, 4000);
+			upsertRow(stmt1, 5000);
+			try {
+				conn.commit();
+				fail();
+			} catch (SQLException e) {
+			} catch (Exception e) {
+			}
+			FailingRegionObserver.FAIL_WRITE = false;
+			ResultSet rs = conn.getMetaData().getTables(null, StringUtil.escapeLike(schemaName), indxTable,
+					new String[] { PTableType.INDEX.toString() });
+			assertTrue(rs.next());
+			assertEquals(indxTable, rs.getString(3));
+			String indexState = rs.getString("INDEX_STATE");
+			assertEquals(PIndexState.DISABLE.toString(), indexState);
+			assertFalse(rs.next());
+			upsertRow(stmt1, 6000);
+			upsertRow(stmt1, 7000);
+			conn.commit();
+			int maxTries = 4, nTries = 0;
+			boolean isInactive = false;
+			do {
+				rs = conn.createStatement()
+						.executeQuery(String.format("SELECT " + PhoenixDatabaseMetaData.INDEX_STATE + ","
+								+ PhoenixDatabaseMetaData.INDEX_DISABLE_TIMESTAMP + " FROM "
+								+ PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME + " ("
+								+ PhoenixDatabaseMetaData.ASYNC_REBUILD_TIMESTAMP + " bigint) where "
+								+ PhoenixDatabaseMetaData.TABLE_SCHEM + "='" + schemaName + "' and "
+								+ PhoenixDatabaseMetaData.TABLE_NAME + "='" + indxTable + "'"));
+				rs.next();
+				if (PIndexState.INACTIVE.getSerializedValue().equals(rs.getString(1)) && rs.getLong(2) > 3000) {
+					isInactive = true;
+					break;
+				}
+				Thread.sleep(10 * 1000); // sleep 10 secs
+			} while (++nTries < maxTries);
+			assertTrue(isInactive);
+			nTries = 0;
+			boolean isActive = false;
+			do {
+				Thread.sleep(15 * 1000); // sleep 15 secs
+				rs = conn.getMetaData().getTables(null, StringUtil.escapeLike(schemaName), indxTable,
+						new String[] { PTableType.INDEX.toString() });
+				assertTrue(rs.next());
+				if (PIndexState.ACTIVE.toString().equals(rs.getString("INDEX_STATE"))) {
+					isActive = true;
+					break;
+				}
+			} while (++nTries < maxTries);
+			assertTrue(isActive);
+
+		} finally {
+			conn.close();
+		}
+	}
+
+	public static void upsertRow(PreparedStatement stmt, int i) throws SQLException {
+		// insert row
+		stmt.setInt(1, i);
+		stmt.setString(2, "uname" + String.valueOf(i));
+		stmt.setInt(3, 95050 + i);
+		stmt.executeUpdate();
+	}
+
+	public static class FailingRegionObserver extends SimpleRegionObserver {
+		public static volatile boolean FAIL_WRITE = false;
+		public static final String INDEX_NAME = "IDX";
+
+		@Override
+		public void preBatchMutate(ObserverContext<RegionCoprocessorEnvironment> c,
+				MiniBatchOperationInProgress<Mutation> miniBatchOp) throws HBaseIOException {
+			if (c.getEnvironment().getRegionInfo().getTable().getNameAsString().contains(INDEX_NAME) && FAIL_WRITE) {
+				throw new DoNotRetryIOException();
+			}
+			Mutation operation = miniBatchOp.getOperation(0);
+			Set<byte[]> keySet = operation.getFamilyMap().keySet();
+			for (byte[] family : keySet) {
+				if (Bytes.toString(family).startsWith(QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX) && FAIL_WRITE) {
+					throw new DoNotRetryIOException();
+				}
+			}
+		}
+
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/83827cd8/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolForPartialBuildIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolForPartialBuildIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolForPartialBuildIT.java
new file mode 100644
index 0000000..116c47f
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolForPartialBuildIT.java
@@ -0,0 +1,298 @@
+/*
+ * 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.end2end;
+
+import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.HBaseIOException;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.coprocessor.SimpleRegionObserver;
+import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
+import org.apache.phoenix.mapreduce.index.IndexTool;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.query.QueryServicesOptions;
+import org.apache.phoenix.schema.PIndexState;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTableType;
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.apache.phoenix.util.QueryUtil;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.StringUtil;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+/**
+ * Tests for the {@link IndexToolForPartialBuildIT}
+ */
+@RunWith(Parameterized.class)
+public class IndexToolForPartialBuildIT extends BaseOwnClusterIT {
+    
+    private final boolean localIndex;
+    protected boolean isNamespaceEnabled = false;
+    protected final String tableDDLOptions;
+    
+    public IndexToolForPartialBuildIT(boolean localIndex) {
+
+        this.localIndex = localIndex;
+        StringBuilder optionBuilder = new StringBuilder();
+        optionBuilder.append(" SPLIT ON(1,2)");
+        this.tableDDLOptions = optionBuilder.toString();
+    }
+    
+    @BeforeClass
+    public static void doSetup() throws Exception {
+        Map<String, String> serverProps = Maps.newHashMapWithExpectedSize(7);
+        serverProps.put(QueryServices.EXTRA_JDBC_ARGUMENTS_ATTRIB, QueryServicesOptions.DEFAULT_EXTRA_JDBC_ARGUMENTS);
+        serverProps.put("hbase.coprocessor.region.classes", FailingRegionObserver.class.getName());
+        serverProps.put(" yarn.scheduler.capacity.maximum-am-resource-percent", "1.0");
+        serverProps.put(HConstants.HBASE_CLIENT_RETRIES_NUMBER, "2");
+        serverProps.put(HConstants.HBASE_RPC_TIMEOUT_KEY, "10000");
+        serverProps.put("hbase.client.pause", "5000");
+        serverProps.put(QueryServices.INDEX_FAILURE_HANDLING_REBUILD_ATTRIB, Boolean.FALSE.toString());
+        Map<String, String> clientProps = Maps.newHashMapWithExpectedSize(1);
+        setUpTestDriver(new ReadOnlyProps(serverProps.entrySet().iterator()), new ReadOnlyProps(clientProps.entrySet().iterator()));
+    }
+    
+    @Parameters(name="localIndex = {0}")
+    public static Collection<Boolean[]> data() {
+        return Arrays.asList(new Boolean[][] {     
+                 { false},{ true }
+           });
+    }
+    
+    @Test
+    public void testSecondaryIndex() throws Exception {
+        String schemaName = generateUniqueName();
+        String dataTableName = generateUniqueName();
+        String fullTableName = SchemaUtil.getTableName(schemaName, dataTableName);
+        final String indxTable = String.format("%s_%s", dataTableName, FailingRegionObserver.INDEX_NAME);
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        props.setProperty(QueryServices.TRANSACTIONS_ENABLED, Boolean.TRUE.toString());
+        props.setProperty(QueryServices.EXPLAIN_ROW_COUNT_ATTRIB, Boolean.FALSE.toString());
+        props.setProperty(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, Boolean.toString(isNamespaceEnabled));
+        final Connection conn = DriverManager.getConnection(getUrl(), props);
+        Statement stmt = conn.createStatement();
+        try {
+            if (isNamespaceEnabled) {
+                conn.createStatement().execute("CREATE SCHEMA IF NOT EXISTS " + schemaName);
+            }
+            stmt.execute(
+                    String.format("CREATE TABLE %s (ID BIGINT NOT NULL, NAME VARCHAR, ZIP INTEGER CONSTRAINT PK PRIMARY KEY(ID ROW_TIMESTAMP)) %s",
+                            fullTableName, tableDDLOptions));
+            String upsertQuery = String.format("UPSERT INTO %s VALUES(?, ?, ?)", fullTableName);
+            PreparedStatement stmt1 = conn.prepareStatement(upsertQuery);
+            FailingRegionObserver.FAIL_WRITE = false;
+            // insert two rows
+            upsertRow(stmt1, 1000);
+            upsertRow(stmt1, 2000);
+
+            conn.commit();
+            stmt.execute(String.format("CREATE %s INDEX %s ON %s  (LPAD(UPPER(NAME),11,'x')||'_xyz') ",
+                    (localIndex ? "LOCAL" : ""), indxTable, fullTableName));
+            FailingRegionObserver.FAIL_WRITE = true;
+            upsertRow(stmt1, 3000);
+            upsertRow(stmt1, 4000);
+            upsertRow(stmt1, 5000);
+            try {
+                conn.commit();
+                fail();
+            } catch (SQLException e) {} catch (Exception e) {}
+            conn.createStatement()
+                    .execute(String.format("ALTER INDEX %s on %s REBUILD ASYNC", indxTable, fullTableName));
+            
+            FailingRegionObserver.FAIL_WRITE = false;
+            ResultSet rs = conn.getMetaData().getTables(null, StringUtil.escapeLike(schemaName), indxTable,
+                    new String[] { PTableType.INDEX.toString() });
+            assertTrue(rs.next());
+            assertEquals(indxTable, rs.getString(3));
+            String indexState = rs.getString("INDEX_STATE");
+            assertEquals(PIndexState.BUILDING.toString(), indexState);            
+            assertFalse(rs.next());
+            upsertRow(stmt1, 6000);
+            upsertRow(stmt1, 7000);
+            conn.commit();
+            
+			rs = conn.createStatement()
+					.executeQuery(String.format("SELECT " + PhoenixDatabaseMetaData.ASYNC_REBUILD_TIMESTAMP + ","
+							+ PhoenixDatabaseMetaData.INDEX_DISABLE_TIMESTAMP + " FROM "
+							+ PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME + " ("
+							+ PhoenixDatabaseMetaData.ASYNC_REBUILD_TIMESTAMP + " bigint) where "
+							+ PhoenixDatabaseMetaData.TABLE_SCHEM + "='" + schemaName + "' and "
+							+ PhoenixDatabaseMetaData.TABLE_NAME + "='" + indxTable + "'"));
+			rs.next();
+            PTable pindexTable = PhoenixRuntime.getTable(conn, SchemaUtil.getTableName(schemaName, indxTable));
+            assertEquals(PIndexState.BUILDING, pindexTable.getIndexState());
+            assertEquals(rs.getLong(1), pindexTable.getTimeStamp());
+            //assert disabled timestamp
+            assertEquals(rs.getLong(2), 3000);
+
+            String selectSql = String.format("SELECT LPAD(UPPER(NAME),11,'x')||'_xyz',ID FROM %s", fullTableName);
+            rs = conn.createStatement().executeQuery("EXPLAIN " + selectSql);
+            String actualExplainPlan = QueryUtil.getExplainPlan(rs);
+
+            // assert we are pulling from data table.
+			assertExplainPlan(actualExplainPlan, schemaName, dataTableName, null, false, isNamespaceEnabled);
+
+            rs = stmt1.executeQuery(selectSql);
+            for (int i = 1; i <= 7; i++) {
+                assertTrue(rs.next());
+                assertEquals("xxUNAME" + i*1000 + "_xyz", rs.getString(1));
+            }
+
+            // Validate Index table data till disabled timestamp
+            rs = stmt1.executeQuery(String.format("SELECT * FROM %s", SchemaUtil.getTableName(schemaName, indxTable)));
+            for (int i = 1; i <= 2; i++) {
+                assertTrue(rs.next());
+                assertEquals("xxUNAME" + i*1000 + "_xyz", rs.getString(1));
+            }
+            assertFalse(rs.next());
+            // run the index MR job.
+            final IndexTool indexingTool = new IndexTool();
+            Configuration conf = new Configuration(getUtility().getConfiguration());
+            conf.set(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, Boolean.toString(isNamespaceEnabled));
+            indexingTool.setConf(conf);
+
+            final String[] cmdArgs = getArgValues(schemaName, dataTableName);
+            int status = indexingTool.run(cmdArgs);
+            assertEquals(0, status);
+
+            // insert two more rows
+            upsertRow(stmt1, 8000);
+            upsertRow(stmt1, 9000);
+            conn.commit();
+
+            // assert we are pulling from index table.
+            rs = conn.createStatement().executeQuery("EXPLAIN " + selectSql);
+            actualExplainPlan = QueryUtil.getExplainPlan(rs);
+            assertExplainPlan(actualExplainPlan, schemaName, dataTableName, indxTable, localIndex, isNamespaceEnabled);
+
+            rs = stmt.executeQuery(selectSql);
+
+            for (int i = 1; i <= 9; i++) {
+                assertTrue(rs.next());
+                assertEquals("xxUNAME" + i*1000 + "_xyz", rs.getString(1));
+            }
+
+            assertFalse(rs.next());
+
+           // conn.createStatement().execute(String.format("DROP INDEX  %s ON %s", indxTable, fullTableName));
+        } finally {
+            conn.close();
+        }
+    }
+    
+	public static void assertExplainPlan(final String actualExplainPlan, String schemaName, String dataTable,
+			String indxTable, boolean isLocal, boolean isNamespaceMapped) {
+
+		String expectedExplainPlan = "";
+		if (indxTable != null) {
+			if (isLocal) {
+				final String localIndexName = SchemaUtil
+						.getPhysicalHBaseTableName(SchemaUtil.getTableName(schemaName, dataTable), isNamespaceMapped,
+								PTableType.INDEX)
+						.getString();
+				expectedExplainPlan = String.format("CLIENT PARALLEL 3-WAY RANGE SCAN OVER %s [1]", localIndexName);
+			} else {
+				expectedExplainPlan = String.format("CLIENT PARALLEL 1-WAY FULL SCAN OVER %s",
+						SchemaUtil.getPhysicalHBaseTableName(SchemaUtil.getTableName(schemaName, indxTable),
+								isNamespaceMapped, PTableType.INDEX));
+			}
+		} else {
+			expectedExplainPlan = String.format("CLIENT PARALLEL 1-WAY FULL SCAN OVER %s",
+					SchemaUtil.getPhysicalHBaseTableName(SchemaUtil.getTableName(schemaName, dataTable),
+							isNamespaceMapped, PTableType.TABLE));
+		}
+		assertTrue(actualExplainPlan.contains(expectedExplainPlan));
+	}
+
+    public String[] getArgValues(String schemaName, String dataTable) {
+        final List<String> args = Lists.newArrayList();
+        if (schemaName!=null) {
+            args.add("-s");
+            args.add(schemaName);
+        }
+        args.add("-dt");
+        args.add(dataTable);
+        args.add("-pr");
+        args.add("-op");
+        args.add("/tmp/output/partialTable_"+localIndex);
+        return args.toArray(new String[0]);
+    }
+
+    public static void upsertRow(PreparedStatement stmt, int i) throws SQLException {
+        // insert row
+        stmt.setInt(1, i);
+        stmt.setString(2, "uname" + String.valueOf(i));
+        stmt.setInt(3, 95050 + i);
+        stmt.executeUpdate();
+    }
+    
+
+    public static class FailingRegionObserver extends SimpleRegionObserver {
+        public static volatile boolean FAIL_WRITE = false;
+        public static final String INDEX_NAME = "IDX";
+        @Override
+        public void preBatchMutate(ObserverContext<RegionCoprocessorEnvironment> c, MiniBatchOperationInProgress<Mutation> miniBatchOp) throws HBaseIOException {
+            if (c.getEnvironment().getRegionInfo().getTable().getNameAsString().contains(INDEX_NAME) && FAIL_WRITE) {
+                throw new DoNotRetryIOException();
+            }
+            Mutation operation = miniBatchOp.getOperation(0);
+            Set<byte[]> keySet = operation.getFamilyMap().keySet();
+            for(byte[] family: keySet) {
+                if(Bytes.toString(family).startsWith(QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX) && FAIL_WRITE) {
+                    throw new DoNotRetryIOException();
+                }
+            }
+        }
+
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/83827cd8/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolForPartialBuildWithNamespaceEnabledIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolForPartialBuildWithNamespaceEnabledIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolForPartialBuildWithNamespaceEnabledIT.java
new file mode 100644
index 0000000..5e16b05
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolForPartialBuildWithNamespaceEnabledIT.java
@@ -0,0 +1,70 @@
+/*
+ * 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.end2end;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Map;
+
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.query.QueryServicesOptions;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.junit.BeforeClass;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+import com.google.common.collect.Maps;
+
+/**
+ * Tests for the {@link IndexToolForPartialBuildWithNamespaceEnabled}
+ */
+@RunWith(Parameterized.class)
+public class IndexToolForPartialBuildWithNamespaceEnabled extends IndexToolForPartialBuildIT {
+    
+    
+    public IndexToolForPartialBuildWithNamespaceEnabled(boolean localIndex, boolean isNamespaceEnabled) {
+        super(localIndex);
+        this.isNamespaceEnabled=isNamespaceEnabled;
+    }
+    
+    @BeforeClass
+    public static void doSetup() throws Exception {
+        Map<String, String> serverProps = Maps.newHashMapWithExpectedSize(7);
+        serverProps.put(QueryServices.EXTRA_JDBC_ARGUMENTS_ATTRIB, QueryServicesOptions.DEFAULT_EXTRA_JDBC_ARGUMENTS);
+        serverProps.put("hbase.coprocessor.region.classes", FailingRegionObserver.class.getName());
+        serverProps.put(HConstants.HBASE_CLIENT_RETRIES_NUMBER, "2");
+        serverProps.put(HConstants.HBASE_RPC_TIMEOUT_KEY, "10000");
+        serverProps.put("hbase.client.pause", "5000");
+        serverProps.put(QueryServices.INDEX_FAILURE_HANDLING_REBUILD_PERIOD, "2000");
+        serverProps.put(QueryServices.INDEX_FAILURE_HANDLING_REBUILD_INTERVAL_ATTRIB, "1000");
+        serverProps.put(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, "true");
+        Map<String, String> clientProps = Maps.newHashMapWithExpectedSize(1);
+        clientProps.put(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, "true");
+        setUpTestDriver(new ReadOnlyProps(serverProps.entrySet().iterator()), new ReadOnlyProps(clientProps.entrySet().iterator()));
+    }
+    
+    @Parameters(name="localIndex = {0} , isNamespaceEnabled = {1}")
+    public static Collection<Boolean[]> data() {
+        return Arrays.asList(new Boolean[][] {     
+                 { false, true},{ true, false }
+           });
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/83827cd8/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexMetadataIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexMetadataIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexMetadataIT.java
index f0c670b..63a6bd6 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexMetadataIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexMetadataIT.java
@@ -31,6 +31,7 @@ import java.sql.DriverManager;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.SQLException;
+import java.sql.Statement;
 import java.sql.Types;
 import java.util.Properties;
 
@@ -43,10 +44,13 @@ import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.AmbiguousColumnException;
 import org.apache.phoenix.schema.ColumnAlreadyExistsException;
 import org.apache.phoenix.schema.PIndexState;
+import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTableKey;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.TableNotFoundException;
 import org.apache.phoenix.schema.types.PDate;
+import org.apache.phoenix.schema.types.PLong;
+import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.StringUtil;
@@ -216,6 +220,15 @@ public class IndexMetadataIT extends ParallelStatsDisabledIT {
             assertFalse(rs.next());
             
             assertActiveIndex(conn, INDEX_DATA_SCHEMA, indexDataTable);
+            
+            ddl = "ALTER INDEX " + indexName + " ON " + INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + indexDataTable + " REBUILD ASYNC";
+            conn.createStatement().execute(ddl);
+            // Verify the metadata for index is correct.
+            rs = conn.getMetaData().getTables(null, StringUtil.escapeLike(INDEX_DATA_SCHEMA), indexName , new String[] {PTableType.INDEX.toString()});
+            assertTrue(rs.next());
+            assertEquals(indexName , rs.getString(3));
+            assertEquals(PIndexState.BUILDING.toString(), rs.getString("INDEX_STATE"));
+            assertFalse(rs.next());
 
             ddl = "DROP INDEX " + indexName + " ON " + INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + indexDataTable;
             stmt = conn.prepareStatement(ddl);
@@ -568,4 +581,49 @@ public class IndexMetadataIT extends ParallelStatsDisabledIT {
         assertTrue(d2.after(d1));
         assertFalse(rs.next());
     }
+    
+    @Test
+    public void testAsyncRebuildTimestamp() throws Exception {
+        long startTimestamp = System.currentTimeMillis();
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        conn.setAutoCommit(false);
+        String testTable = generateUniqueName();
+
+
+        String ddl = "create table " + testTable  + " (k varchar primary key, v1 varchar, v2 varchar, v3 varchar)";
+        Statement stmt = conn.createStatement();
+        stmt.execute(ddl);
+        String indexName = "R_ASYNCIND_" + generateUniqueName();
+        
+        ddl = "CREATE INDEX " + indexName + "1 ON " + testTable  + " (v1) ";
+        stmt.execute(ddl);
+        ddl = "CREATE INDEX " + indexName + "2 ON " + testTable  + " (v2) ";
+        stmt.execute(ddl);
+        ddl = "CREATE INDEX " + indexName + "3 ON " + testTable  + " (v3)";
+        stmt.execute(ddl);
+        conn.createStatement().execute("ALTER INDEX "+indexName+"1 ON " + testTable +" DISABLE ");
+        conn.createStatement().execute("ALTER INDEX "+indexName+"2 ON " + testTable +" REBUILD ");
+        conn.createStatement().execute("ALTER INDEX "+indexName+"3 ON " + testTable +" REBUILD ASYNC");
+        
+        ResultSet rs = conn.createStatement().executeQuery(
+            "select table_name, " + PhoenixDatabaseMetaData.ASYNC_REBUILD_TIMESTAMP + " " +
+            "from system.catalog (" + PhoenixDatabaseMetaData.ASYNC_REBUILD_TIMESTAMP + " " + PLong.INSTANCE.getSqlTypeName() + ") " +
+            "where " + PhoenixDatabaseMetaData.ASYNC_REBUILD_TIMESTAMP + " !=0 and table_name like 'R_ASYNCIND_%' " +
+            "order by table_name");
+        assertTrue(rs.next());
+        assertEquals(indexName + "3", rs.getString(1));
+        long asyncTimestamp = rs.getLong(2);
+		assertTrue("Async timestamp is recent timestamp", asyncTimestamp > startTimestamp);
+        PTable table = PhoenixRuntime.getTable(conn, indexName+"3");
+        assertEquals(table.getTimeStamp(), asyncTimestamp);
+        assertFalse(rs.next());
+        conn.createStatement().execute("ALTER INDEX "+indexName+"3 ON " + testTable +" DISABLE");
+        rs = conn.createStatement().executeQuery(
+                "select table_name, " + PhoenixDatabaseMetaData.ASYNC_REBUILD_TIMESTAMP + " " +
+                "from system.catalog (" + PhoenixDatabaseMetaData.ASYNC_REBUILD_TIMESTAMP + " " + PLong.INSTANCE.getSqlTypeName() + ") " +
+                "where " + PhoenixDatabaseMetaData.ASYNC_REBUILD_TIMESTAMP + " !=0 and table_name like 'ASYNCIND_%' " +
+                "order by table_name" );
+        assertFalse(rs.next());
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/83827cd8/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexFailureIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexFailureIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexFailureIT.java
index 687b2c2..e9205c9 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexFailureIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexFailureIT.java
@@ -102,7 +102,7 @@ public class MutableIndexFailureIT extends BaseTest {
         this.tableDDLOptions = " SALT_BUCKETS=2 " + (transactional ? ", TRANSACTIONAL=true " : "");
         this.tableName = (localIndex ? "L_" : "") + TestUtil.DEFAULT_DATA_TABLE_NAME + (transactional ? "_TXN" : "")
                 + (isNamespaceMapped ? "_NM" : "");
-        this.indexName = INDEX_NAME;
+        this.indexName = FailingRegionObserver.INDEX_NAME;
         fullTableName = SchemaUtil.getTableName(schema, tableName);
         this.fullIndexName = SchemaUtil.getTableName(schema, indexName);
         this.isNamespaceMapped = isNamespaceMapped;
@@ -155,7 +155,7 @@ public class MutableIndexFailureIT extends BaseTest {
             rs = conn.createStatement().executeQuery(query);
             assertFalse(rs.next());
 
-            FAIL_WRITE = false;
+            FailingRegionObserver.FAIL_WRITE = false;
             conn.createStatement().execute(
                     "CREATE " + (localIndex ? "LOCAL " : "") + "INDEX " + indexName + " ON " + fullTableName + " (v1) INCLUDE (v2)");
             // Create other index which should be local/global if the other index is global/local to
@@ -202,7 +202,7 @@ public class MutableIndexFailureIT extends BaseTest {
             assertEquals("z", rs.getString(2));
             assertFalse(rs.next());
 
-            FAIL_WRITE = true;
+            FailingRegionObserver.FAIL_WRITE = true;
             updateTable(conn, fullTableName);
             updateTable(conn, secondTableName);
             // Verify the metadata for index is correct.
@@ -259,7 +259,7 @@ public class MutableIndexFailureIT extends BaseTest {
             }
 
             // re-enable index table
-            FAIL_WRITE = false;
+            FailingRegionObserver.FAIL_WRITE = false;
             waitForIndexToBeActive(conn,indexName);
             waitForIndexToBeActive(conn,indexName+"_2");
             waitForIndexToBeActive(conn,secondIndexName);
@@ -391,6 +391,8 @@ public class MutableIndexFailureIT extends BaseTest {
     }
 
     public static class FailingRegionObserver extends SimpleRegionObserver {
+        public static volatile boolean FAIL_WRITE = false;
+        public static final String INDEX_NAME = "IDX";
         @Override
         public void preBatchMutate(ObserverContext<RegionCoprocessorEnvironment> c, MiniBatchOperationInProgress<Mutation> miniBatchOp) throws HBaseIOException {
             if (c.getEnvironment().getRegionInfo().getTable().getNameAsString().contains(INDEX_NAME) && FAIL_WRITE) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/83827cd8/phoenix-core/src/main/antlr3/PhoenixSQL.g
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/antlr3/PhoenixSQL.g b/phoenix-core/src/main/antlr3/PhoenixSQL.g
index 3e09766..07a51ce 100644
--- a/phoenix-core/src/main/antlr3/PhoenixSQL.g
+++ b/phoenix-core/src/main/antlr3/PhoenixSQL.g
@@ -567,8 +567,8 @@ drop_index_node returns [DropIndexStatement ret]
 
 // Parse a alter index statement
 alter_index_node returns [AlterIndexStatement ret]
-    : ALTER INDEX (IF ex=EXISTS)? i=index_name ON t=from_table_name s=(USABLE | UNUSABLE | REBUILD | DISABLE | ACTIVE)
-      {ret = factory.alterIndex(factory.namedTable(null, TableName.create(t.getSchemaName(), i.getName())), t.getTableName(), ex!=null, PIndexState.valueOf(SchemaUtil.normalizeIdentifier(s.getText()))); }
+    : ALTER INDEX (IF ex=EXISTS)? i=index_name ON t=from_table_name s=(USABLE | UNUSABLE | REBUILD | DISABLE | ACTIVE) (async=ASYNC)?
+      {ret = factory.alterIndex(factory.namedTable(null, TableName.create(t.getSchemaName(), i.getName())), t.getTableName(), ex!=null, PIndexState.valueOf(SchemaUtil.normalizeIdentifier(s.getText())), async!=null); }
     ;
 
 // Parse a trace statement.

http://git-wip-us.apache.org/repos/asf/phoenix/blob/83827cd8/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
index 38dc494..a0681fb 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
@@ -3296,11 +3296,13 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             List<Cell> newKVs = tableMetadata.get(0).getFamilyCellMap().get(TABLE_FAMILY_BYTES);
             Cell newKV = null;
             int disableTimeStampKVIndex = -1;
+            int indexStateKVIndex = 0;
             int index = 0;
             for(Cell cell : newKVs){
                 if(Bytes.compareTo(cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength(),
                       INDEX_STATE_BYTES, 0, INDEX_STATE_BYTES.length) == 0){
                   newKV = cell;
+                  indexStateKVIndex = index;
                 } else if (Bytes.compareTo(cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength(),
                   INDEX_DISABLE_TIMESTAMP_BYTES, 0, INDEX_DISABLE_TIMESTAMP_BYTES.length) == 0){
                   disableTimeStampKVIndex = index;
@@ -3378,11 +3380,11 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 if ((currentState == PIndexState.UNUSABLE && newState == PIndexState.ACTIVE)
                         || (currentState == PIndexState.ACTIVE && newState == PIndexState.UNUSABLE)) {
                     newState = PIndexState.INACTIVE;
-                    newKVs.set(0, KeyValueUtil.newKeyValue(key, TABLE_FAMILY_BYTES,
+                    newKVs.set(indexStateKVIndex, KeyValueUtil.newKeyValue(key, TABLE_FAMILY_BYTES,
                         INDEX_STATE_BYTES, timeStamp, Bytes.toBytes(newState.getSerializedValue())));
                 } else if (currentState == PIndexState.INACTIVE && newState == PIndexState.USABLE) {
                     newState = PIndexState.ACTIVE;
-                    newKVs.set(0, KeyValueUtil.newKeyValue(key, TABLE_FAMILY_BYTES,
+                    newKVs.set(indexStateKVIndex, KeyValueUtil.newKeyValue(key, TABLE_FAMILY_BYTES,
                         INDEX_STATE_BYTES, timeStamp, Bytes.toBytes(newState.getSerializedValue())));
                 }
 
@@ -3414,7 +3416,8 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     if(dataTableKey != null) {
                         metaDataCache.invalidate(new ImmutableBytesPtr(dataTableKey));
                     }
-                    if (setRowKeyOrderOptimizableCell || disableTimeStampKVIndex != -1) {
+                    if (setRowKeyOrderOptimizableCell || disableTimeStampKVIndex != -1
+                            || currentState == PIndexState.DISABLE || newState == PIndexState.BUILDING) {
                         returnTable = doGetTable(key, HConstants.LATEST_TIMESTAMP, rowLock);
                     }
                 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/83827cd8/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java
index e790b59..a60de03 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java
@@ -17,12 +17,19 @@
  */
 package org.apache.phoenix.coprocessor;
 
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_DISABLE_TIMESTAMP;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_NAME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_SCHEM;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TENANT_ID;
 import static org.apache.phoenix.schema.types.PDataType.TRUE_BYTES;
 
 import java.io.IOException;
+import java.sql.PreparedStatement;
 import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
@@ -45,6 +52,7 @@ import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
 import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.hadoop.hbase.filter.CompareFilter;
+import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
 import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
@@ -69,6 +77,7 @@ import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
 import org.apache.phoenix.schema.MetaDataClient;
 import org.apache.phoenix.schema.PIndexState;
+import org.apache.phoenix.schema.PName;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.TableNotFoundException;
@@ -97,6 +106,7 @@ public class MetaDataRegionObserver extends BaseRegionObserver {
     private boolean enableRebuildIndex = QueryServicesOptions.DEFAULT_INDEX_FAILURE_HANDLING_REBUILD;
     private long rebuildIndexTimeInterval = QueryServicesOptions.DEFAULT_INDEX_FAILURE_HANDLING_REBUILD_INTERVAL;
     private boolean blockWriteRebuildIndex = false;
+    private static Map<PName, Long> batchExecutedPerTableMap = new HashMap<PName, Long>();
 
     @Override
     public void preClose(final ObserverContext<RegionCoprocessorEnvironment> c,
@@ -125,6 +135,7 @@ public class MetaDataRegionObserver extends BaseRegionObserver {
             QueryServicesOptions.DEFAULT_INDEX_FAILURE_HANDLING_REBUILD_INTERVAL);
         blockWriteRebuildIndex = env.getConfiguration().getBoolean(QueryServices.INDEX_FAILURE_BLOCK_WRITE,
         	QueryServicesOptions.DEFAULT_INDEX_FAILURE_BLOCK_WRITE);
+        
     }
     
     @Override
@@ -195,9 +206,15 @@ public class MetaDataRegionObserver extends BaseRegionObserver {
         // running
         private final static AtomicInteger inProgress = new AtomicInteger(0);
         RegionCoprocessorEnvironment env;
+        private long rebuildIndexBatchSize = HConstants.LATEST_TIMESTAMP;
+        private long configuredBatches = 10;
 
         public BuildIndexScheduleTask(RegionCoprocessorEnvironment env) {
             this.env = env;
+            this.rebuildIndexBatchSize = env.getConfiguration().getLong(
+                    QueryServices.INDEX_FAILURE_HANDLING_REBUILD_PERIOD, HConstants.LATEST_TIMESTAMP);
+            this.configuredBatches = env.getConfiguration().getLong(
+                    QueryServices.INDEX_FAILURE_HANDLING_REBUILD_NUMBER_OF_BATCHES_PER_TABLE, configuredBatches);
         }
 
         @Override
@@ -228,6 +245,7 @@ public class MetaDataRegionObserver extends BaseRegionObserver {
                     PhoenixDatabaseMetaData.INDEX_STATE_BYTES);
                 scan.addColumn(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES,
                     PhoenixDatabaseMetaData.INDEX_DISABLE_TIMESTAMP_BYTES);
+                PreparedStatement updateDisabledTimeStampSmt = null;
 
                 Map<PTable, List<PTable>> dataTableToIndexesMap = null;
                 MetaDataClient client = null;
@@ -243,8 +261,13 @@ public class MetaDataRegionObserver extends BaseRegionObserver {
                     Result r = Result.create(results);
                     byte[] disabledTimeStamp = r.getValue(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES,
                         PhoenixDatabaseMetaData.INDEX_DISABLE_TIMESTAMP_BYTES);
+                    byte[] indexState = r.getValue(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES,
+                            PhoenixDatabaseMetaData.INDEX_STATE_BYTES);
+
+                    if (disabledTimeStamp == null || disabledTimeStamp.length == 0 || (indexState != null
+                            && PIndexState.BUILDING == PIndexState.fromSerializedValue(Bytes.toString(indexState)))) {
 
-                    if (disabledTimeStamp == null || disabledTimeStamp.length == 0) {
+                        // Don't rebuild the building index , because they are marked for aysnc
                         continue;
                     }
 
@@ -255,8 +278,6 @@ public class MetaDataRegionObserver extends BaseRegionObserver {
                     }
                     byte[] dataTable = r.getValue(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES,
                         PhoenixDatabaseMetaData.DATA_TABLE_NAME_BYTES);
-                    byte[] indexState = r.getValue(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES,
-                        PhoenixDatabaseMetaData.INDEX_STATE_BYTES);
                     if ((dataTable == null || dataTable.length == 0) || (indexState == null || indexState.length == 0)) {
                         // data table name can't be empty
                         continue;
@@ -317,109 +338,169 @@ public class MetaDataRegionObserver extends BaseRegionObserver {
                     indexesToPartiallyRebuild.add(indexPTable);
                 } while (hasMore);
 
-                if (dataTableToIndexesMap != null) {
-                    long overlapTime = env.getConfiguration().getLong(
-                            QueryServices.INDEX_FAILURE_HANDLING_REBUILD_OVERLAP_TIME_ATTRIB,
-                            QueryServicesOptions.DEFAULT_INDEX_FAILURE_HANDLING_REBUILD_OVERLAP_TIME);
-                    for (Map.Entry<PTable, List<PTable>> entry : dataTableToIndexesMap.entrySet()) {
-                        PTable dataPTable = entry.getKey();
-                        List<PTable> indexesToPartiallyRebuild = entry.getValue();
-                        try {
-                            long earliestDisableTimestamp = Long.MAX_VALUE;
-                            List<IndexMaintainer> maintainers = Lists
-                                    .newArrayListWithExpectedSize(indexesToPartiallyRebuild.size());
-                            for (PTable index : indexesToPartiallyRebuild) {
-                                long disabledTimeStampVal = index.getIndexDisableTimestamp();
-                                if (disabledTimeStampVal > 0) {
-                                    if (disabledTimeStampVal < earliestDisableTimestamp) {
-                                        earliestDisableTimestamp = disabledTimeStampVal;
-                                    }
-    
-                                    maintainers.add(index.getIndexMaintainer(dataPTable, conn));
-                                }
-                            }
-                            // No indexes are disabled, so skip this table
-                            if (earliestDisableTimestamp == Long.MAX_VALUE) {
-                                continue;
-                            }
-
-                            long timeStamp = Math.max(0, earliestDisableTimestamp - overlapTime);
-                            LOG.info("Starting to build " + dataPTable + " indexes " + indexesToPartiallyRebuild
-                                    + " from timestamp=" + timeStamp);
-                            TableRef tableRef = new TableRef(null, dataPTable, HConstants.LATEST_TIMESTAMP, false);
-                            // TODO Need to set high timeout 
-                            PostDDLCompiler compiler = new PostDDLCompiler(conn);
-                            MutationPlan plan = compiler.compile(Collections.singletonList(tableRef), null, null, null,
-                                    HConstants.LATEST_TIMESTAMP);
-                            Scan dataTableScan = IndexManagementUtil.newLocalStateScan(plan.getContext().getScan(),
-                                    maintainers);
-                            dataTableScan.setTimeRange(timeStamp, HConstants.LATEST_TIMESTAMP);
-                            dataTableScan.setCacheBlocks(false);
-                            dataTableScan.setAttribute(BaseScannerRegionObserver.REBUILD_INDEXES, TRUE_BYTES);
-                            
-                            ImmutableBytesWritable indexMetaDataPtr = new ImmutableBytesWritable(
-                                    ByteUtil.EMPTY_BYTE_ARRAY);
-                            IndexMaintainer.serializeAdditional(dataPTable, indexMetaDataPtr, indexesToPartiallyRebuild,
-                                    conn);
-                            byte[] attribValue = ByteUtil.copyKeyBytesIfNecessary(indexMetaDataPtr);
-
-                            dataTableScan.setAttribute(PhoenixIndexCodec.INDEX_MD, attribValue);
-                            MutationState mutationState = plan.execute();
-                            long rowCount = mutationState.getUpdateCount();
-                            LOG.info(rowCount + " rows of index which are rebuild");
-                            for (PTable indexPTable : indexesToPartiallyRebuild) {
-                                String indexTableFullName = SchemaUtil.getTableName(indexPTable.getSchemaName()
-                                        .getString(), indexPTable.getTableName().getString());
-                                updateIndexState(conn, indexTableFullName, env, PIndexState.INACTIVE, PIndexState.ACTIVE);
-                            }
-                        } catch (Exception e) { // Log, but try next table's indexes
-                            LOG.warn("Unable to rebuild " + dataPTable + " indexes " + indexesToPartiallyRebuild
-                                    + ". Will try again next on next scheduled invocation.", e);
-                        }
-                    }
-                }
-            } catch (Throwable t) {
-                LOG.warn("ScheduledBuildIndexTask failed!", t);
-            } finally {
-                inProgress.decrementAndGet();
-                if (scanner != null) {
-                    try {
-                        scanner.close();
-                    } catch (IOException ignored) {
-                        LOG.debug("ScheduledBuildIndexTask can't close scanner.", ignored);
-                    }
-                }
-                if (conn != null) {
-                    try {
-                        conn.close();
-                    } catch (SQLException ignored) {
-                        LOG.debug("ScheduledBuildIndexTask can't close connection", ignored);
-                    }
-                }
-            }
+				if (dataTableToIndexesMap != null) {
+					long overlapTime = env.getConfiguration().getLong(
+							QueryServices.INDEX_FAILURE_HANDLING_REBUILD_OVERLAP_TIME_ATTRIB,
+							QueryServicesOptions.DEFAULT_INDEX_FAILURE_HANDLING_REBUILD_OVERLAP_TIME);
+					for (Map.Entry<PTable, List<PTable>> entry : dataTableToIndexesMap.entrySet()) {
+						PTable dataPTable = entry.getKey();
+						List<PTable> indexesToPartiallyRebuild = entry.getValue();
+						ReadOnlyProps props = new ReadOnlyProps(env.getConfiguration().iterator());
+						try (HTableInterface metaTable = env.getTable(
+								SchemaUtil.getPhysicalName(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES, props))) {
+							long earliestDisableTimestamp = Long.MAX_VALUE;
+							List<IndexMaintainer> maintainers = Lists
+									.newArrayListWithExpectedSize(indexesToPartiallyRebuild.size());
+							for (PTable index : indexesToPartiallyRebuild) {
+								long disabledTimeStampVal = index.getIndexDisableTimestamp();
+								if (disabledTimeStampVal > 0) {
+									if (disabledTimeStampVal < earliestDisableTimestamp) {
+										earliestDisableTimestamp = disabledTimeStampVal;
+									}
+
+									maintainers.add(index.getIndexMaintainer(dataPTable, conn));
+								}
+							}
+							// No indexes are disabled, so skip this table
+							if (earliestDisableTimestamp == Long.MAX_VALUE) {
+								continue;
+							}
+							long timeStamp = Math.max(0, earliestDisableTimestamp - overlapTime);
+							LOG.info("Starting to build " + dataPTable + " indexes " + indexesToPartiallyRebuild
+									+ " from timestamp=" + timeStamp);
+
+							TableRef tableRef = new TableRef(null, dataPTable, HConstants.LATEST_TIMESTAMP, false);
+							// TODO Need to set high timeout
+							PostDDLCompiler compiler = new PostDDLCompiler(conn);
+							MutationPlan plan = compiler.compile(Collections.singletonList(tableRef), null, null, null,
+									HConstants.LATEST_TIMESTAMP);
+							Scan dataTableScan = IndexManagementUtil.newLocalStateScan(plan.getContext().getScan(),
+									maintainers);
+
+							long scanEndTime = getTimestampForBatch(timeStamp,
+									batchExecutedPerTableMap.get(dataPTable.getName()));
+							dataTableScan.setTimeRange(timeStamp, scanEndTime);
+							dataTableScan.setCacheBlocks(false);
+							dataTableScan.setAttribute(BaseScannerRegionObserver.REBUILD_INDEXES, TRUE_BYTES);
+
+							ImmutableBytesWritable indexMetaDataPtr = new ImmutableBytesWritable(
+									ByteUtil.EMPTY_BYTE_ARRAY);
+							IndexMaintainer.serializeAdditional(dataPTable, indexMetaDataPtr, indexesToPartiallyRebuild,
+									conn);
+							byte[] attribValue = ByteUtil.copyKeyBytesIfNecessary(indexMetaDataPtr);
+							dataTableScan.setAttribute(PhoenixIndexCodec.INDEX_MD, attribValue);
+							MutationState mutationState = plan.execute();
+							long rowCount = mutationState.getUpdateCount();
+							LOG.info(rowCount + " rows of index which are rebuild");
+							for (PTable indexPTable : indexesToPartiallyRebuild) {
+								String indexTableFullName = SchemaUtil.getTableName(
+										indexPTable.getSchemaName().getString(),
+										indexPTable.getTableName().getString());
+								if (scanEndTime == HConstants.LATEST_TIMESTAMP) {
+									updateIndexState(conn, indexTableFullName, env, PIndexState.INACTIVE,
+											PIndexState.ACTIVE);
+									batchExecutedPerTableMap.remove(dataPTable.getName());
+								} else {
+
+									updateDisableTimestamp(conn, indexTableFullName, env, scanEndTime, metaTable);
+									Long noOfBatches = batchExecutedPerTableMap.get(dataPTable.getName());
+									if (noOfBatches == null) {
+										noOfBatches = 0l;
+									}
+									batchExecutedPerTableMap.put(dataPTable.getName(), ++noOfBatches);
+									// clearing cache to get the updated
+									// disabled timestamp
+									new MetaDataClient(conn).updateCache(dataPTable.getSchemaName().getString(),
+											dataPTable.getTableName().getString());
+									new MetaDataClient(conn).updateCache(indexPTable.getSchemaName().getString(),
+											indexPTable.getTableName().getString());
+									LOG.info(
+											"During Round-robin build: Successfully updated index disabled timestamp  for "
+													+ indexTableFullName + " to " + scanEndTime);
+								}
+
+							}
+						} catch (Exception e) { // Log, but try next table's
+												// indexes
+							LOG.warn("Unable to rebuild " + dataPTable + " indexes " + indexesToPartiallyRebuild
+									+ ". Will try again next on next scheduled invocation.", e);
+						}
+					}
+				}
+			} catch (Throwable t) {
+				LOG.warn("ScheduledBuildIndexTask failed!", t);
+			} finally {
+				inProgress.decrementAndGet();
+				if (scanner != null) {
+					try {
+						scanner.close();
+					} catch (IOException ignored) {
+						LOG.debug("ScheduledBuildIndexTask can't close scanner.", ignored);
+					}
+				}
+				if (conn != null) {
+					try {
+						conn.close();
+					} catch (SQLException ignored) {
+						LOG.debug("ScheduledBuildIndexTask can't close connection", ignored);
+					}
+				}
+			}
         }
-    }
-    
-    private static void updateIndexState(PhoenixConnection conn, String indexTableName, RegionCoprocessorEnvironment env, PIndexState oldState,
-            PIndexState newState) throws ServiceException, Throwable {
-        byte[] indexTableKey = SchemaUtil.getTableKeyFromFullName(indexTableName);
-        String schemaName = SchemaUtil.getSchemaNameFromFullName(indexTableName);
-        String indexName = SchemaUtil.getTableNameFromFullName(indexTableName);
-        // Mimic the Put that gets generated by the client on an update of the index state
-        Put put = new Put(indexTableKey);
-        put.addColumn(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES, PhoenixDatabaseMetaData.INDEX_STATE_BYTES,
-                newState.getSerializedBytes());
-        if (newState == PIndexState.ACTIVE) {
-            put.addColumn(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES, PhoenixDatabaseMetaData.INDEX_DISABLE_TIMESTAMP_BYTES,
-                    PLong.INSTANCE.toBytes(0));
+
+        private long getTimestampForBatch(long disabledTimeStamp, Long noOfBatches) {
+            if (disabledTimeStamp < 0 || rebuildIndexBatchSize > (HConstants.LATEST_TIMESTAMP
+                    - disabledTimeStamp)) { return HConstants.LATEST_TIMESTAMP; }
+            long timestampForNextBatch = disabledTimeStamp + rebuildIndexBatchSize;
+			if (timestampForNextBatch < 0 || timestampForNextBatch > System.currentTimeMillis()
+					|| (noOfBatches != null && noOfBatches > configuredBatches)) {
+				// if timestampForNextBatch cross current time , then we should
+				// build the complete index
+				timestampForNextBatch = HConstants.LATEST_TIMESTAMP;
+			}
+            return timestampForNextBatch;
         }
-        final List<Mutation> tableMetadata = Collections.<Mutation> singletonList(put);
-        MetaDataMutationResult result = conn.getQueryServices().updateIndexState(tableMetadata, null);
-        MutationCode code = result.getMutationCode();
-        if (code == MutationCode.TABLE_NOT_FOUND) { throw new TableNotFoundException(schemaName, indexName); }
-        if (code == MutationCode.UNALLOWED_TABLE_MUTATION) { throw new SQLExceptionInfo.Builder(
-                SQLExceptionCode.INVALID_INDEX_STATE_TRANSITION)
-                .setMessage(" currentState=" + oldState + ". requestedState=" + newState).setSchemaName(schemaName)
-                .setTableName(indexName).build().buildException(); }
     }
+    
+	private static void updateIndexState(PhoenixConnection conn, String indexTableName,
+			RegionCoprocessorEnvironment env, PIndexState oldState, PIndexState newState)
+					throws ServiceException, Throwable {
+		byte[] indexTableKey = SchemaUtil.getTableKeyFromFullName(indexTableName);
+		String schemaName = SchemaUtil.getSchemaNameFromFullName(indexTableName);
+		String indexName = SchemaUtil.getTableNameFromFullName(indexTableName);
+		// Mimic the Put that gets generated by the client on an update of the
+		// index state
+		Put put = new Put(indexTableKey);
+		put.addColumn(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES, PhoenixDatabaseMetaData.INDEX_STATE_BYTES,
+				newState.getSerializedBytes());
+		if (newState == PIndexState.ACTIVE) {
+			put.addColumn(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES,
+					PhoenixDatabaseMetaData.INDEX_DISABLE_TIMESTAMP_BYTES, PLong.INSTANCE.toBytes(0));
+			put.addColumn(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES,
+					PhoenixDatabaseMetaData.ASYNC_REBUILD_TIMESTAMP_BYTES, PLong.INSTANCE.toBytes(0));
+		}
+		final List<Mutation> tableMetadata = Collections.<Mutation> singletonList(put);
+		MetaDataMutationResult result = conn.getQueryServices().updateIndexState(tableMetadata, null);
+		MutationCode code = result.getMutationCode();
+		if (code == MutationCode.TABLE_NOT_FOUND) {
+			throw new TableNotFoundException(schemaName, indexName);
+		}
+		if (code == MutationCode.UNALLOWED_TABLE_MUTATION) {
+			throw new SQLExceptionInfo.Builder(SQLExceptionCode.INVALID_INDEX_STATE_TRANSITION)
+					.setMessage(" currentState=" + oldState + ". requestedState=" + newState).setSchemaName(schemaName)
+					.setTableName(indexName).build().buildException();
+		}
+	}
+
+	private static void updateDisableTimestamp(PhoenixConnection conn, String indexTableName,
+			RegionCoprocessorEnvironment env, long disabledTimestamp, HTableInterface metaTable) throws IOException {
+		byte[] indexTableKey = SchemaUtil.getTableKeyFromFullName(indexTableName);
+		Put put = new Put(indexTableKey);
+		put.addColumn(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES, PhoenixDatabaseMetaData.INDEX_DISABLE_TIMESTAMP_BYTES,
+				PLong.INSTANCE.toBytes(disabledTimestamp));
+		metaTable.checkAndPut(indexTableKey, PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES,
+				PhoenixDatabaseMetaData.INDEX_STATE_BYTES, CompareOp.EQUAL, PIndexState.INACTIVE.getSerializedBytes(),
+				put);
+
+	}
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/83827cd8/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java b/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
index fb4e3c3..fde403c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
@@ -421,7 +421,8 @@ public enum SQLExceptionCode {
             724, "43M07", "Schema name not allowed!!"), CREATE_SCHEMA_NOT_ALLOWED(725, "43M08",
                     "Cannot create schema because config " + QueryServices.IS_NAMESPACE_MAPPING_ENABLED
                             + " for enabling name space mapping isn't enabled."), INCONSISTENET_NAMESPACE_MAPPING_PROPERTIES(
-                                    726, "43M10", " Inconsistent namespace mapping properites..");
+                                    726, "43M10", " Inconsistent namespace mapping properites.."), ASYNC_NOT_ALLOWED(
+                                    727, "43M11", " ASYNC option is not allowed.. ");
 
     private final int errorCode;
     private final String sqlState;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/83827cd8/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexFailurePolicy.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexFailurePolicy.java b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexFailurePolicy.java
index eb73d6b..e515dbb 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexFailurePolicy.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexFailurePolicy.java
@@ -35,30 +35,21 @@ import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.Stoppable;
 import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Mutation;
-import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.coprocessor.Batch;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.hadoop.hbase.ipc.BlockingRpcCallback;
-import org.apache.hadoop.hbase.ipc.ServerRpcController;
-import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto;
 import org.apache.phoenix.coprocessor.MetaDataProtocol.MetaDataMutationResult;
 import org.apache.phoenix.coprocessor.MetaDataProtocol.MutationCode;
-import org.apache.phoenix.coprocessor.generated.MetaDataProtos.MetaDataResponse;
-import org.apache.phoenix.coprocessor.generated.MetaDataProtos.MetaDataService;
-import org.apache.phoenix.coprocessor.generated.MetaDataProtos.UpdateIndexStateRequest;
 import org.apache.phoenix.hbase.index.table.HTableInterfaceReference;
 import org.apache.phoenix.hbase.index.write.DelegateIndexFailurePolicy;
 import org.apache.phoenix.hbase.index.write.KillServerOnFailurePolicy;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
-import org.apache.phoenix.protobuf.ProtobufUtil;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
 import org.apache.phoenix.schema.PIndexState;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTable.IndexType;
-import org.apache.phoenix.schema.types.PLong;
+import org.apache.phoenix.util.IndexUtil;
 import org.apache.phoenix.util.MetaDataUtil;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.QueryUtil;
@@ -153,47 +144,10 @@ public class PhoenixIndexFailurePolicy extends DelegateIndexFailurePolicy {
             String indexTableName = tableTimeElement.getKey();
             long minTimeStamp = tableTimeElement.getValue();
             // Disable the index by using the updateIndexState method of MetaDataProtocol end point coprocessor.
-            byte[] indexTableKey = SchemaUtil.getTableKeyFromFullName(indexTableName);
             HTableInterface systemTable = env.getTable(SchemaUtil
                     .getPhysicalTableName(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES, env.getConfiguration()));
-            // Mimic the Put that gets generated by the client on an update of the index state
-            Put put = new Put(indexTableKey);
-            if (blockWriteRebuildIndex) 
-                put.add(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES, PhoenixDatabaseMetaData.INDEX_STATE_BYTES,
-                        PIndexState.ACTIVE.getSerializedBytes());
-            else  
-                put.add(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES, PhoenixDatabaseMetaData.INDEX_STATE_BYTES,
-                        PIndexState.DISABLE.getSerializedBytes());
-            put.add(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES, PhoenixDatabaseMetaData.INDEX_DISABLE_TIMESTAMP_BYTES,
-                PLong.INSTANCE.toBytes(minTimeStamp));
-            final List<Mutation> tableMetadata = Collections.<Mutation>singletonList(put);
-
-            final Map<byte[], MetaDataResponse> results =
-                    systemTable.coprocessorService(MetaDataService.class, indexTableKey, indexTableKey,
-                            new Batch.Call<MetaDataService, MetaDataResponse>() {
-                                @Override
-                                public MetaDataResponse call(MetaDataService instance) throws IOException {
-                                    ServerRpcController controller = new ServerRpcController();
-                                    BlockingRpcCallback<MetaDataResponse> rpcCallback =
-                                            new BlockingRpcCallback<MetaDataResponse>();
-                                    UpdateIndexStateRequest.Builder builder = UpdateIndexStateRequest.newBuilder();
-                                    for (Mutation m : tableMetadata) {
-                                        MutationProto mp = ProtobufUtil.toProto(m);
-                                        builder.addTableMetadataMutations(mp.toByteString());
-                                    }
-                                    instance.updateIndexState(controller, builder.build(), rpcCallback);
-                                    if (controller.getFailedOn() != null) {
-                                        throw controller.getFailedOn();
-                                    }
-                                    return rpcCallback.get();
-                                }
-                            });
-            if (results.isEmpty()) {
-                throw new IOException("Didn't get expected result size");
-            }
-            MetaDataResponse tmpResponse = results.values().iterator().next();
-            MetaDataMutationResult result = MetaDataMutationResult.constructFromProto(tmpResponse);
-
+            MetaDataMutationResult result = IndexUtil.disableIndexWithTimestamp(indexTableName, minTimeStamp,
+                    systemTable, blockWriteRebuildIndex);
             if (result.getMutationCode() == MutationCode.TABLE_NOT_FOUND) {
                 LOG.info("Index " + indexTableName + " has been dropped. Ignore uncommitted mutations");
                 continue;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/83827cd8/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
index 54080d1..5142b57 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
@@ -279,6 +279,8 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData {
     public static final String LAST_STATS_UPDATE_TIME = "LAST_STATS_UPDATE_TIME";
     public static final byte[] LAST_STATS_UPDATE_TIME_BYTES = Bytes.toBytes(LAST_STATS_UPDATE_TIME);
     public static final String GUIDE_POST_KEY = "GUIDE_POST_KEY";
+    public static final String ASYNC_REBUILD_TIMESTAMP = "ASYNC_REBUILD_TIMESTAMP";
+    public static final byte[] ASYNC_REBUILD_TIMESTAMP_BYTES = Bytes.toBytes(ASYNC_REBUILD_TIMESTAMP);
 
     public static final String PARENT_TENANT_ID = "PARENT_TENANT_ID";
     public static final byte[] PARENT_TENANT_ID_BYTES = Bytes.toBytes(PARENT_TENANT_ID);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/83827cd8/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
index d57c250..f3c6d30 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
@@ -989,8 +989,8 @@ public class PhoenixStatement implements Statement, SQLCloseable {
 
     private static class ExecutableAlterIndexStatement extends AlterIndexStatement implements CompilableStatement {
 
-        public ExecutableAlterIndexStatement(NamedTableNode indexTableNode, String dataTableName, boolean ifExists, PIndexState state) {
-            super(indexTableNode, dataTableName, ifExists, state);
+        public ExecutableAlterIndexStatement(NamedTableNode indexTableNode, String dataTableName, boolean ifExists, PIndexState state, boolean async) {
+            super(indexTableNode, dataTableName, ifExists, state, async);
         }
 
         @SuppressWarnings("unchecked")
@@ -1302,8 +1302,8 @@ public class PhoenixStatement implements Statement, SQLCloseable {
         }
         
         @Override
-        public AlterIndexStatement alterIndex(NamedTableNode indexTableNode, String dataTableName, boolean ifExists, PIndexState state) {
-            return new ExecutableAlterIndexStatement(indexTableNode, dataTableName, ifExists, state);
+        public AlterIndexStatement alterIndex(NamedTableNode indexTableNode, String dataTableName, boolean ifExists, PIndexState state, boolean async) {
+            return new ExecutableAlterIndexStatement(indexTableNode, dataTableName, ifExists, state, async);
         }
 
         @Override


[21/26] phoenix git commit: PHOENIX-3613 Avoid possible SQL Injection with proper input validations(Rajeshbabu)

Posted by ma...@apache.org.
PHOENIX-3613 Avoid possible SQL Injection with proper input validations(Rajeshbabu)


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

Branch: refs/heads/calcite
Commit: 4e4f7ddba9bddec94793e92698346dedfb520c9e
Parents: e7ef25e
Author: Rajeshbabu Chintaguntla <ra...@apache.org>
Authored: Fri Jan 20 23:11:43 2017 +0530
Committer: Rajeshbabu Chintaguntla <ra...@apache.org>
Committed: Fri Jan 20 23:11:43 2017 +0530

----------------------------------------------------------------------
 .../tracingwebapp/http/EntityFactory.java       | 19 +-----------------
 .../tracingwebapp/http/TraceServlet.java        | 21 ++++++++++++++++++--
 2 files changed, 20 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/4e4f7ddb/phoenix-tracing-webapp/src/main/java/org/apache/phoenix/tracingwebapp/http/EntityFactory.java
----------------------------------------------------------------------
diff --git a/phoenix-tracing-webapp/src/main/java/org/apache/phoenix/tracingwebapp/http/EntityFactory.java b/phoenix-tracing-webapp/src/main/java/org/apache/phoenix/tracingwebapp/http/EntityFactory.java
index afb6312..a17630d 100644
--- a/phoenix-tracing-webapp/src/main/java/org/apache/phoenix/tracingwebapp/http/EntityFactory.java
+++ b/phoenix-tracing-webapp/src/main/java/org/apache/phoenix/tracingwebapp/http/EntityFactory.java
@@ -39,29 +39,12 @@ public class EntityFactory {
     this.connection = connection;
   }
 
-  public Map<String, Object> findSingle(Object[] params) throws SQLException {
-    List<Map<String, Object>> objects = this.findMultiple(params);
-
-    if (objects.size() != 1) {
-      throw new SQLException("Query did not produce one object it produced: "
-          + objects.size() + " objects.");
-    }
-
-    Map<String, Object> object = objects.get(0); // get first record;
-
-    return object;
-  }
-
-  public List<Map<String, Object>> findMultiple(Object[] params)
+  public List<Map<String, Object>> findMultiple()
       throws SQLException {
     ResultSet rs = null;
     PreparedStatement ps = null;
     try {
       ps = this.connection.prepareStatement(this.queryString);
-      for (int i = 0; i < params.length; ++i) {
-        ps.setObject(1, params[i]);
-      }
-
       rs = ps.executeQuery();
       return getEntitiesFromResultSet(rs);
     } catch (SQLException e) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4e4f7ddb/phoenix-tracing-webapp/src/main/java/org/apache/phoenix/tracingwebapp/http/TraceServlet.java
----------------------------------------------------------------------
diff --git a/phoenix-tracing-webapp/src/main/java/org/apache/phoenix/tracingwebapp/http/TraceServlet.java b/phoenix-tracing-webapp/src/main/java/org/apache/phoenix/tracingwebapp/http/TraceServlet.java
index de047ba..c20b20d 100755
--- a/phoenix-tracing-webapp/src/main/java/org/apache/phoenix/tracingwebapp/http/TraceServlet.java
+++ b/phoenix-tracing-webapp/src/main/java/org/apache/phoenix/tracingwebapp/http/TraceServlet.java
@@ -25,7 +25,7 @@ import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 
 import org.codehaus.jackson.map.ObjectMapper;
-
+import org.apache.phoenix.metrics.MetricInfo;
 import java.sql.Connection;
 import java.sql.SQLException;
 import java.util.List;
@@ -82,6 +82,11 @@ public class TraceServlet extends HttpServlet {
     if(limit == null) {
       limit = DEFAULT_LIMIT;
     }
+    try{
+        Long.parseLong(limit);
+    } catch (NumberFormatException e) {
+    	throw new RuntimeException("The LIMIT passed to the query is not a number.", e);
+    }
     String sqlQuery = "SELECT * FROM " + TRACING_TABLE + " LIMIT "+limit;
     json = getResults(sqlQuery);
     return getJson(json);
@@ -93,6 +98,8 @@ public class TraceServlet extends HttpServlet {
     if(countby == null) {
       countby = DEFAULT_COUNTBY;
     }
+    // Throws exception if the column not present in the trace table.
+    MetricInfo.getColumnName(countby.toLowerCase());
     String sqlQuery = "SELECT "+countby+", COUNT(*) AS count FROM " + TRACING_TABLE + " GROUP BY "+countby+" HAVING COUNT(*) > 1 ";
     json = getResults(sqlQuery);
     return json;
@@ -102,6 +109,16 @@ public class TraceServlet extends HttpServlet {
   protected String searchTrace(String parentId, String traceId,String logic) {
     String json = null;
     String query = null;
+    // Check the parent Id, trace id type or long or not.
+    try {
+        Long.parseLong(parentId);
+        Long.parseLong(traceId);
+    } catch (NumberFormatException e) {
+    	throw new RuntimeException("The passed parentId/traceId is not a number.", e);
+    }
+    if(!logic.equals(LOGIC_AND) || !logic.equals(LOGIC_OR)) {
+    	throw new RuntimeException("Wrong logical operator passed to the query. Only "+ LOGIC_AND+","+LOGIC_OR+" are allowed.") ;
+    }
     if(parentId != null && traceId != null) {
       query = "SELECT * FROM " + TRACING_TABLE + " WHERE parent_id="+parentId+" "+logic+" trace_id="+traceId;
     }else if (parentId != null && traceId == null) {
@@ -132,7 +149,7 @@ public class TraceServlet extends HttpServlet {
       con = ConnectionFactory.getConnection();
       EntityFactory nutrientEntityFactory = new EntityFactory(con,sqlQuery);
       List<Map<String, Object>> nutrients = nutrientEntityFactory
-          .findMultiple(new Object[] {});
+          .findMultiple();
       ObjectMapper mapper = new ObjectMapper();
       json = mapper.writeValueAsString(nutrients);
     } catch (Exception e) {


[15/26] phoenix git commit: PHOENIX-3553 Zookeeper connection should be closed immediately after DefaultStatisticsCollector's collecting stats done (Yeonseop Kim)

Posted by ma...@apache.org.
PHOENIX-3553 Zookeeper connection should be closed immediately after DefaultStatisticsCollector's collecting stats done (Yeonseop Kim)


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

Branch: refs/heads/calcite
Commit: eb72fa892ac105251e21355eb8b9fefb35e92819
Parents: f5de28b
Author: James Taylor <ja...@apache.org>
Authored: Fri Jan 13 11:12:48 2017 -0800
Committer: James Taylor <ja...@apache.org>
Committed: Fri Jan 13 11:12:48 2017 -0800

----------------------------------------------------------------------
 .../stats/DefaultStatisticsCollector.java       | 32 ++++++++++++++------
 1 file changed, 23 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/eb72fa89/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/DefaultStatisticsCollector.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/DefaultStatisticsCollector.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/DefaultStatisticsCollector.java
index 17010ac..47ee39b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/DefaultStatisticsCollector.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/DefaultStatisticsCollector.java
@@ -23,6 +23,8 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.KeyValue;
@@ -70,6 +72,7 @@ class DefaultStatisticsCollector implements StatisticsCollector {
 
     private long guidePostDepth;
     private long maxTimeStamp = MetaDataProtocol.MIN_TABLE_TIMESTAMP;
+    private static final Log LOG = LogFactory.getLog(DefaultStatisticsCollector.class);
 
     DefaultStatisticsCollector(RegionCoprocessorEnvironment env, String tableName, long clientTimeStamp, byte[] family,
             byte[] gp_width_bytes, byte[] gp_per_region_bytes) throws IOException {
@@ -119,16 +122,27 @@ class DefaultStatisticsCollector implements StatisticsCollector {
             this.guidePostDepth = StatisticsUtil.getGuidePostDepth(guidepostPerRegion, guidepostWidth,
                     env.getRegion().getTableDesc());
         } else {
-            // Next check for GUIDE_POST_WIDTH on table
-            HTableInterface htable = env.getTable(
-                    SchemaUtil.getPhysicalTableName(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES, env.getConfiguration()));
-            Get get = new Get(ptableKey);
-            get.addColumn(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES, PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH_BYTES);
-            Result result = htable.get(get);
             long guidepostWidth = -1;
-            if (!result.isEmpty()) {
-                Cell cell = result.listCells().get(0);
-                guidepostWidth = PLong.INSTANCE.getCodec().decodeLong(cell.getValueArray(), cell.getValueOffset(), SortOrder.getDefault());
+            HTableInterface htable = null;
+            try {
+                // Next check for GUIDE_POST_WIDTH on table
+                htable = env.getTable(
+                        SchemaUtil.getPhysicalTableName(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES, env.getConfiguration()));
+                Get get = new Get(ptableKey);
+                get.addColumn(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES, PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH_BYTES);
+                Result result = htable.get(get);
+                if (!result.isEmpty()) {
+                    Cell cell = result.listCells().get(0);
+                    guidepostWidth = PLong.INSTANCE.getCodec().decodeLong(cell.getValueArray(), cell.getValueOffset(), SortOrder.getDefault());
+                }
+            } finally {
+                if (htable != null) {
+                    try {
+                        htable.close();
+                    } catch (IOException e) {
+                        LOG.warn("Failed to close " + htable.getName(), e);
+                    }
+                }
             }
             if (guidepostWidth >= 0) {
                 this.guidePostDepth = guidepostWidth;


[16/26] phoenix git commit: PHOENIX-3567 Use argparse for sqlline

Posted by ma...@apache.org.
PHOENIX-3567 Use argparse for sqlline


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

Branch: refs/heads/calcite
Commit: 7332fc30c66b67d36cee7474afb04c73d01e43c3
Parents: eb72fa8
Author: Josh Elser <el...@apache.org>
Authored: Thu Jan 5 12:34:48 2017 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Mon Jan 16 16:18:08 2017 -0500

----------------------------------------------------------------------
 bin/phoenix_utils.py |  5 +++++
 bin/sqlline-thin.py  | 17 ++++++++--------
 bin/sqlline.py       | 52 ++++++++++++++++++++---------------------------
 3 files changed, 35 insertions(+), 39 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/7332fc30/bin/phoenix_utils.py
----------------------------------------------------------------------
diff --git a/bin/phoenix_utils.py b/bin/phoenix_utils.py
index 2da094f..580a78b 100755
--- a/bin/phoenix_utils.py
+++ b/bin/phoenix_utils.py
@@ -198,6 +198,11 @@ def shell_quote(args):
         import pipes
         return " ".join([pipes.quote(v) for v in args])
 
+def common_sqlline_args(parser):
+    parser.add_argument('-v', '--verbose', help='Verbosity on sqlline.', default='true')
+    parser.add_argument('-c', '--color', help='Color setting for sqlline.', default='true')
+    parser.add_argument('-fc', '--fastconnect', help='Fetch all schemas on initial connection', default='false')
+
 if __name__ == "__main__":
     setPath()
     print "phoenix_class_path:", phoenix_class_path

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7332fc30/bin/sqlline-thin.py
----------------------------------------------------------------------
diff --git a/bin/sqlline-thin.py b/bin/sqlline-thin.py
index e4cb540..47384d8 100755
--- a/bin/sqlline-thin.py
+++ b/bin/sqlline-thin.py
@@ -42,14 +42,16 @@ parser = argparse.ArgumentParser(description='Launches the Apache Phoenix Thin C
 parser.add_argument('url', nargs='?', help='The URL to the Phoenix Query Server.', default='http://localhost:8765')
 # Positional argument "sqlfile" is optional
 parser.add_argument('sqlfile', nargs='?', help='A file of SQL commands to execute.', default='')
-parser.add_argument('-u', '--user', help='Username for database authentication (unsupported).', default='none')
-parser.add_argument('-p', '--password', help='Password for database authentication (unsupported).', default='none')
+# Avatica wire authentication
 parser.add_argument('-a', '--authentication', help='Mechanism for HTTP authentication.', choices=('SPNEGO', 'BASIC', 'DIGEST', 'NONE'), default='')
+# Avatica wire serialization
 parser.add_argument('-s', '--serialization', help='Serialization type for HTTP API.', choices=('PROTOBUF', 'JSON'), default=None)
+# Avatica authentication
 parser.add_argument('-au', '--auth-user', help='Username for HTTP authentication.')
 parser.add_argument('-ap', '--auth-password', help='Password for HTTP authentication.')
-parser.add_argument('-v', '--verbose', help='Verbosity on sqlline.', default='true')
-parser.add_argument('-c', '--color', help='Color setting for sqlline.', default='true')
+# Common arguments across sqlline.py and sqlline-thin.py
+phoenix_utils.common_sqlline_args(parser)
+# Parse the args
 args=parser.parse_args()
 
 phoenix_utils.setPath()
@@ -58,9 +60,6 @@ url = args.url
 sqlfile = args.sqlfile
 serialization_key = 'phoenix.queryserver.serialization'
 
-def usage_and_exit():
-    sys.exit("usage: sqlline-thin.py [host[:port]] [sql_file]")
-
 def cleanup_url(url):
     parsed = urlparse.urlparse(url)
     if parsed.scheme == "":
@@ -161,8 +160,8 @@ java_cmd = java + ' $PHOENIX_OPTS ' + \
     os.pathsep + phoenix_utils.hadoop_conf + os.pathsep + phoenix_utils.hadoop_classpath + '" -Dlog4j.configuration=file:' + \
     os.path.join(phoenix_utils.current_dir, "log4j.properties") + \
     " org.apache.phoenix.queryserver.client.SqllineWrapper -d org.apache.phoenix.queryserver.client.Driver " + \
-    ' -u "' + jdbc_url + '"' + " -n " + args.user + " -p " + args.password + \
-    " --color=" + colorSetting + " --fastConnect=false --verbose=" + args.verbose + \
+    ' -u "' + jdbc_url + '"' + " -n none -p none " + \
+    " --color=" + colorSetting + " --fastConnect=" + args.fastconnect + " --verbose=" + args.verbose + \
     " --incremental=false --isolation=TRANSACTION_READ_COMMITTED " + sqlfile
 
 exitcode = subprocess.call(java_cmd, shell=True)

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7332fc30/bin/sqlline.py
----------------------------------------------------------------------
diff --git a/bin/sqlline.py b/bin/sqlline.py
index 474968f..7a724de 100755
--- a/bin/sqlline.py
+++ b/bin/sqlline.py
@@ -24,6 +24,7 @@ import subprocess
 import sys
 import phoenix_utils
 import atexit
+import argparse
 
 global childProc
 childProc = None
@@ -37,36 +38,26 @@ atexit.register(kill_child)
 
 phoenix_utils.setPath()
 
-def printUsage():
-    print "\nUsage: sqlline.py [zookeeper] \
-[optional_sql_file] \nExample: \n 1. sqlline.py \n \
-2. sqlline.py localhost:2181:/hbase \n 3. sqlline.py \
-localhost:2181:/hbase ../examples/stock_symbol.sql \n \
-4. sqlline.py ../examples/stock_symbol.sql"
-    sys.exit(-1)
-
-if len(sys.argv) > 3:
-    printUsage()
+parser = argparse.ArgumentParser(description='Launches the Apache Phoenix Client.')
+# Positional argument 'zookeepers' is optional
+parser.add_argument('zookeepers', nargs='?', help='The ZooKeeper quorum string', default='localhost:2181:/hbase')
+# Positional argument 'sqlfile' is optional
+parser.add_argument('sqlfile', nargs='?', help='A file of SQL commands to execute', default='')
+# Common arguments across sqlline.py and sqlline-thin.py
+phoenix_utils.common_sqlline_args(parser)
+# Parse the args
+args=parser.parse_args()
 
-sqlfile = ""
-zookeeper = ""
+zookeeper = args.zookeepers
+sqlfile = args.sqlfile
 
 # HBase configuration folder path (where hbase-site.xml reside) for
 # HBase/Phoenix client side property override
 hbase_config_path = os.getenv('HBASE_CONF_DIR', phoenix_utils.current_dir)
 
-if len(sys.argv) == 2:
-    if os.path.isfile(sys.argv[1]):
-        sqlfile = sys.argv[1]
-    else:
-        zookeeper = sys.argv[1]
-
-if len(sys.argv) == 3:
-    if os.path.isfile(sys.argv[1]):
-        printUsage()
-    else:
-        zookeeper = sys.argv[1]
-        sqlfile = sys.argv[2]
+if sqlfile and not os.path.isfile(sqlfile):
+    parser.print_help()
+    sys.exit(-1)
 
 if sqlfile:
     sqlfile = "--run=" + phoenix_utils.shell_quote([sqlfile])
@@ -101,19 +92,20 @@ if java_home:
 else:
     java = 'java'
 
-colorSetting = "true"
+colorSetting = args.color
 # disable color setting for windows OS
 if os.name == 'nt':
     colorSetting = "false"
 
 java_cmd = java + ' $PHOENIX_OPTS ' + \
-    ' -cp "' + hbase_config_path + os.pathsep + phoenix_utils.hbase_conf_dir + os.pathsep + phoenix_utils.phoenix_client_jar + os.pathsep + phoenix_utils.hadoop_common_jar + os.pathsep + phoenix_utils.hadoop_hdfs_jar + \
+    ' -cp "' + hbase_config_path + os.pathsep + phoenix_utils.hbase_conf_dir + os.pathsep + phoenix_utils.phoenix_client_jar + \
+    os.pathsep + phoenix_utils.hadoop_common_jar + os.pathsep + phoenix_utils.hadoop_hdfs_jar + \
     os.pathsep + phoenix_utils.hadoop_conf + os.pathsep + phoenix_utils.hadoop_classpath + '" -Dlog4j.configuration=file:' + \
     os.path.join(phoenix_utils.current_dir, "log4j.properties") + \
-    " sqlline.SqlLine -d org.apache.phoenix.jdbc.PhoenixDriver \
--u jdbc:phoenix:" + phoenix_utils.shell_quote([zookeeper]) + \
-    " -n none -p none --color=" + colorSetting + " --fastConnect=false --verbose=true \
---incremental=false --isolation=TRANSACTION_READ_COMMITTED " + sqlfile
+    " sqlline.SqlLine -d org.apache.phoenix.jdbc.PhoenixDriver" + \
+    " -u jdbc:phoenix:" + phoenix_utils.shell_quote([zookeeper]) + \
+    " -n none -p none --color=" + colorSetting + " --fastConnect=" + args.fastconnect + \
+    " --verbose=" + args.verbose + " --incremental=false --isolation=TRANSACTION_READ_COMMITTED " + sqlfile
 
 childProc = subprocess.Popen(java_cmd, shell=True)
 #Wait for child process exit


[04/26] phoenix git commit: PHOENIX-32451 Support DATE and TIMESTAMP in CONVERT_TZ()

Posted by ma...@apache.org.
PHOENIX-32451 Support DATE and TIMESTAMP in CONVERT_TZ()


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

Branch: refs/heads/calcite
Commit: 70dc3836ac4d472bb55e79a723a1d2a1d30d75de
Parents: e23634a
Author: Josh Elser <el...@apache.org>
Authored: Wed Nov 9 13:30:15 2016 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Sun Dec 25 21:44:03 2016 -0500

----------------------------------------------------------------------
 .../end2end/ConvertTimezoneFunctionIT.java      | 22 +++++++++++++++++++-
 .../function/ConvertTimezoneFunction.java       |  5 +++--
 2 files changed, 24 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/70dc3836/phoenix-core/src/it/java/org/apache/phoenix/end2end/ConvertTimezoneFunctionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ConvertTimezoneFunctionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ConvertTimezoneFunctionIT.java
index 229e705..a51b6c9 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ConvertTimezoneFunctionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ConvertTimezoneFunctionIT.java
@@ -39,7 +39,7 @@ import org.junit.Test;
 public class ConvertTimezoneFunctionIT extends ParallelStatsDisabledIT {
 
     @Test
-    public void testConvertTimezoneEurope() throws Exception {
+    public void testDateConvertTimezoneEurope() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
         String timezone_offset_test = generateUniqueName();
         String ddl = "CREATE TABLE IF NOT EXISTS " + timezone_offset_test
@@ -59,6 +59,26 @@ public class ConvertTimezoneFunctionIT extends ParallelStatsDisabledIT {
     }
 
     @Test
+    public void testTimestampConvertTimezoneEurope() throws Exception {
+        Connection conn = DriverManager.getConnection(getUrl());
+        String timezone_offset_test = generateUniqueName();
+        String ddl = "CREATE TABLE IF NOT EXISTS " + timezone_offset_test
+            + " (k1 INTEGER NOT NULL, timestamps TIMESTAMP CONSTRAINT pk PRIMARY KEY (k1))";
+        conn.createStatement().execute(ddl);
+        String dml = "UPSERT INTO " + timezone_offset_test
+            + " (k1, timestamps) VALUES (1, TO_TIMESTAMP('2014-03-01 00:00:00'))";
+        conn.createStatement().execute(dml);
+        conn.commit();
+
+        ResultSet rs = conn.createStatement().executeQuery(
+            "SELECT k1, timestamps, CONVERT_TZ(timestamps, 'UTC', 'Europe/Prague') FROM "
+                + timezone_offset_test);
+
+        assertTrue(rs.next());
+        assertEquals(1393635600000L, rs.getDate(3).getTime()); //Sat, 01 Mar 2014 01:00:00
+    }
+
+    @Test
     public void testConvertTimezoneAmerica() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
         String timezone_offset_test = generateUniqueName();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/70dc3836/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ConvertTimezoneFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ConvertTimezoneFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ConvertTimezoneFunction.java
index f06ddbc..8d13ab6 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ConvertTimezoneFunction.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ConvertTimezoneFunction.java
@@ -23,10 +23,11 @@ import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.phoenix.cache.JodaTimezoneCache;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.parse.FunctionParseNode;
+import org.apache.phoenix.schema.tuple.Tuple;
 import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.types.PDate;
+import org.apache.phoenix.schema.types.PTimestamp;
 import org.apache.phoenix.schema.types.PVarchar;
-import org.apache.phoenix.schema.tuple.Tuple;
 import org.joda.time.DateTimeZone;
 
 /**
@@ -35,7 +36,7 @@ import org.joda.time.DateTimeZone;
  *
  */
 @FunctionParseNode.BuiltInFunction(name = ConvertTimezoneFunction.NAME, args = {
-    @FunctionParseNode.Argument(allowedTypes = { PDate.class }),
+    @FunctionParseNode.Argument(allowedTypes = { PTimestamp.class }),
     @FunctionParseNode.Argument(allowedTypes = { PVarchar.class }),
     @FunctionParseNode.Argument(allowedTypes = { PVarchar.class })})
 public class ConvertTimezoneFunction extends ScalarFunction {


[02/26] phoenix git commit: PHOENIX-3516 Performance Issues with queries that have compound filters and specify phoenix.query.force.rowkeyorder=true

Posted by ma...@apache.org.
PHOENIX-3516 Performance Issues with queries that have compound filters and specify phoenix.query.force.rowkeyorder=true


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

Branch: refs/heads/calcite
Commit: c5046047a78e0365d75bc696dff4870304c2b5b2
Parents: bd2acd5
Author: Thomas D'Silva <td...@salesforce.com>
Authored: Tue Dec 20 17:56:37 2016 -0800
Committer: Thomas D'Silva <td...@salesforce.com>
Committed: Thu Dec 22 10:54:24 2016 -0800

----------------------------------------------------------------------
 .../end2end/TenantSpecificViewIndexIT.java      | 47 ++++++++++++++++++++
 .../apache/phoenix/compile/WhereCompiler.java   |  3 +-
 .../org/apache/phoenix/util/ExpressionUtil.java | 10 +++++
 .../phoenix/query/KeyRangeIntersectTest.java    |  9 +++-
 4 files changed, 67 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/c5046047/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificViewIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificViewIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificViewIndexIT.java
index b7b8902..6ae1445 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificViewIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificViewIndexIT.java
@@ -19,6 +19,7 @@ package org.apache.phoenix.end2end;
 
 import static org.apache.phoenix.util.MetaDataUtil.getViewIndexSequenceName;
 import static org.apache.phoenix.util.MetaDataUtil.getViewIndexSequenceSchemaName;
+import static org.apache.phoenix.util.PhoenixRuntime.TENANT_ID_ATTRIB;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
@@ -28,6 +29,7 @@ import java.sql.Connection;
 import java.sql.DriverManager;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
+import java.sql.SQLException;
 import java.util.Properties;
 
 import org.apache.hadoop.hbase.util.Bytes;
@@ -284,4 +286,49 @@ public class TenantSpecificViewIndexIT extends BaseTenantSpecificViewIndexIT {
         assertEquals("value1", rs.getString(1));
         assertFalse("No other rows should have been returned for the tenant", rs.next()); // should have just returned one record since for org1 we have only one row.
     }
+    
+    @Test
+    public void testOverlappingDatesFilter() throws SQLException {
+        String tenantUrl = getUrl() + ';' + TENANT_ID_ATTRIB + "=tenant1" + ";" + QueryServices.FORCE_ROW_KEY_ORDER_ATTRIB + "=true";
+        String tableName = generateUniqueName();
+        String viewName = generateUniqueName();
+        String ddl = "CREATE TABLE " + tableName 
+                + "(ORGANIZATION_ID CHAR(15) NOT NULL, "
+                + "PARENT_TYPE CHAR(3) NOT NULL, "
+                + "PARENT_ID CHAR(15) NOT NULL,"
+                + "CREATED_DATE DATE NOT NULL "
+                + "CONSTRAINT PK PRIMARY KEY (ORGANIZATION_ID, PARENT_TYPE, PARENT_ID, CREATED_DATE DESC)"
+                + ") VERSIONS=1,MULTI_TENANT=true,REPLICATION_SCOPE=1"; 
+                
+        try (Connection conn = DriverManager.getConnection(getUrl());
+                Connection viewConn = DriverManager.getConnection(tenantUrl) ) {
+            // create table
+            conn.createStatement().execute(ddl);
+            // create index
+            conn.createStatement().execute("CREATE INDEX IF NOT EXISTS IDX ON " + tableName + "(PARENT_TYPE, CREATED_DATE, PARENT_ID)");
+            // create view
+            viewConn.createStatement().execute("CREATE VIEW IF NOT EXISTS " + viewName + " AS SELECT * FROM "+ tableName );
+            
+            String query ="EXPLAIN SELECT PARENT_ID FROM " + viewName
+                    + " WHERE PARENT_TYPE='001' "
+                    + "AND (CREATED_DATE > to_date('2011-01-01') AND CREATED_DATE < to_date('2016-10-31'))"
+                    + "ORDER BY PARENT_TYPE,CREATED_DATE LIMIT 501";
+            
+            ResultSet rs = viewConn.createStatement().executeQuery(query);
+            String expectedPlanFormat = "CLIENT SERIAL 1-WAY RANGE SCAN OVER IDX ['tenant1        ','001','%s 00:00:00.001'] - ['tenant1        ','001','%s 00:00:00.000']" + "\n" +
+                        "    SERVER FILTER BY FIRST KEY ONLY" + "\n" +
+                        "    SERVER 501 ROW LIMIT" + "\n" +
+                        "CLIENT 501 ROW LIMIT";
+            assertEquals(String.format(expectedPlanFormat, "2011-01-01", "2016-10-31"), QueryUtil.getExplainPlan(rs));
+            
+            query ="EXPLAIN SELECT PARENT_ID FROM " + viewName
+                    + " WHERE PARENT_TYPE='001' "
+                    + " AND (CREATED_DATE >= to_date('2011-01-01') AND CREATED_DATE <= to_date('2016-01-01'))"
+                    + " AND (CREATED_DATE > to_date('2012-10-21') AND CREATED_DATE < to_date('2016-10-31')) "
+                    + "ORDER BY PARENT_TYPE,CREATED_DATE LIMIT 501";
+            
+            rs = viewConn.createStatement().executeQuery(query);
+            assertEquals(String.format(expectedPlanFormat, "2012-10-21", "2016-01-01"), QueryUtil.getExplainPlan(rs));
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c5046047/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereCompiler.java
index 13963d7..39451b8 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereCompiler.java
@@ -56,6 +56,7 @@ import org.apache.phoenix.schema.TableRef;
 import org.apache.phoenix.schema.TypeMismatchException;
 import org.apache.phoenix.schema.types.PBoolean;
 import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.ExpressionUtil;
 import org.apache.phoenix.util.ScanUtil;
 import org.apache.phoenix.util.SchemaUtil;
 
@@ -237,7 +238,7 @@ public class WhereCompiler {
 
         if (LiteralExpression.isBooleanFalseOrNull(whereClause)) {
             context.setScanRanges(ScanRanges.NOTHING);
-        } else if (whereClause != null && !LiteralExpression.isTrue(whereClause) && !hashJoinOptimization) {
+        } else if (whereClause != null && !ExpressionUtil.evaluatesToTrue(whereClause) && !hashJoinOptimization) {
             Filter filter = null;
             final Counter counter = new Counter();
             whereClause.accept(new KeyValueExpressionVisitor() {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c5046047/phoenix-core/src/main/java/org/apache/phoenix/util/ExpressionUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/ExpressionUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/ExpressionUtil.java
index 65cf075..1fbb534 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/ExpressionUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/ExpressionUtil.java
@@ -15,6 +15,7 @@ import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.phoenix.expression.Determinism;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.expression.LiteralExpression;
+import org.apache.phoenix.schema.types.PBoolean;
 import org.apache.phoenix.schema.types.PDataType;
 
 public class ExpressionUtil {
@@ -43,5 +44,14 @@ public class ExpressionUtil {
     public static LiteralExpression getNullExpression(Expression expression) throws SQLException {
         return LiteralExpression.newConstant(null, expression.getDataType(), expression.getDeterminism());
     }
+    
+    public static boolean evaluatesToTrue(Expression expression) {
+        if (isConstant(expression)) {
+            ImmutableBytesWritable ptr = new ImmutableBytesWritable();
+            expression.evaluate(null, ptr);
+            return Boolean.TRUE.equals(PBoolean.INSTANCE.toObject(ptr));
+        }
+        return false;
+    }
 
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c5046047/phoenix-core/src/test/java/org/apache/phoenix/query/KeyRangeIntersectTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/query/KeyRangeIntersectTest.java b/phoenix-core/src/test/java/org/apache/phoenix/query/KeyRangeIntersectTest.java
index 98c10a8..9e21b69 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/query/KeyRangeIntersectTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/query/KeyRangeIntersectTest.java
@@ -18,9 +18,9 @@
 
 package org.apache.phoenix.query;
 
+import static org.apache.hadoop.hbase.util.Bytes.toBytes;
 import static org.apache.phoenix.query.KeyRange.EMPTY_RANGE;
 import static org.apache.phoenix.query.KeyRange.EVERYTHING_RANGE;
-import static org.apache.hadoop.hbase.util.Bytes.toBytes;
 
 import java.util.Arrays;
 import java.util.Collection;
@@ -28,6 +28,8 @@ import java.util.Collection;
 import junit.framework.TestCase;
 
 import org.apache.phoenix.schema.types.PChar;
+import org.apache.phoenix.schema.types.PDate;
+import org.apache.phoenix.util.DateUtil;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
@@ -86,6 +88,11 @@ public class KeyRangeIntersectTest extends TestCase {
                     PChar.INSTANCE.getKeyRange(toBytes("E"), false, toBytes("F"), true),
                     EMPTY_RANGE
                 },
+                {
+                    PDate.INSTANCE.getKeyRange(PDate.INSTANCE.toBytes(DateUtil.parseDate("2011-01-01")), true, PDate.INSTANCE.toBytes(DateUtil.parseDate("2016-01-01")), true),
+                    PDate.INSTANCE.getKeyRange(PDate.INSTANCE.toBytes(DateUtil.parseDate("2012-10-21")), false, PDate.INSTANCE.toBytes(DateUtil.parseDate("2016-10-31")), false),
+                    PDate.INSTANCE.getKeyRange(PDate.INSTANCE.toBytes(DateUtil.parseDate("2012-10-21")), false, PDate.INSTANCE.toBytes(DateUtil.parseDate("2016-01-01")), true)
+                }
         });
     }
     @Test


[12/26] phoenix git commit: PHOENIX-3565 Set failIfNoTests=false on surefire and failsafe

Posted by ma...@apache.org.
PHOENIX-3565 Set failIfNoTests=false on surefire and failsafe


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

Branch: refs/heads/calcite
Commit: a164f0327e568a515ce9da2f5760c57a4d4dd188
Parents: e4c6f5c
Author: Josh Elser <el...@apache.org>
Authored: Thu Jan 5 12:48:21 2017 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Thu Jan 5 22:19:35 2017 -0500

----------------------------------------------------------------------
 pom.xml | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/a164f032/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 4a0292d..d47a6b8 100644
--- a/pom.xml
+++ b/pom.xml
@@ -119,6 +119,8 @@
     <!-- Plugin options -->
     <numForkedUT>8</numForkedUT>
     <numForkedIT>8</numForkedIT>
+    <it.failIfNoSpecifiedTests>false</it.failIfNoSpecifiedTests>
+    <surefire.failIfNoSpecifiedTests>false</surefire.failIfNoSpecifiedTests>
 
     <!-- Set default encoding so multi-byte tests work correctly on the Mac -->
     <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>


[23/26] phoenix git commit: PHOENIX-3351 Implement TODOs in PhoenixTableModify#upsert to allow writes to tenant specific tables(Rajeshbabu)

Posted by ma...@apache.org.
PHOENIX-3351 Implement TODOs in PhoenixTableModify#upsert to allow writes to tenant specific tables(Rajeshbabu)


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

Branch: refs/heads/calcite
Commit: b9323e1d30ba6b449f059b86ae7b8157de16b13d
Parents: 9b7f3ca
Author: Rajeshbabu Chintaguntla <ra...@apache.org>
Authored: Mon Jan 23 15:05:37 2017 +0530
Committer: Rajeshbabu Chintaguntla <ra...@apache.org>
Committed: Mon Jan 23 15:05:37 2017 +0530

----------------------------------------------------------------------
 .../apache/phoenix/compile/UpsertCompiler.java  | 44 ++++++++++++++------
 1 file changed, 32 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/b9323e1d/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
index 8837445..32ce6ad 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
@@ -98,6 +98,7 @@ import org.apache.phoenix.schema.TypeMismatchException;
 import org.apache.phoenix.schema.tuple.Tuple;
 import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.types.PLong;
+import org.apache.phoenix.schema.types.PSmallint;
 import org.apache.phoenix.schema.types.PTimestamp;
 import org.apache.phoenix.schema.types.PUnsignedLong;
 import org.apache.phoenix.schema.types.PVarbinary;
@@ -116,7 +117,7 @@ public class UpsertCompiler {
     private static void setValues(byte[][] values, int[] pkSlotIndex, int[] columnIndexes,
             PTable table, Map<ImmutableBytesPtr, RowMutationState> mutation,
             PhoenixStatement statement, boolean useServerTimestamp, IndexMaintainer maintainer,
-            byte[][] viewConstants, byte[] onDupKeyBytes) throws SQLException {
+            byte[][] viewConstants, byte[] onDupKeyBytes, int numSplColumns) throws SQLException {
         Map<PColumn,byte[]> columnValues = Maps.newHashMapWithExpectedSize(columnIndexes.length);
         byte[][] pkValues = new byte[table.getPKColumns().size()][];
         // If the table uses salting, the first byte is the salting byte, set to an empty array
@@ -124,10 +125,13 @@ public class UpsertCompiler {
         if (table.getBucketNum() != null) {
             pkValues[0] = new byte[] {0};
         }
+        for(int i = 0; i < numSplColumns; i++) {
+            pkValues[i] = values[i];
+        }
         Long rowTimestamp = null; // case when the table doesn't have a row timestamp column
         RowTimestampColInfo rowTsColInfo = new RowTimestampColInfo(useServerTimestamp, rowTimestamp);
-        for (int i = 0; i < values.length; i++) {
-            byte[] value = values[i];
+        for (int i = 0, j = numSplColumns; j < values.length; j++, i++) {
+            byte[] value = values[j];
             PColumn column = table.getColumns().get(columnIndexes[i]);
             if (SchemaUtil.isPKColumn(column)) {
                 pkValues[pkSlotIndex[i]] = value;
@@ -163,8 +167,8 @@ public class UpsertCompiler {
         mutation.put(ptr, new RowMutationState(columnValues, statement.getConnection().getStatementExecutionCounter(), rowTsColInfo, onDupKeyBytes));
     }
     
-    private static MutationState upsertSelect(StatementContext childContext, TableRef tableRef, RowProjector projector,
-            ResultIterator iterator, int[] columnIndexes, int[] pkSlotIndexes, boolean useServerTimestamp) throws SQLException {
+    public static MutationState upsertSelect(StatementContext childContext, TableRef tableRef, RowProjector projector,
+            ResultIterator iterator, int[] columnIndexes, int[] pkSlotIndexes, boolean useServerTimestamp, boolean prefixSysColValues) throws SQLException {
         PhoenixStatement statement = childContext.getStatement();
         PhoenixConnection connection = statement.getConnection();
         ConnectionQueryServices services = connection.getQueryServices();
@@ -172,7 +176,23 @@ public class UpsertCompiler {
                 QueryServicesOptions.DEFAULT_MAX_MUTATION_SIZE);
         int batchSize = Math.min(connection.getMutateBatchSize(), maxSize);
         boolean isAutoCommit = connection.getAutoCommit();
-        byte[][] values = new byte[columnIndexes.length][];
+        int numSplColumns =
+                (tableRef.getTable().isMultiTenant() ? 1 : 0)
+                        + (tableRef.getTable().getViewIndexId() != null ? 1 : 0);
+        byte[][] values = new byte[columnIndexes.length + numSplColumns][];
+        if(prefixSysColValues) {
+            int i = 0;
+            if(tableRef.getTable().isMultiTenant()) {
+                values[i++] = connection.getTenantId().getBytes();
+            }
+            if(tableRef.getTable().getViewIndexId() != null) {
+                values[i++] = PSmallint.INSTANCE.toBytes(tableRef.getTable().getViewIndexId());
+            }
+            
+            for(int k = 0; k <  pkSlotIndexes.length; k++) {
+                pkSlotIndexes[k] += (i + (tableRef.getTable().getBucketNum() != null ? 1 : 0));
+            }
+        }
         int rowCount = 0;
         Map<ImmutableBytesPtr, RowMutationState> mutation = Maps.newHashMapWithExpectedSize(batchSize);
         PTable table = tableRef.getTable();
@@ -192,7 +212,7 @@ public class UpsertCompiler {
         try (ResultSet rs = new PhoenixResultSet(iterator, projector, childContext)) {
             ImmutableBytesWritable ptr = new ImmutableBytesWritable();
             while (rs.next()) {
-                for (int i = 0; i < values.length; i++) {
+                for (int i = 0, j = numSplColumns; j < values.length; j++, i++) {
                     PColumn column = table.getColumns().get(columnIndexes[i]);
                     byte[] bytes = rs.getBytes(i + 1);
                     ptr.set(bytes == null ? ByteUtil.EMPTY_BYTE_ARRAY : bytes);
@@ -212,9 +232,9 @@ public class UpsertCompiler {
                             precision, scale, SortOrder.getDefault(), 
                             column.getMaxLength(), column.getScale(), column.getSortOrder(),
                             table.rowKeyOrderOptimizable());
-                    values[i] = ByteUtil.copyKeyBytesIfNecessary(ptr);
+                    values[j] = ByteUtil.copyKeyBytesIfNecessary(ptr);
                 }
-                setValues(values, pkSlotIndexes, columnIndexes, table, mutation, statement, useServerTimestamp, indexMaintainer, viewConstants, null);
+                setValues(values, pkSlotIndexes, columnIndexes, table, mutation, statement, useServerTimestamp, indexMaintainer, viewConstants, null, numSplColumns);
                 rowCount++;
                 // Commit a batch if auto commit is true and we're at our batch size
                 if (isAutoCommit && rowCount % batchSize == 0) {
@@ -256,7 +276,7 @@ public class UpsertCompiler {
             StatementContext childContext = new StatementContext(statement, false);
             // Clone the row projector as it's not thread safe and would be used simultaneously by
             // multiple threads otherwise.
-            MutationState state = upsertSelect(childContext, tableRef, projector.cloneIfNecessary(), iterator, columnIndexes, pkSlotIndexes, useSeverTimestamp);
+            MutationState state = upsertSelect(childContext, tableRef, projector.cloneIfNecessary(), iterator, columnIndexes, pkSlotIndexes, useSeverTimestamp, false);
             return state;
         }
         
@@ -798,7 +818,7 @@ public class UpsertCompiler {
                 public MutationState execute() throws SQLException {
                     ResultIterator iterator = queryPlan.iterator();
                     if (parallelIteratorFactory == null) {
-                        return upsertSelect(new StatementContext(statement), tableRef, projector, iterator, columnIndexes, pkSlotIndexes, useServerTimestamp);
+                        return upsertSelect(new StatementContext(statement), tableRef, projector, iterator, columnIndexes, pkSlotIndexes, useServerTimestamp, false);
                     }
                     try {
                         parallelIteratorFactory.setRowProjector(projector);
@@ -1043,7 +1063,7 @@ public class UpsertCompiler {
                     indexMaintainer = table.getIndexMaintainer(parentTable, connection);
                     viewConstants = IndexUtil.getViewConstants(parentTable);
                 }
-                setValues(values, pkSlotIndexes, columnIndexes, table, mutation, statement, useServerTimestamp, indexMaintainer, viewConstants, onDupKeyBytes);
+                setValues(values, pkSlotIndexes, columnIndexes, table, mutation, statement, useServerTimestamp, indexMaintainer, viewConstants, onDupKeyBytes, 0);
                 return new MutationState(tableRef, mutation, 0, maxSize, connection);
             }