You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by pb...@apache.org on 2019/05/28 22:52:50 UTC

[phoenix] branch 4.x-cdh5.16 created (now b129be9)

This is an automated email from the ASF dual-hosted git repository.

pboado pushed a change to branch 4.x-cdh5.16
in repository https://gitbox.apache.org/repos/asf/phoenix.git.


      at b129be9  PHOENIX-5059 Use the Datasource v2 api in the spark connector

This branch includes the following new commits:

     new de4e041  PHOENIX-4781 Create artifact jar so that shaded jar replaces it properly
     new c360f87  ScanningResultIterator metric RowsScanned not set. PHOENIX-5051
     new 1d9073c  PHOENIX-4832: Add Canary Test Tool for Phoenix Query Server.
     new 622fcf4  PHOENIX-4763: Changing a base table property value should be reflected in child views (if the property wasn't changed)
     new dff179b  PHOENIX-5025 Tool to clean up orphan views
     new 4de622a  PHOENIX-4983: Allow using a connection with a SCN set to write data to tables EXCEPT transactional tables or mutable tables with indexes or tables with ROW_TIMESTAMP column.
     new 4db9a6f  PHOENIX-5048 Index Rebuilder does not handle INDEX_STATE timestamp check for all index
     new f530f94  PHOENIX-4983: Added missing apache license header.
     new bd4f52f  PHOENIX-5025 Tool to clean up orphan views (addendum)
     new 9c7ee72  PHOENIX-5070 NPE when upgrading Phoenix 4.13.0 to Phoenix 4.14.1 with hbase-1.x branch in secure setup
     new 81f8503  [PHOENIX-3623] Integrate Omid with Phoenix.
     new 3f17a89  PHOENIX-5074 DropTableWithViewsIT.testDropTableWithChildViews is flapping
     new 82172a1  PHOENIX-5074; fix compilation failure.
     new 5873214  PHOENIX-5084 Changes from Transactional Tables are not visible to query in different client.
     new 460da61  PHOENIX-4820 Optimize OrderBy for ClientAggregatePlan
     new 0e5a263  PHOENIX-5055 Split mutations batches probably affects correctness of index data
     new cea84e8  Changes for CDH 5.16.x
     new b129be9  PHOENIX-5059 Use the Datasource v2 api in the spark connector

The 18 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.



[phoenix] 09/18: PHOENIX-5025 Tool to clean up orphan views (addendum)

Posted by pb...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

pboado pushed a commit to branch 4.x-cdh5.16
in repository https://gitbox.apache.org/repos/asf/phoenix.git

commit bd4f52f576d11b779a82d89dd20354188adaf850
Author: Kadir <ko...@salesforce.com>
AuthorDate: Thu Dec 13 01:53:38 2018 +0000

    PHOENIX-5025 Tool to clean up orphan views (addendum)
---
 .../apache/phoenix/end2end/OrphanViewToolIT.java   | 23 +++++++++++++--------
 .../apache/phoenix/mapreduce/OrphanViewTool.java   | 24 ++++++++++++----------
 2 files changed, 28 insertions(+), 19 deletions(-)

diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/OrphanViewToolIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/OrphanViewToolIT.java
index f9a1785..ab78ecd 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/OrphanViewToolIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/OrphanViewToolIT.java
@@ -27,9 +27,9 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
 import java.io.File;
+import java.io.FileReader;
 import java.io.IOException;
-import java.nio.file.Files;
-import java.nio.file.Paths;
+import java.io.LineNumberReader;
 import java.sql.Connection;
 import java.sql.DriverManager;
 import java.sql.ResultSet;
@@ -206,9 +206,13 @@ public class OrphanViewToolIT extends ParallelStatsDisabledIT {
     }
 
     private void verifyLineCount(String fileName, long lineCount) throws IOException {
-        if (Files.lines(Paths.get(fileName)).count() != lineCount)
-            LOG.debug(Files.lines(Paths.get(fileName)).count() + " != " + lineCount);
-        assertTrue(Files.lines(Paths.get(fileName)).count() == lineCount);
+        LineNumberReader reader = new LineNumberReader(new FileReader(fileName));
+        while (reader.readLine() != null) {
+        }
+        int count = reader.getLineNumber();
+        if (count != lineCount)
+            LOG.debug(count + " != " + lineCount);
+        assertTrue(count == lineCount);
     }
 
     private void verifyCountQuery(Connection connection, String query, String schemaName, long count)
@@ -238,7 +242,6 @@ public class OrphanViewToolIT extends ParallelStatsDisabledIT {
         }
     }
 
-
     private void verifyNoChildLink(Connection connection, String viewSchemaName) throws Exception {
         // Verify that there there is no link in the system child link table
         verifyCountQuery(connection, countChildLinksQuery, viewSchemaName, 0);
@@ -264,6 +267,7 @@ public class OrphanViewToolIT extends ParallelStatsDisabledIT {
                 schemaName == null ? "IS NULL" : "= '" + schemaName + "'"));
         connection.commit();
     }
+
     @Test
     public void testDeleteBaseTableRows() throws Exception {
         String baseTableName = generateUniqueName();
@@ -438,7 +442,8 @@ public class OrphanViewToolIT extends ParallelStatsDisabledIT {
         }
     }
 
-    public static String[] getArgValues(boolean clean, boolean identify, boolean outputPath, boolean inputPath) {
+    public static String[] getArgValues(boolean clean, boolean identify, boolean outputPath, boolean inputPath)
+            throws InterruptedException{
         final List<String> args = Lists.newArrayList();
         if (outputPath) {
             args.add("-op");
@@ -454,8 +459,10 @@ public class OrphanViewToolIT extends ParallelStatsDisabledIT {
         if (identify) {
             args.add("-i");
         }
+        final long ageMs = 2000;
+        Thread.sleep(ageMs);
         args.add("-a");
-        args.add("0");
+        args.add(Long.toString(ageMs));
         return args.toArray(new String[0]);
     }
 
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/OrphanViewTool.java b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/OrphanViewTool.java
index a8a30b6..713fb05 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/OrphanViewTool.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/OrphanViewTool.java
@@ -812,17 +812,6 @@ public class OrphanViewTool extends Configured implements Tool {
             } catch (IllegalStateException e) {
                 printHelpAndExit(e.getMessage(), getOptions());
             }
-
-            Properties props = new Properties();
-            long scn = System.currentTimeMillis() - ageMs;
-            props.setProperty("CurrentSCN", Long.toString(scn));
-            connection = ConnectionUtil.getInputConnection(configuration);
-            PhoenixConnection phoenixConnection = connection.unwrap(PhoenixConnection.class);
-
-            if (clean) {
-                // Take a snapshot of system tables to be modified
-                createSnapshot(phoenixConnection, scn);
-            }
             if (outputPath != null) {
                 // Create files to log orphan views and links
                 for (int i = VIEW; i < ORPHAN_TYPE_COUNT; i++) {
@@ -834,7 +823,20 @@ public class OrphanViewTool extends Configured implements Tool {
                     writer[i] = new BufferedWriter(new FileWriter(file));
                 }
             }
+            Properties props = new Properties();
+            long scn = System.currentTimeMillis() - ageMs;
+            props.setProperty("CurrentSCN", Long.toString(scn));
+            connection = ConnectionUtil.getInputConnection(configuration, props);
+            PhoenixConnection phoenixConnection = connection.unwrap(PhoenixConnection.class);
             identifyOrphanViews(phoenixConnection);
+            if (clean) {
+                // Close the connection with SCN
+                phoenixConnection.close();
+                connection = ConnectionUtil.getInputConnection(configuration);
+                phoenixConnection = connection.unwrap(PhoenixConnection.class);
+                // Take a snapshot of system tables to be modified
+                createSnapshot(phoenixConnection, scn);
+            }
             for (Map.Entry<Key, View> entry : orphanViewSet.entrySet()) {
                 try {
                     dropOrLogOrphanViews(phoenixConnection, configuration, entry.getKey());


[phoenix] 01/18: PHOENIX-4781 Create artifact jar so that shaded jar replaces it properly

Posted by pb...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

pboado pushed a commit to branch 4.x-cdh5.16
in repository https://gitbox.apache.org/repos/asf/phoenix.git

commit de4e0416017ae27f78f9cb1bf98f09b88d844cfb
Author: Vincent Poon <vi...@apache.org>
AuthorDate: Sat Dec 1 01:55:34 2018 +0000

    PHOENIX-4781 Create artifact jar so that shaded jar replaces it properly
---
 phoenix-client/pom.xml | 9 +++------
 phoenix-server/pom.xml | 9 +++------
 2 files changed, 6 insertions(+), 12 deletions(-)

diff --git a/phoenix-client/pom.xml b/phoenix-client/pom.xml
index 83c7ad9..cfed3ce 100644
--- a/phoenix-client/pom.xml
+++ b/phoenix-client/pom.xml
@@ -57,12 +57,9 @@
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-jar-plugin</artifactId>
-        <executions>
-          <execution>
-            <id>default-jar</id>
-            <phase>none</phase>
-          </execution>
-        </executions>
+        <configuration>
+          <finalName>phoenix-${project.version}-client</finalName>
+        </configuration>
       </plugin>
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
diff --git a/phoenix-server/pom.xml b/phoenix-server/pom.xml
index 648e4d1..e6a7afe 100644
--- a/phoenix-server/pom.xml
+++ b/phoenix-server/pom.xml
@@ -61,12 +61,9 @@
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-jar-plugin</artifactId>
-        <executions>
-          <execution>
-            <id>default-jar</id>
-            <phase>none</phase>
-          </execution>
-        </executions>
+        <configuration>
+          <finalName>phoenix-${project.version}-server</finalName>
+        </configuration>
       </plugin>
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>


[phoenix] 10/18: PHOENIX-5070 NPE when upgrading Phoenix 4.13.0 to Phoenix 4.14.1 with hbase-1.x branch in secure setup

Posted by pb...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

pboado pushed a commit to branch 4.x-cdh5.16
in repository https://gitbox.apache.org/repos/asf/phoenix.git

commit 9c7ee727aacce0b5d428160ed167345f8febf369
Author: Monani Mihir <mo...@gmail.com>
AuthorDate: Fri Dec 14 10:50:17 2018 +0000

    PHOENIX-5070 NPE when upgrading Phoenix 4.13.0 to Phoenix 4.14.1 with hbase-1.x branch in secure setup
---
 .../java/org/apache/phoenix/coprocessor/PhoenixAccessController.java    | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/PhoenixAccessController.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/PhoenixAccessController.java
index 62c158c..ef26d2c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/PhoenixAccessController.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/PhoenixAccessController.java
@@ -406,7 +406,7 @@ public class PhoenixAccessController extends BaseMetaDataEndpointObserver {
                 final List<UserPermission> userPermissions = new ArrayList<UserPermission>();
                 try (Connection connection = ConnectionFactory.createConnection(env.getConfiguration())) {
                     // Merge permissions from all accessController coprocessors loaded in memory
-                    for (BaseMasterAndRegionObserver service : accessControllers) {
+                    for (BaseMasterAndRegionObserver service : getAccessControllers()) {
                         // Use AccessControlClient API's if the accessController is an instance of org.apache.hadoop.hbase.security.access.AccessController
                         if (service.getClass().getName().equals(org.apache.hadoop.hbase.security.access.AccessController.class.getName())) {
                             userPermissions.addAll(AccessControlClient.getUserPermissions(connection, tableName.getNameAsString()));


[phoenix] 11/18: [PHOENIX-3623] Integrate Omid with Phoenix.

Posted by pb...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

pboado pushed a commit to branch 4.x-cdh5.16
in repository https://gitbox.apache.org/repos/asf/phoenix.git

commit 81f850311c4b03cb900a9dea079ee311d9c685fd
Author: Ohad Shacham <oh...@yahoo-inc.com>
AuthorDate: Thu Dec 20 12:15:03 2018 +0000

    [PHOENIX-3623] Integrate Omid with Phoenix.
    
    This commit finishes the integration of Omid as Phoenix transaction processor engine.
    More information regarding the integration exists at [PHOENIX-3623] and at [OMID-82], which is the corresponding jira in Omid.
---
 bin/omid-env.sh                                    |  43 ++++
 bin/omid-server-configuration.yml                  |  25 +++
 bin/omid.sh                                        |  93 +++++++++
 phoenix-assembly/pom.xml                           |   5 +
 .../build/components/all-common-dependencies.xml   |  28 +++
 phoenix-core/pom.xml                               |  46 +++++
 .../phoenix/coprocessor/OmidGCProcessor.java       |   7 +-
 .../coprocessor/OmidTransactionalProcessor.java    |   8 +-
 .../transaction/OmidTransactionContext.java        | 217 ++++++++++++++++++++-
 .../transaction/OmidTransactionProvider.java       | 106 +++++++++-
 .../phoenix/transaction/OmidTransactionTable.java  |  64 +++++-
 .../phoenix/transaction/TransactionFactory.java    |   5 +-
 .../phoenix/query/QueryServicesTestImpl.java       |   1 -
 phoenix-server/pom.xml                             |   1 +
 pom.xml                                            |  47 +++++
 15 files changed, 665 insertions(+), 31 deletions(-)

diff --git a/bin/omid-env.sh b/bin/omid-env.sh
new file mode 100644
index 0000000..820cdaa
--- /dev/null
+++ b/bin/omid-env.sh
@@ -0,0 +1,43 @@
+#
+# 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.
+#
+
+# Set the flags to pass to the jvm when running omid
+# export JVM_FLAGS=-Xmx8096m
+# ---------------------------------------------------------------------------------------------------------------------
+# Check if HADOOP_CONF_DIR and HBASE_CONF_DIR are set
+# ---------------------------------------------------------------------------------------------------------------------
+export JVM_FLAGS=-Xmx4096m
+if [ -z ${HADOOP_CONF_DIR+x} ]; then
+    if [ -z ${HADOOP_HOME+x} ]; then
+        echo "WARNING: HADOOP_HOME or HADOOP_CONF_DIR are unset";
+    else
+        export HADOOP_CONF_DIR=${HADOOP_HOME}/conf
+    fi
+else
+    echo "HADOOP_CONF_DIR is set to '$HADOOP_CONF_DIR'";
+fi
+
+if [ -z ${HBASE_CONF_DIR+x} ]; then
+    if [ -z ${HBASE_HOME+x} ]; then
+        echo "WARNING: HBASE_HOME or HBASE_CONF_DIR are unset";
+    else
+        export HBASE_CONF_DIR=${HBASE_HOME}/conf
+    fi
+else
+    echo "HBASE_CONF_DIR is set to '$HBASE_CONF_DIR'";
+fi
diff --git a/bin/omid-server-configuration.yml b/bin/omid-server-configuration.yml
new file mode 100644
index 0000000..8d1616e
--- /dev/null
+++ b/bin/omid-server-configuration.yml
@@ -0,0 +1,25 @@
+# =====================================================================================================================
+#
+# Omid TSO Server Configuration
+# ---------------------------------------------------------------------------------------------------------------------
+#
+# Tune here the default values for TSO server config parameters found in 'default-omid-server-configuration.yml' file
+#
+# =====================================================================================================================
+
+
+timestampStoreModule: !!org.apache.omid.timestamp.storage.HBaseTimestampStorageModule [ ]
+commitTableStoreModule: !!org.apache.omid.committable.hbase.HBaseCommitTableStorageModule [ ]
+
+metrics: !!org.apache.omid.metrics.CodahaleMetricsProvider [
+!!org.apache.omid.metrics.CodahaleMetricsConfig {
+  outputFreqInSecs: 10,
+  reporters: !!set {
+    !!org.apache.omid.metrics.CodahaleMetricsConfig$Reporter CSV
+  },
+  csvDir: "csvMetrics",
+}
+]
+
+timestampType: WORLD_TIME
+lowLatency: false
diff --git a/bin/omid.sh b/bin/omid.sh
new file mode 100755
index 0000000..5b33ed5
--- /dev/null
+++ b/bin/omid.sh
@@ -0,0 +1,93 @@
+#!/bin/bash
+#
+# 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.
+#
+
+SCRIPTDIR=`dirname $0`
+cd $SCRIPTDIR;
+
+# Load Omid environment variables
+source omid-env.sh
+
+# Configure classpath...
+CLASSPATH=./:../conf:${HBASE_CONF_DIR}:${HADOOP_CONF_DIR}
+
+# ...for source release and...
+for j in ../target/omid-tso*.jar; do
+    CLASSPATH=$CLASSPATH:$j
+done
+
+# and for binary release
+for j in ../omid-tso*.jar; do
+    CLASSPATH=$CLASSPATH:$j
+done
+for j in ../lib/*.jar; do
+    CLASSPATH=$CLASSPATH:$j
+done
+
+tso() {
+    exec java $JVM_FLAGS -cp $CLASSPATH org.apache.omid.tso.TSOServer $@
+}
+
+tsoRelauncher() {
+    until ./omid.sh tso $@; do
+        echo "TSO Server crashed with exit code $?.  Re-launching..." >&2
+        sleep 1
+    done
+}
+
+createHBaseCommitTable() {
+    java -cp $CLASSPATH org.apache.omid.tools.hbase.OmidTableManager commit-table $@
+}
+
+createHBaseTimestampTable() {
+    java -cp $CLASSPATH org.apache.omid.tools.hbase.OmidTableManager timestamp-table $@
+}
+
+usage() {
+    echo "Usage: omid.sh <command> <options>"
+    echo "where <command> is one of:"
+    echo "  tso                           Starts The Status Oracle server (TSO)"
+    echo "  tso-relauncher                Starts The Status Oracle server (TSO) re-launching it if the process exits"
+    echo "  create-hbase-commit-table     Creates the hbase commit table."
+    echo "  create-hbase-timestamp-table  Creates the hbase timestamp table."
+}
+
+# if no args specified, show usage
+if [ $# = 0 ]; then
+    usage;
+    exit 1
+fi
+
+COMMAND=$1
+shift
+
+if [ "$COMMAND" = "tso" ]; then
+    createHBaseTimestampTable $@;
+    createHBaseCommitTable $@;
+    tso $@;
+elif [ "$COMMAND" = "tso-relauncher" ]; then
+    tsoRelauncher $@;
+elif [ "$COMMAND" = "create-hbase-commit-table" ]; then
+    createHBaseCommitTable $@;
+elif [ "$COMMAND" = "create-hbase-timestamp-table" ]; then
+    createHBaseTimestampTable $@;
+else
+    exec java -cp $CLASSPATH $COMMAND $@
+fi
+
+
diff --git a/phoenix-assembly/pom.xml b/phoenix-assembly/pom.xml
index 61da2b3..300b4f6 100644
--- a/phoenix-assembly/pom.xml
+++ b/phoenix-assembly/pom.xml
@@ -130,5 +130,10 @@
       <groupId>org.apache.phoenix</groupId>
       <artifactId>phoenix-load-balancer</artifactId>
     </dependency>
+    <dependency>
+      <groupId>org.apache.omid</groupId>
+      <artifactId>omid-hbase-tools-hbase1.x</artifactId>
+      <version>${omid.version}</version>
+    </dependency>
   </dependencies>
 </project>
diff --git a/phoenix-assembly/src/build/components/all-common-dependencies.xml b/phoenix-assembly/src/build/components/all-common-dependencies.xml
index 6bde495..797fabb 100644
--- a/phoenix-assembly/src/build/components/all-common-dependencies.xml
+++ b/phoenix-assembly/src/build/components/all-common-dependencies.xml
@@ -45,6 +45,34 @@
         <include>io.netty:netty</include>
         <include>commons-codec:commons-codec</include>
         <include>org.apache.calcite:calcite-avatica*</include>
+
+        <!-- For omid TSO -->
+        <include>org.apache.omid:omid-tso-server-hbase1.x</include>
+        <include>org.apache.omid:omid-hbase-common-hbase1.x</include>
+        <include>org.apache.omid:omid-hbase-tools-hbase1.x</include>
+        <include>org.apache.omid:omid-common</include>
+        <include>org.apache.omid:omid-metrics</include>
+        <include>org.apache.omid:omid-timestamp-storage-hbase1.x</include>
+        <include>org.apache.omid:omid-hbase-shims-hbase1.x</include>
+        <include>org.apache.omid:omid-commit-table</include>
+        <include>org.apache.omid:omid-codahale-metrics</include>
+        <include>org.apache.omid:omid-hbase-commit-table-hbase1.x</include>
+        <include>org.yaml:snakeyaml</include>
+        <include>com.google.inject:guice</include>
+        <include>commons-beanutils:commons-beanutils</include>
+        <include>javax.inject:javax.inject</include>
+        <include>aopalliance:aopalliance</include>
+        <include>org.apache.commons:commons-pool2</include>
+        <include>com.lmax:disruptor</include>
+        <include>com.codahale.metrics:metrics-core</include>
+        <include>com.beust:jcommander</include>
+        <include>commons-collections:commons-collections</include>
+        <include>io.netty:netty-all</include>
+        <include>org.apache.htrace:htrace-core</include>
+        <include>javax.servlet:javax.servlet-api</include>
+        <include>commons-cli:commons-cli</include>
+        <include>com.yammer.metrics:metrics-core</include>
+        <include>com.codahale.metrics:metrics-graphite</include>
       </includes>
     </dependencySet>
     <!-- Separate dependency set to just pull in the jackson stuff since its test 
diff --git a/phoenix-core/pom.xml b/phoenix-core/pom.xml
index f8112fe..8caf88f 100644
--- a/phoenix-core/pom.xml
+++ b/phoenix-core/pom.xml
@@ -195,6 +195,52 @@
   </build>
 
   <dependencies>
+  <dependency>
+    <groupId>org.apache.omid</groupId>
+    <artifactId>omid-hbase-client-hbase1.x</artifactId>
+    <version>${omid.version}</version>
+    <exclusions>
+      <exclusion>
+        <groupId>org.testng</groupId>
+        <artifactId>testng</artifactId>
+      </exclusion>
+    </exclusions>
+  </dependency>
+  <dependency>
+    <groupId>org.apache.omid</groupId>
+    <artifactId>omid-hbase-coprocessor-hbase1.x</artifactId>
+    <version>${omid.version}</version>
+    <exclusions>
+      <exclusion>
+        <groupId>org.testng</groupId>
+        <artifactId>testng</artifactId>
+      </exclusion>
+    </exclusions>
+  </dependency>
+  <dependency>
+    <groupId>org.apache.omid</groupId>
+    <artifactId>omid-tso-server-hbase1.x</artifactId>
+    <version>${omid.version}</version>
+    <exclusions>
+      <exclusion>
+        <groupId>org.testng</groupId>
+        <artifactId>testng</artifactId>
+      </exclusion>
+    </exclusions>
+  </dependency>
+  <dependency>
+    <groupId>org.apache.omid</groupId>
+    <artifactId>omid-tso-server-hbase1.x</artifactId>
+    <version>${omid.version}</version>
+    <type>test-jar</type>
+    <exclusions>
+      <exclusion>
+        <groupId>org.testng</groupId>
+        <artifactId>testng</artifactId>
+      </exclusion>
+    </exclusions>
+  </dependency>
+
     <!-- Transaction dependencies -->
     <dependency>
       <groupId>org.apache.tephra</groupId>
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/OmidGCProcessor.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/OmidGCProcessor.java
index d90a31e..b4a1a0f 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/OmidGCProcessor.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/OmidGCProcessor.java
@@ -17,14 +17,15 @@
  */
 package org.apache.phoenix.coprocessor;
 
-import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
+import org.apache.omid.committable.CommitTable;
+import org.apache.omid.transaction.OmidCompactor;
+import org.apache.omid.transaction.OmidSnapshotFilter;
 
 
 public class OmidGCProcessor extends DelegateRegionObserver {
 
     public OmidGCProcessor() {
-        super(new BaseRegionObserver() {
-        });
+        super(new OmidCompactor(true));
     }
 
 }
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/OmidTransactionalProcessor.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/OmidTransactionalProcessor.java
index f977cfe..b84b5ae 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/OmidTransactionalProcessor.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/OmidTransactionalProcessor.java
@@ -17,14 +17,16 @@
  */
 package org.apache.phoenix.coprocessor;
 
-import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
+import org.apache.omid.transaction.OmidSnapshotFilter;
+import org.apache.phoenix.transaction.OmidTransactionProvider;
 
 
 public class OmidTransactionalProcessor extends DelegateRegionObserver {
 
     public OmidTransactionalProcessor() {
-        super(new BaseRegionObserver() {
-        });
+        // Hack for testing - retrieves the commit table client from the singleton OmidTransactionProvider
+        // TODO: use real commit table and get port from config
+        super(new OmidSnapshotFilter(OmidTransactionProvider.getInstance().getCommitTableClient()));
     }
 
 }
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/transaction/OmidTransactionContext.java b/phoenix-core/src/main/java/org/apache/phoenix/transaction/OmidTransactionContext.java
index 9edc58b..392de78 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/transaction/OmidTransactionContext.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/transaction/OmidTransactionContext.java
@@ -18,89 +18,276 @@
 package org.apache.phoenix.transaction;
 
 import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Set;
 
 import org.apache.hadoop.hbase.client.Table;
+import org.apache.omid.proto.TSOProto;
+import org.apache.omid.transaction.AbstractTransaction.VisibilityLevel;
+import org.apache.omid.transaction.HBaseCellId;
+import org.apache.omid.transaction.HBaseTransaction;
+import org.apache.omid.transaction.HBaseTransactionManager;
+import org.apache.omid.transaction.RollbackException;
+import org.apache.omid.transaction.Transaction;
+import org.apache.omid.transaction.Transaction.Status;
+import org.apache.omid.transaction.TransactionException;
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.transaction.TransactionFactory.Provider;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.protobuf.InvalidProtocolBufferException;
 
 import com.google.protobuf.InvalidProtocolBufferException;
-//import org.apache.omid.tso.TSOMockModule;
 
 public class OmidTransactionContext implements PhoenixTransactionContext {
 
+    private static final Logger logger = LoggerFactory.getLogger(OmidTransactionContext.class);
+
+    private HBaseTransactionManager tm;
+    private HBaseTransaction tx;
+
     public OmidTransactionContext() {
+        this.tx = null;
+        this.tm = null;
     }
 
     public OmidTransactionContext(PhoenixConnection connection) throws SQLException {
+        PhoenixTransactionClient client = connection.getQueryServices().initTransactionClient(getProvider());
+        assert (client instanceof OmidTransactionProvider.OmidTransactionClient);
+        this.tm = ((OmidTransactionProvider.OmidTransactionClient)client).getTransactionClient();
+        this.tx = null;
     }
 
     public OmidTransactionContext(byte[] txnBytes) throws InvalidProtocolBufferException {
+        this();
+        if (txnBytes != null && txnBytes.length > 0) {
+            TSOProto.Transaction transaction = TSOProto.Transaction.parseFrom(txnBytes);
+            tx = new HBaseTransaction(transaction.getTimestamp(), transaction.getEpoch(), new HashSet<HBaseCellId>(),
+                    new HashSet<HBaseCellId>(), null, tm.isLowLatency());
+        } else {
+            tx = null;
+        }
     }
 
     public OmidTransactionContext(PhoenixTransactionContext ctx, boolean subTask) {
+        assert (ctx instanceof OmidTransactionContext);
+        OmidTransactionContext omidTransactionContext = (OmidTransactionContext) ctx;
+
+        this.tm = omidTransactionContext.tm;
+        
+        if (subTask) {
+            if (omidTransactionContext.isTransactionRunning()) {
+                Transaction transaction = omidTransactionContext.getTransaction();
+                this.tx = new HBaseTransaction(transaction.getTransactionId(), transaction.getEpoch(),
+                        new HashSet<HBaseCellId>(), new HashSet<HBaseCellId>(), this.tm,
+                        transaction.getReadTimestamp(), transaction.getWriteTimestamp(), tm.isLowLatency());
+            } else {
+                this.tx = null;
+            }
+
+            this.tm = null;
+        } else {
+            this.tx = omidTransactionContext.getTransaction();
+        }
     }
 
     @Override
     public void begin() throws SQLException {
+        if (tm == null) {
+            throw new SQLExceptionInfo.Builder(
+                    SQLExceptionCode.NULL_TRANSACTION_CONTEXT).build()
+                    .buildException();
+        }
+
+
+        try {
+            tx = (HBaseTransaction) tm.begin();
+        } catch (TransactionException e) {
+            throw new SQLExceptionInfo.Builder(
+                    SQLExceptionCode.TRANSACTION_FAILED)
+                    .setMessage(e.getMessage()).setRootCause(e).build()
+                    .buildException();
+        }
     }
 
     @Override
     public void commit() throws SQLException {
+        if (tx == null || tm == null)
+            return;
+
+        try {
+            tm.commit(tx);
+        } catch (TransactionException e) {
+            throw new SQLExceptionInfo.Builder(
+                    SQLExceptionCode.TRANSACTION_FAILED)
+                    .setMessage(e.getMessage()).setRootCause(e).build()
+                    .buildException();
+        } catch (RollbackException e) {
+            throw new SQLExceptionInfo.Builder(
+                    SQLExceptionCode.TRANSACTION_CONFLICT_EXCEPTION)
+                    .setMessage(e.getMessage()).setRootCause(e).build()
+                    .buildException();
+        }
     }
 
     @Override
     public void abort() throws SQLException {
+        if (tx == null || tm == null || tx.getStatus() != Status.RUNNING) {
+            return;
+        }
+
+        try {
+            tm.rollback(tx);
+        } catch (TransactionException e) {
+            throw new SQLExceptionInfo.Builder(
+                    SQLExceptionCode.TRANSACTION_FAILED)
+                    .setMessage(e.getMessage()).setRootCause(e).build()
+                    .buildException();
+        }
     }
 
     @Override
     public void checkpoint(boolean hasUncommittedData) throws SQLException {
+        if (hasUncommittedData) {
+            try {
+                tx.checkpoint();
+            } catch (TransactionException e) {
+                throw new SQLException(e);
+            }
+        }
+        tx.setVisibilityLevel(VisibilityLevel.SNAPSHOT_EXCLUDE_CURRENT);
     }
 
     @Override
     public void commitDDLFence(PTable dataTable) throws SQLException {
+
+        try {
+            tx = (HBaseTransaction) tm.fence(dataTable.getName().getBytes());
+            if (logger.isInfoEnabled()) {
+                logger.info("Added write fence at ~"
+                        + tx.getReadTimestamp());
+            }
+        } catch (TransactionException e) {
+            throw new SQLExceptionInfo.Builder(
+                    SQLExceptionCode.TX_UNABLE_TO_GET_WRITE_FENCE)
+            .setSchemaName(dataTable.getSchemaName().getString())
+            .setTableName(dataTable.getTableName().getString()).build()
+            .buildException();
+        }
     }
 
     @Override
     public void join(PhoenixTransactionContext ctx) {
+
+        if (ctx == PhoenixTransactionContext.NULL_CONTEXT) {
+            return;
+        }
+
+        assert (ctx instanceof OmidTransactionContext);
+        OmidTransactionContext omidContext = (OmidTransactionContext) ctx;
+        
+        HBaseTransaction transaction = omidContext.getTransaction();
+        if (transaction == null || tx == null) return;
+
+        Set<HBaseCellId> writeSet = transaction.getWriteSet();
+
+        for (HBaseCellId cell : writeSet) {
+            tx.addWriteSetElement(cell);
+        }
     }
 
     @Override
     public boolean isTransactionRunning() {
-        return false;
+        return (tx != null);
     }
 
     @Override
     public void reset() {
+        tx = null;
     }
 
     @Override
     public long getTransactionId() {
-        return 0;
+        return tx.getTransactionId();
     }
 
     @Override
     public long getReadPointer() {
-        return 0;
+        return tx.getReadTimestamp();
     }
 
     @Override
     public long getWritePointer() {
-        return 0;
+        return tx.getWriteTimestamp();
     }
 
     @Override
     public PhoenixVisibilityLevel getVisibilityLevel() {
-        return null;
+        VisibilityLevel visibilityLevel = null;
+
+        assert(tx != null);
+        visibilityLevel = tx.getVisibilityLevel();
+
+        PhoenixVisibilityLevel phoenixVisibilityLevel;
+        switch (visibilityLevel) {
+        case SNAPSHOT:
+            phoenixVisibilityLevel = PhoenixVisibilityLevel.SNAPSHOT;
+            break;
+        case SNAPSHOT_EXCLUDE_CURRENT:
+            phoenixVisibilityLevel = PhoenixVisibilityLevel.SNAPSHOT_EXCLUDE_CURRENT;
+            break;
+        case SNAPSHOT_ALL:
+            phoenixVisibilityLevel = PhoenixVisibilityLevel.SNAPSHOT_ALL;
+        default:
+            phoenixVisibilityLevel = null;
+        }
+
+        return phoenixVisibilityLevel;
     }
 
     @Override
     public void setVisibilityLevel(PhoenixVisibilityLevel visibilityLevel) {
+
+        VisibilityLevel omidVisibilityLevel = null;
+
+        switch (visibilityLevel) {
+        case SNAPSHOT:
+            omidVisibilityLevel = VisibilityLevel.SNAPSHOT;
+            break;
+        case SNAPSHOT_EXCLUDE_CURRENT:
+            omidVisibilityLevel = VisibilityLevel.SNAPSHOT_EXCLUDE_CURRENT;
+            break;
+        case SNAPSHOT_ALL:
+            omidVisibilityLevel = VisibilityLevel.SNAPSHOT_ALL;
+            break;
+        default:
+            assert (false);
+        }
+
+        assert(tx != null);
+        tx.setVisibilityLevel(omidVisibilityLevel);
+
     }
 
     @Override
     public byte[] encodeTransaction() throws SQLException {
-        return null;
+        assert(tx != null);
+
+        TSOProto.Transaction.Builder transactionBuilder = TSOProto.Transaction.newBuilder();
+
+        transactionBuilder.setTimestamp(tx.getTransactionId());
+        transactionBuilder.setEpoch(tx.getEpoch());
+
+        byte[] encodedTxBytes = transactionBuilder.build().toByteArray();
+        // Add code of TransactionProvider at end of byte array
+        encodedTxBytes = Arrays.copyOf(encodedTxBytes, encodedTxBytes.length + 1);
+        encodedTxBytes[encodedTxBytes.length - 1] = getProvider().getCode();
+        return encodedTxBytes;
     }
 
     @Override
@@ -110,13 +297,22 @@ public class OmidTransactionContext implements PhoenixTransactionContext {
 
     @Override
     public PhoenixTransactionContext newTransactionContext(PhoenixTransactionContext context, boolean subTask) {
-        return null;
+        return new OmidTransactionContext(context, subTask);
     }
 
     @Override
     public void markDMLFence(PTable dataTable) {
     }
 
+    /**
+    *  OmidTransactionContext specific functions
+    */
+
+    public HBaseTransaction getTransaction() {
+        return tx;
+    }
+
+
     @Override
     public Table getTransactionalTable(Table htable, boolean isConflictFree) throws SQLException {
         return new OmidTransactionTable(this, htable, isConflictFree);
@@ -124,6 +320,9 @@ public class OmidTransactionContext implements PhoenixTransactionContext {
 
     @Override
     public Table getTransactionalTableWriter(PhoenixConnection connection, PTable table, Table htable, boolean isIndex) throws SQLException {
-        return new OmidTransactionTable(this, htable, table.isImmutableRows() || isIndex);
+        // When we're getting a table for writing, if the table being written to is an index,
+        // write the shadow cells immediately since the only time we write to an index is
+        // when we initially populate it synchronously.
+        return new OmidTransactionTable(this, htable, table.isImmutableRows() || isIndex, isIndex);
     }
 }
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/transaction/OmidTransactionProvider.java b/phoenix-core/src/main/java/org/apache/phoenix/transaction/OmidTransactionProvider.java
index bace2bc..87d7225 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/transaction/OmidTransactionProvider.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/transaction/OmidTransactionProvider.java
@@ -24,11 +24,27 @@ import java.util.Arrays;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.coprocessor.RegionObserver;
+import org.apache.omid.committable.CommitTable;
+import org.apache.omid.committable.InMemoryCommitTable;
+import org.apache.omid.transaction.HBaseOmidClientConfiguration;
+import org.apache.omid.transaction.HBaseTransactionManager;
+import org.apache.omid.transaction.TTable;
+import org.apache.omid.tso.TSOMockModule;
+import org.apache.omid.tso.TSOServer;
+import org.apache.omid.tso.TSOServerConfig;
+import org.apache.omid.tso.client.OmidClientConfiguration;
+import org.apache.omid.tso.client.TSOClient;
 import org.apache.phoenix.coprocessor.OmidGCProcessor;
 import org.apache.phoenix.coprocessor.OmidTransactionalProcessor;
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixEmbeddedDriver.ConnectionInfo;
 import org.apache.phoenix.transaction.TransactionFactory.Provider;
+import org.apache.phoenix.util.TransactionUtil;
+
+import com.google.inject.Guice;
+import com.google.inject.Injector;
 
 public class OmidTransactionProvider implements PhoenixTransactionProvider {
     private static final OmidTransactionProvider INSTANCE = new OmidTransactionProvider();
@@ -38,6 +54,10 @@ public class OmidTransactionProvider implements PhoenixTransactionProvider {
     public static final int DEFAULT_OMID_TSO_CONFLICT_MAP_SIZE = 1000;
     public static final String DEFAULT_OMID_TSO_TIMESTAMP_TYPE = "WORLD_TIME";
 
+    private HBaseTransactionManager transactionManager = null;
+    private volatile CommitTable.Client commitTableClient = null;
+    private CommitTable.Writer commitTableWriter = null;
+
     public static final OmidTransactionProvider getInstance() {
         return INSTANCE;
     }
@@ -63,26 +83,106 @@ public class OmidTransactionProvider implements PhoenixTransactionProvider {
 
     @Override
     public PhoenixTransactionClient getTransactionClient(Configuration config, ConnectionInfo connectionInfo) throws SQLException{
-        return new OmidTransactionClient();
+        if (transactionManager == null) {
+            try {
+                HBaseOmidClientConfiguration clientConf = new HBaseOmidClientConfiguration();
+                clientConf.setConflictAnalysisLevel(OmidClientConfiguration.ConflictDetectionLevel.ROW);
+                transactionManager = (HBaseTransactionManager) HBaseTransactionManager.newInstance(clientConf);
+            } catch (IOException | InterruptedException e) {
+                throw new SQLExceptionInfo.Builder(
+                        SQLExceptionCode.TRANSACTION_FAILED)
+                        .setMessage(e.getMessage()).setRootCause(e).build()
+                        .buildException();
+            }
+        }
+
+        return new OmidTransactionClient(transactionManager);
     }
 
     static class OmidTransactionClient implements PhoenixTransactionClient {
+        private final HBaseTransactionManager transactionManager;
+
+        public OmidTransactionClient(HBaseTransactionManager transactionManager) {
+            this.transactionManager = transactionManager;
+        }
+
+        public HBaseTransactionManager getTransactionClient() {
+            return transactionManager;
+        }
+
         @Override
         public void close() throws IOException {}
     }
 
+    // For testing only
+    public CommitTable.Client getCommitTableClient() {
+        return commitTableClient;
+    }
+    
     @Override
     public PhoenixTransactionService getTransactionService(Configuration config, ConnectionInfo connectionInfo, int port) throws  SQLException{
-        return new OmidTransactionService();
+        TSOServerConfig tsoConfig = new TSOServerConfig();
+        TSOServer tso;
+
+        tsoConfig.setPort(port);
+        tsoConfig.setConflictMapSize(config.getInt(OMID_TSO_CONFLICT_MAP_SIZE, DEFAULT_OMID_TSO_CONFLICT_MAP_SIZE));
+        tsoConfig.setTimestampType(config.get(OMID_TSO_TIMESTAMP_TYPE, DEFAULT_OMID_TSO_TIMESTAMP_TYPE));
+
+        Injector injector = Guice.createInjector(new TSOMockModule(tsoConfig));
+        tso = injector.getInstance(TSOServer.class);
+        tso.startAndWait();
+
+        OmidClientConfiguration clientConfig = new OmidClientConfiguration();
+        clientConfig.setConnectionString("localhost:" + port);
+        clientConfig.setConflictAnalysisLevel(OmidClientConfiguration.ConflictDetectionLevel.ROW);
+
+        InMemoryCommitTable commitTable = (InMemoryCommitTable) injector.getInstance(CommitTable.class);
+
+        try {
+            // Create the associated Handler
+            TSOClient client = TSOClient.newInstance(clientConfig);
+
+            HBaseOmidClientConfiguration clientConf = new HBaseOmidClientConfiguration();
+            clientConf.setConnectionString("localhost:" + port);
+            clientConf.setConflictAnalysisLevel(OmidClientConfiguration.ConflictDetectionLevel.ROW);
+            clientConf.setHBaseConfiguration(config);
+            commitTableClient = commitTable.getClient();
+            commitTableWriter = commitTable.getWriter();
+            transactionManager = HBaseTransactionManager.builder(clientConf)
+                    .commitTableClient(commitTableClient)
+                    .commitTableWriter(commitTableWriter)
+                    .tsoClient(client).build();
+        } catch (IOException | InterruptedException e) {
+            throw new SQLExceptionInfo.Builder(
+                    SQLExceptionCode.TRANSACTION_FAILED)
+                    .setMessage(e.getMessage()).setRootCause(e).build()
+                    .buildException();
+        }
+
+        return new OmidTransactionService(tso, transactionManager);
     }
 
     static class OmidTransactionService implements PhoenixTransactionService {
+        private final HBaseTransactionManager transactionManager;
+        private TSOServer tso;
+
+        public OmidTransactionService(TSOServer tso, HBaseTransactionManager transactionManager) {
+            this.tso = tso;
+            this.transactionManager = transactionManager;
+        }
 
         public void start() {
+
         }
 
         @Override
         public void close() throws IOException {
+            if (transactionManager != null) {
+                transactionManager.close();
+            }
+            if (tso != null) {
+                tso.stopAndWait();
+            }
         }
     }
 
@@ -108,6 +208,6 @@ public class OmidTransactionProvider implements PhoenixTransactionProvider {
 
     @Override
     public Put markPutAsCommitted(Put put, long timestamp, long commitTimestamp) {
-        return put;
+        return TTable.markPutAsCommitted(put, timestamp, timestamp);
     }
 }
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/transaction/OmidTransactionTable.java b/phoenix-core/src/main/java/org/apache/phoenix/transaction/OmidTransactionTable.java
index b7339d9..73c3b6d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/transaction/OmidTransactionTable.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/transaction/OmidTransactionTable.java
@@ -19,6 +19,7 @@ package org.apache.phoenix.transaction;
 
 import java.io.IOException;
 import java.sql.SQLException;
+import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
 
@@ -41,6 +42,10 @@ import org.apache.hadoop.hbase.client.coprocessor.Batch.Call;
 import org.apache.hadoop.hbase.client.coprocessor.Batch.Callback;
 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
 import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
+import org.apache.omid.transaction.TTable;
+import org.apache.omid.transaction.Transaction;
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
 
 import com.google.protobuf.Descriptors.MethodDescriptor;
 import com.google.protobuf.Message;
@@ -48,80 +53,115 @@ import com.google.protobuf.Service;
 import com.google.protobuf.ServiceException;
 
 public class OmidTransactionTable implements Table {
+    // Copied from HBase ProtobufUtil since it's not accessible
+    final static Result EMPTY_RESULT_EXISTS_TRUE = Result.create(null, true);
+
+    private TTable tTable;
+    private Transaction tx;
+    private final boolean addShadowCells;
 
     public OmidTransactionTable() throws SQLException {
+        this.tTable = null;
+        this.tx = null;
+        this.addShadowCells = false;
     }
 
     public OmidTransactionTable(PhoenixTransactionContext ctx, Table hTable) throws SQLException {
+        this(ctx, hTable, false);
     }
 
     public OmidTransactionTable(PhoenixTransactionContext ctx, Table hTable, boolean isConflictFree) throws SQLException  {
+        this(ctx, hTable, isConflictFree, false);
+    }
+    
+    public OmidTransactionTable(PhoenixTransactionContext ctx, Table hTable, boolean isConflictFree, boolean addShadowCells) throws SQLException  {
+        assert(ctx instanceof OmidTransactionContext);
+
+        OmidTransactionContext omidTransactionContext = (OmidTransactionContext) ctx;
+        this.addShadowCells = addShadowCells;
+        try {
+            tTable = new TTable(hTable, true, isConflictFree);
+        } catch (IOException e) {
+            throw new SQLExceptionInfo.Builder(
+                    SQLExceptionCode.TRANSACTION_FAILED)
+            .setMessage(e.getMessage()).setRootCause(e).build()
+            .buildException();
+        }
+
+        this.tx = omidTransactionContext.getTransaction();
     }
 
     @Override
     public Result get(Get get) throws IOException {
-        return null;
+        return tTable.get(tx, get);
     }
 
     @Override
     public void put(Put put) throws IOException {
+        tTable.put(tx, put, addShadowCells);
     }
 
     @Override
     public void delete(Delete delete) throws IOException {
+        tTable.delete(tx, delete);
     }
 
     @Override
     public ResultScanner getScanner(Scan scan) throws IOException {
-        return null;
+        scan.setTimeRange(0, Long.MAX_VALUE);
+        return tTable.getScanner(tx, scan);
     }
 
     @Override
     public Configuration getConfiguration() {
-        return null;
+        return tTable.getConfiguration();
     }
 
     @Override
     public HTableDescriptor getTableDescriptor() throws IOException {
-        return null;
+        return tTable.getTableDescriptor();
     }
 
     @Override
     public boolean exists(Get get) throws IOException {
-        return false;
+       return tTable.exists(tx, get);
     }
 
     @Override
     public Result[] get(List<Get> gets) throws IOException {
-        return null;
+        return tTable.get(tx, gets);
     }
 
     @Override
     public ResultScanner getScanner(byte[] family) throws IOException {
-        return null;
+        return tTable.getScanner(tx, family);
     }
 
     @Override
     public ResultScanner getScanner(byte[] family, byte[] qualifier)
             throws IOException {
-        return null;
+        return tTable.getScanner(tx, family, qualifier);
     }
 
     @Override
     public void put(List<Put> puts) throws IOException {
+        tTable.put(tx, puts, addShadowCells);
     }
 
     @Override
     public void delete(List<Delete> deletes) throws IOException {
+        tTable.delete(tx, deletes);
     }
 
     @Override
     public void close() throws IOException {
+        tTable.close();
     }
 
     @Override
     public TableName getName() {
-        return null;
+        byte[] name = tTable.getTableName();
+        return TableName.valueOf(name);
     }
 
     @Override
@@ -132,12 +172,16 @@ public class OmidTransactionTable implements Table {
     @Override
     public void batch(List<? extends Row> actions, Object[] results)
             throws IOException, InterruptedException {
+        tTable.batch(tx, actions, addShadowCells);
+        Arrays.fill(results, EMPTY_RESULT_EXISTS_TRUE);
     }
 
     @Override
     public Object[] batch(List<? extends Row> actions) throws IOException,
             InterruptedException {
-        return null;
+        Object[] results;
+        batch(actions, results = new Object[actions.size()]);
+        return results;
     }
 
     @Override
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/transaction/TransactionFactory.java b/phoenix-core/src/main/java/org/apache/phoenix/transaction/TransactionFactory.java
index 3e9182f..0f10b37 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/transaction/TransactionFactory.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/transaction/TransactionFactory.java
@@ -22,10 +22,11 @@ import java.io.IOException;
 import org.apache.phoenix.coprocessor.MetaDataProtocol;
 
 
+
 public class TransactionFactory {
     public enum Provider {
         TEPHRA((byte)1, TephraTransactionProvider.getInstance(), true),
-        OMID((byte)2, OmidTransactionProvider.getInstance(), false);
+        OMID((byte)2, OmidTransactionProvider.getInstance(), true);
         
         private final byte code;
         private final PhoenixTransactionProvider provider;
@@ -49,7 +50,7 @@ public class TransactionFactory {
         }
         
         public static Provider getDefault() {
-            return TEPHRA;
+            return OMID;
         }
 
         public PhoenixTransactionProvider getTransactionProvider()  {
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/query/QueryServicesTestImpl.java b/phoenix-core/src/test/java/org/apache/phoenix/query/QueryServicesTestImpl.java
index 2049390..59e7fd3 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/query/QueryServicesTestImpl.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/query/QueryServicesTestImpl.java
@@ -22,7 +22,6 @@ import static org.apache.phoenix.query.QueryServicesOptions.withDefaults;
 
 import org.apache.curator.shaded.com.google.common.io.Files;
 import org.apache.hadoop.hbase.regionserver.wal.IndexedWALEditCodec;
-import org.apache.phoenix.transaction.OmidTransactionProvider;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.ReadOnlyProps;
 import org.apache.phoenix.util.TestUtil;
diff --git a/phoenix-server/pom.xml b/phoenix-server/pom.xml
index e6a7afe..def100c 100644
--- a/phoenix-server/pom.xml
+++ b/phoenix-server/pom.xml
@@ -122,6 +122,7 @@
                   <include>org.iq80.snappy:snappy</include>
                   <include>org.antlr:antlr*</include>
                   <include>org.apache.tephra:tephra*</include>
+                  <include>org.apache.omid:omid*</include>
                   <include>com.google.code.gson:gson</include>
                   <include>org.jruby.joni:joni</include>
                   <include>org.jruby.jcodings:jcodings</include>
diff --git a/pom.xml b/pom.xml
index 571ddb3..74f3381 100644
--- a/pom.xml
+++ b/pom.xml
@@ -144,6 +144,7 @@
     <avatica.version>1.12.0</avatica.version>
     <jettyVersion>8.1.7.v20120910</jettyVersion>
     <tephra.version>0.15.0-incubating</tephra.version>
+    <omid.version>1.0.0</omid.version>
     <spark.version>${cdh.spark.version}</spark.version>
     <scala.version>2.10.4</scala.version>
     <scala.binary.version>2.10</scala.binary.version>
@@ -846,6 +847,52 @@
         <artifactId>tephra-hbase-compat-1.2-cdh</artifactId>
         <version>${tephra.version}</version>
       </dependency>
+      <dependency>
+        <groupId>org.apache.omid</groupId>
+        <artifactId>omid-hbase-client-hbase1.x</artifactId>
+        <version>${omid.version}</version>
+        <exclusions>
+          <exclusion>
+            <groupId>org.testng</groupId>
+            <artifactId>testng</artifactId>
+          </exclusion>
+        </exclusions>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.omid</groupId>
+        <artifactId>omid-hbase-coprocessor-hbase1.x</artifactId>
+        <version>${omid.version}</version>
+        <exclusions>
+          <exclusion>
+            <groupId>org.testng</groupId>
+            <artifactId>testng</artifactId>
+          </exclusion>
+        </exclusions>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.omid</groupId>
+        <artifactId>omid-tso-server-hbase1.x</artifactId>
+        <version>${omid.version}</version>
+        <exclusions>
+          <exclusion>
+          <groupId>org.testng</groupId>
+          <artifactId>testng</artifactId>
+        </exclusion>
+      </exclusions>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.omid</groupId>
+        <artifactId>omid-tso-server-hbase1.x</artifactId>
+        <version>${omid.version}</version>
+        <type>test-jar</type>
+        <exclusions>
+          <exclusion>
+            <groupId>org.testng</groupId>
+            <artifactId>testng</artifactId>
+          </exclusion>
+        </exclusions>
+      </dependency>
+
 
       <!-- Make sure we have all the antlr dependencies -->
       <dependency>


[phoenix] 15/18: PHOENIX-4820 Optimize OrderBy for ClientAggregatePlan

Posted by pb...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

pboado pushed a commit to branch 4.x-cdh5.16
in repository https://gitbox.apache.org/repos/asf/phoenix.git

commit 460da6136a75245b119d4f0393e08e9f61d579d5
Author: chenglei <ch...@apache.org>
AuthorDate: Sat Jan 5 01:58:00 2019 +0000

    PHOENIX-4820 Optimize OrderBy for ClientAggregatePlan
---
 .../org/apache/phoenix/end2end/AggregateIT.java    | 104 +++++++
 .../apache/phoenix/compile/GroupByCompiler.java    |   8 +-
 .../apache/phoenix/compile/OrderByCompiler.java    |  18 +-
 .../phoenix/compile/OrderPreservingTracker.java    |  53 ++--
 .../org/apache/phoenix/compile/QueryCompiler.java  |  12 +-
 .../org/apache/phoenix/compile/RowProjector.java   |  15 +-
 .../phoenix/expression/BaseCompoundExpression.java |  11 +-
 .../apache/phoenix/expression/BaseExpression.java  |  11 +
 .../phoenix/expression/BaseSingleExpression.java   |   5 +
 .../phoenix/expression/DelegateExpression.java     |   5 +
 .../org/apache/phoenix/expression/Expression.java  |   6 +
 .../expression/ProjectedColumnExpression.java      |   8 +-
 .../expression/function/RandomFunction.java        |   5 +
 .../expression/visitor/CloneExpressionVisitor.java |   6 +-
 .../CloneNonDeterministicExpressionVisitor.java    |  31 --
 .../org/apache/phoenix/util/ExpressionUtil.java    | 160 +++++++++-
 .../apache/phoenix/compile/QueryCompilerTest.java  | 324 +++++++++++++++++++++
 .../expression/ArithmeticOperationTest.java        |   8 +-
 18 files changed, 705 insertions(+), 85 deletions(-)

diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AggregateIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AggregateIT.java
index 8916d4d..d52025e 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AggregateIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AggregateIT.java
@@ -227,5 +227,109 @@ public class AggregateIT extends BaseAggregateIT {
             assertEquals(4, rs.getLong(1));
         }
     }
+
+    @Test
+    public void testOrderByOptimizeForClientAggregatePlanBug4820() throws Exception {
+        doTestOrderByOptimizeForClientAggregatePlanBug4820(false,false);
+        doTestOrderByOptimizeForClientAggregatePlanBug4820(false,true);
+        doTestOrderByOptimizeForClientAggregatePlanBug4820(true,false);
+        doTestOrderByOptimizeForClientAggregatePlanBug4820(true,true);
+    }
+
+    private void doTestOrderByOptimizeForClientAggregatePlanBug4820(boolean desc ,boolean salted) throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = null;
+        try {
+            conn = DriverManager.getConnection(getUrl(), props);
+            String tableName = generateUniqueName();
+            String sql = "create table " + tableName + "( "+
+                    " pk1 varchar not null , " +
+                    " pk2 varchar not null, " +
+                    " pk3 varchar not null," +
+                    " v1 varchar, " +
+                    " v2 varchar, " +
+                    " CONSTRAINT TEST_PK PRIMARY KEY ( "+
+                    "pk1 "+(desc ? "desc" : "")+", "+
+                    "pk2 "+(desc ? "desc" : "")+", "+
+                    "pk3 "+(desc ? "desc" : "")+
+                    " )) "+(salted ? "SALT_BUCKETS =4" : "split on('b')");
+            conn.createStatement().execute(sql);
+
+            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES ('a11','a12','a13','a14','a15')");
+            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES ('a21','a22','a23','a24','a25')");
+            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES ('a31','a32','a33','a34','a35')");
+            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES ('b11','b12','b13','b14','b15')");
+            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES ('b21','b22','b23','b24','b25')");
+            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES ('b31','b32','b33','b34','b35')");
+            conn.commit();
+
+            sql = "select a.ak3 "+
+                  "from (select pk1 ak1,pk2 ak2,pk3 ak3, substr(v1,1,1) av1,substr(v2,1,1) av2 from "+tableName+" order by pk2,pk3 limit 10) a "+
+                  "group by a.ak3,a.av1 order by a.ak3 desc,a.av1";
+            ResultSet rs = conn.prepareStatement(sql).executeQuery();
+            assertResultSet(rs, new Object[][]{{"b33"},{"b23"},{"b13"},{"a33"},{"a23"},{"a13"}});
+
+            sql = "select a.ak3 "+
+                  "from (select pk1 ak1,pk2 ak2,pk3 ak3, substr(v1,1,1) av1,substr(v2,1,1) av2 from "+tableName+" order by pk2,pk3 limit 10) a "+
+                  "group by a.ak3,a.av1 order by a.ak3,a.av1";
+            rs = conn.prepareStatement(sql).executeQuery();
+            assertResultSet(rs, new Object[][]{{"a13"},{"a23"},{"a33"},{"b13"},{"b23"},{"b33"}});
+
+            sql = "select a.ak3 "+
+                  "from (select pk1 ak1,pk2 ak2,pk3 ak3,substr(v1,1,1) av1,substr(v2,1,1) av2 from "+tableName+" order by pk2,pk3 limit 10) a "+
+                  "where a.av1 = 'a' group by a.av1,a.ak3 order by a.ak3 desc";
+            rs = conn.prepareStatement(sql).executeQuery();
+            assertResultSet(rs, new Object[][]{{"a33"},{"a23"},{"a13"}});
+
+            sql = "select a.ak3 "+
+                    "from (select pk1 ak1,pk2 ak2,pk3 ak3,substr(v1,1,1) av1,substr(v2,1,1) av2 from "+tableName+" order by pk2,pk3 limit 10) a "+
+                    "where a.av1 = 'a' group by a.av1,a.ak3 order by a.ak3";
+            rs = conn.prepareStatement(sql).executeQuery();
+            assertResultSet(rs, new Object[][]{{"a13"},{"a23"},{"a33"}});
+
+            sql = "select a.ak3 "+
+                    "from (select pk1 ak1,pk2 ak2,pk3 ak3,substr(v1,1,1) av1,substr(v2,1,1) av2 from "+tableName+" order by pk2,pk3 limit 10) a "+
+                    "where a.av1 = 'b' and a.av2= 'b' group by CASE WHEN a.av1 > a.av2 THEN a.av1 ELSE a.av2 END,a.ak3,a.ak2 order by a.ak3 desc,a.ak2 desc";
+            rs = conn.prepareStatement(sql).executeQuery();
+            assertResultSet(rs, new Object[][]{{"b33"},{"b23"},{"b13"}});
+
+            sql = "select a.ak3 "+
+                    "from (select pk1 ak1,pk2 ak2,pk3 ak3,substr(v1,1,1) av1,substr(v2,1,1) av2 from "+tableName+" order by pk2,pk3 limit 10) a "+
+                    "where a.av1 = 'b' and a.av2= 'b' group by CASE WHEN a.av1 > a.av2 THEN a.av1 ELSE a.av2 END,a.ak3,a.ak2 order by a.ak3,a.ak2 desc";
+            rs = conn.prepareStatement(sql).executeQuery();
+            assertResultSet(rs, new Object[][]{{"b13"},{"b23"},{"b33"}});
+
+            tableName = generateUniqueName();
+            sql = "create table " + tableName + "( "+
+                    " pk1 double not null , " +
+                    " pk2 double not null, " +
+                    " pk3 double not null," +
+                    " v1 varchar, " +
+                    " CONSTRAINT TEST_PK PRIMARY KEY ( "+
+                    "pk1 "+(desc ? "desc" : "")+", "+
+                    "pk2 "+(desc ? "desc" : "")+", "+
+                    "pk3 "+(desc ? "desc" : "")+
+                    " )) "+(salted ? "SALT_BUCKETS =4" : "split on(2.3)");
+            conn.createStatement().execute(sql);
+            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (2.1,2.11,2.12,'e')");
+            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (2.2,2.21,2.23,'d')");
+            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (2.3,2.31,2.32,'c')");
+            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (2.4,2.41,2.42,'b')");
+            conn.createStatement().execute("UPSERT INTO "+tableName+" VALUES (2.5,2.51,2.52,'a')");
+            conn.commit();
+
+            sql = "select a.av1 "+
+                  "from (select pk1 ak1,pk2 ak2,pk3 ak3, substr(v1,1,1) av1 from "+tableName+" order by pk1,pk2 limit 10) a "+
+                  "where cast(a.ak1 as integer)=2 group by a.ak1,a.av1 order by a.av1";
+            rs = conn.prepareStatement(sql).executeQuery();
+            assertResultSet(rs, new Object[][]{{"a"},{"b"},{"c"},{"d"},{"e"}});
+
+        } finally {
+            if(conn != null) {
+                conn.close();
+            }
+        }
+    }
+
 }
 
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/GroupByCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/GroupByCompiler.java
index 4777c29..2bdea9a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/GroupByCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/GroupByCompiler.java
@@ -143,7 +143,13 @@ public class GroupByCompiler {
             boolean isOrderPreserving = this.isOrderPreserving;
             int orderPreservingColumnCount = 0;
             if (isOrderPreserving) {
-                OrderPreservingTracker tracker = new OrderPreservingTracker(context, GroupBy.EMPTY_GROUP_BY, Ordering.UNORDERED, expressions.size(), tupleProjector);
+                OrderPreservingTracker tracker = new OrderPreservingTracker(
+                        context,
+                        GroupBy.EMPTY_GROUP_BY,
+                        Ordering.UNORDERED,
+                        expressions.size(),
+                        tupleProjector,
+                        null);
                 for (int i = 0; i < expressions.size(); i++) {
                     Expression expression = expressions.get(i);
                     tracker.track(expression);
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/OrderByCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/OrderByCompiler.java
index b83c7a8..3c3f429 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/OrderByCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/OrderByCompiler.java
@@ -88,7 +88,8 @@ public class OrderByCompiler {
                                   Integer offset,
                                   RowProjector rowProjector,
                                   TupleProjector tupleProjector,
-                                  boolean isInRowKeyOrder) throws SQLException {
+                                  boolean isInRowKeyOrder,
+                                  Expression whereExpression) throws SQLException {
         List<OrderByNode> orderByNodes = statement.getOrderBy();
         if (orderByNodes.isEmpty()) {
             return OrderBy.EMPTY_ORDER_BY;
@@ -105,9 +106,22 @@ public class OrderByCompiler {
         } else {
             compiler = new ExpressionCompiler(context, groupBy);
         }
+
+        if(groupBy != GroupBy.EMPTY_GROUP_BY) {
+            //if there is groupBy,the groupBy.expressions are viewed as new rowKey columns,so
+            //tupleProjector and isInRowKeyOrder is cleared
+            tupleProjector = null;
+            isInRowKeyOrder = true;
+        }
         // accumulate columns in ORDER BY
         OrderPreservingTracker tracker = 
-                new OrderPreservingTracker(context, groupBy, Ordering.ORDERED, orderByNodes.size(), tupleProjector);
+                new OrderPreservingTracker(
+                        context,
+                        groupBy,
+                        Ordering.ORDERED,
+                        orderByNodes.size(),
+                        tupleProjector,
+                        whereExpression);
         LinkedHashSet<OrderByExpression> orderByExpressions = Sets.newLinkedHashSetWithExpectedSize(orderByNodes.size());
         for (OrderByNode node : orderByNodes) {
             ParseNode parseNode = node.getNode();
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/OrderPreservingTracker.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/OrderPreservingTracker.java
index d1175f6..29b3794 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/OrderPreservingTracker.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/OrderPreservingTracker.java
@@ -18,8 +18,8 @@ import java.util.List;
 import org.apache.phoenix.compile.GroupByCompiler.GroupBy;
 import org.apache.phoenix.execute.TupleProjector;
 import org.apache.phoenix.expression.CoerceExpression;
-import org.apache.phoenix.expression.Determinism;
 import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.KeyValueColumnExpression;
 import org.apache.phoenix.expression.LiteralExpression;
 import org.apache.phoenix.expression.ProjectedColumnExpression;
 import org.apache.phoenix.expression.RowKeyColumnExpression;
@@ -30,6 +30,7 @@ import org.apache.phoenix.expression.visitor.StatelessTraverseAllExpressionVisit
 import org.apache.phoenix.expression.visitor.StatelessTraverseNoExpressionVisitor;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.SortOrder;
+import org.apache.phoenix.util.ExpressionUtil;
 
 import com.google.common.collect.Iterators;
 import com.google.common.collect.Lists;
@@ -76,16 +77,22 @@ public class OrderPreservingTracker {
     private final Ordering ordering;
     private final int pkPositionOffset;
     private final List<Info> orderPreservingInfos;
-    private final TupleProjector projector;
     private boolean isOrderPreserving = true;
     private Boolean isReverse = null;
     private int orderPreservingColumnCount = 0;
+    private Expression whereExpression;
     
     public OrderPreservingTracker(StatementContext context, GroupBy groupBy, Ordering ordering, int nNodes) {
-        this(context, groupBy, ordering, nNodes, null);
+        this(context, groupBy, ordering, nNodes, null, null);
     }
     
-    public OrderPreservingTracker(StatementContext context, GroupBy groupBy, Ordering ordering, int nNodes, TupleProjector projector) {
+    public OrderPreservingTracker(
+            StatementContext context,
+            GroupBy groupBy,
+            Ordering ordering,
+            int nNodes,
+            TupleProjector projector,
+            Expression whereExpression) {
         this.context = context;
         if (groupBy.isEmpty()) {
             PTable table = context.getResolver().getTables().get(0).getTable();
@@ -103,7 +110,7 @@ public class OrderPreservingTracker {
         this.visitor = new TrackOrderPreservingExpressionVisitor(projector);
         this.orderPreservingInfos = Lists.newArrayListWithExpectedSize(nNodes);
         this.ordering = ordering;
-        this.projector = projector;
+        this.whereExpression = whereExpression;
     }
     
     public void track(Expression node) {
@@ -208,20 +215,14 @@ public class OrderPreservingTracker {
         // not by the original row key order of the table (see PHOENIX-3451).
         // We check each GROUP BY expression to see if it only references columns that are
         // matched by equality constraints, in which case the expression itself would be constant.
-        // FIXME: this only recognizes row key columns that are held constant, not all columns.
-        // FIXME: we should optimize out any GROUP BY or ORDER BY expression which is deemed to
-        // be held constant based on the WHERE clause.
         if (!groupBy.isEmpty()) {
             for (int pos = startPos; pos < endPos; pos++) {
-                IsConstantVisitor visitor = new IsConstantVisitor(this.projector, ranges);
+                IsConstantVisitor visitor = new IsConstantVisitor(ranges, whereExpression);
                 List<Expression> groupByExpressions = groupBy.getExpressions();
                 if (pos >= groupByExpressions.size()) { // sanity check - shouldn't be necessary
                     return false;
                 }
                 Expression groupByExpression = groupByExpressions.get(pos);
-                if ( groupByExpression.getDeterminism().ordinal() > Determinism.PER_STATEMENT.ordinal() ) {
-                    return false;
-                }
                 Boolean isConstant = groupByExpression.accept(visitor);
                 if (!Boolean.TRUE.equals(isConstant)) {
                     return false;
@@ -248,17 +249,17 @@ public class OrderPreservingTracker {
      *
      */
     private static class IsConstantVisitor extends StatelessTraverseAllExpressionVisitor<Boolean> {
-        private final TupleProjector projector;
         private final ScanRanges scanRanges;
+        private final Expression whereExpression;
         
-        public IsConstantVisitor(TupleProjector projector, ScanRanges scanRanges) {
-            this.projector = projector;
-            this.scanRanges = scanRanges;
-        }
+        public IsConstantVisitor(ScanRanges scanRanges, Expression whereExpression) {
+           this.scanRanges = scanRanges;
+           this.whereExpression = whereExpression;
+       }
         
         @Override
         public Boolean defaultReturn(Expression node, List<Boolean> returnValues) {
-            if (node.getDeterminism().ordinal() > Determinism.PER_STATEMENT.ordinal() || 
+            if (!ExpressionUtil.isContantForStatement(node) ||
                     returnValues.size() < node.getChildren().size()) {
                 return Boolean.FALSE;
             }
@@ -281,16 +282,12 @@ public class OrderPreservingTracker {
         }
 
         @Override
-        public Boolean visit(ProjectedColumnExpression node) {
-            if (projector == null) {
-                return super.visit(node);
-            }
-            Expression expression = projector.getExpressions()[node.getPosition()];
-            // Only look one level down the projection.
-            if (expression instanceof ProjectedColumnExpression) {
-                return super.visit(node);
-            }
-            return expression.accept(this);
+        public Boolean visit(KeyValueColumnExpression keyValueColumnExpression) {
+            return ExpressionUtil.isColumnExpressionConstant(keyValueColumnExpression, whereExpression);
+        }
+         @Override
+        public Boolean visit(ProjectedColumnExpression projectedColumnExpression) {
+            return ExpressionUtil.isColumnExpressionConstant(projectedColumnExpression, whereExpression);
         }
     }
     /**
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/QueryCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/QueryCompiler.java
index 603da0b..6e36158 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/QueryCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/QueryCompiler.java
@@ -562,8 +562,16 @@ public class QueryCompiler {
         groupBy = groupBy.compile(context, innerPlanTupleProjector);
         context.setResolver(resolver); // recover resolver
         RowProjector projector = ProjectionCompiler.compile(context, select, groupBy, asSubquery ? Collections.<PDatum>emptyList() : targetColumns, where);
-        OrderBy orderBy = OrderByCompiler.compile(context, select, groupBy, limit, offset, projector,
-                groupBy == GroupBy.EMPTY_GROUP_BY ? innerPlanTupleProjector : null, isInRowKeyOrder);
+        OrderBy orderBy = OrderByCompiler.compile(
+                context,
+                select,
+                groupBy,
+                limit,
+                offset,
+                projector,
+                innerPlanTupleProjector,
+                isInRowKeyOrder,
+                where);
         context.getAggregationManager().compile(context, groupBy);
         // Final step is to build the query plan
         if (!asSubquery) {
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/RowProjector.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/RowProjector.java
index 2123788..8532e0c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/RowProjector.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/RowProjector.java
@@ -22,9 +22,8 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 
-import org.apache.phoenix.expression.Determinism;
 import org.apache.phoenix.expression.Expression;
-import org.apache.phoenix.expression.visitor.CloneNonDeterministicExpressionVisitor;
+import org.apache.phoenix.expression.visitor.CloneExpressionVisitor;
 import org.apache.phoenix.schema.ColumnNotFoundException;
 import org.apache.phoenix.util.SchemaUtil;
 
@@ -97,17 +96,17 @@ public class RowProjector {
         this.isProjectEmptyKeyValue = isProjectEmptyKeyValue;
         this.isProjectAll = isProjectAll;
         this.hasUDFs = hasUDFs;
-        boolean hasPerInvocationExpression = false;
+        boolean cloneRequired = false;
         if (!hasUDFs) {
             for (int i = 0; i < this.columnProjectors.size(); i++) {
                 Expression expression = this.columnProjectors.get(i).getExpression();
-                if (expression.getDeterminism() == Determinism.PER_INVOCATION) {
-                    hasPerInvocationExpression = true;
+                if (expression.isCloneExpression()) {
+                    cloneRequired = true;
                     break;
                 }
             }
         }
-        this.cloneRequired = hasPerInvocationExpression || hasUDFs;
+        this.cloneRequired = cloneRequired || hasUDFs;
     }
 
     public RowProjector cloneIfNecessary() {
@@ -118,8 +117,8 @@ public class RowProjector {
         for (int i = 0; i < this.columnProjectors.size(); i++) {
             ColumnProjector colProjector = columnProjectors.get(i);
             Expression expression = colProjector.getExpression();
-            if (expression.getDeterminism() == Determinism.PER_INVOCATION) {
-                CloneNonDeterministicExpressionVisitor visitor = new CloneNonDeterministicExpressionVisitor();
+            if (expression.isCloneExpression()) {
+                CloneExpressionVisitor visitor = new CloneExpressionVisitor();
                 Expression clonedExpression = expression.accept(visitor);
                 clonedColProjectors.add(new ExpressionProjector(colProjector.getName(),
                         colProjector.getTableName(), 
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/BaseCompoundExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/BaseCompoundExpression.java
index 5fc8361..e26c902 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/BaseCompoundExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/BaseCompoundExpression.java
@@ -35,6 +35,7 @@ public abstract class BaseCompoundExpression extends BaseExpression {
     private boolean isStateless;
     private Determinism determinism;
     private boolean requiresFinalEvaluation;
+    private boolean cloneExpression;
    
     public BaseCompoundExpression() {
         init(Collections.<Expression>emptyList());
@@ -49,6 +50,7 @@ public abstract class BaseCompoundExpression extends BaseExpression {
         boolean isStateless = true;
         boolean isNullable = false;
         boolean requiresFinalEvaluation = false;
+        boolean cloneExpression = false;
         this.determinism = Determinism.ALWAYS;
         for (int i = 0; i < children.size(); i++) {
             Expression child = children.get(i);
@@ -56,10 +58,12 @@ public abstract class BaseCompoundExpression extends BaseExpression {
             isStateless &= child.isStateless();
             this.determinism = this.determinism.combine(child.getDeterminism());
             requiresFinalEvaluation |= child.requiresFinalEvaluation();
+            cloneExpression |= child.isCloneExpression();
         }
         this.isStateless = isStateless;
         this.isNullable = isNullable;
         this.requiresFinalEvaluation = requiresFinalEvaluation;
+        this.cloneExpression = cloneExpression;
     }
     
     @Override
@@ -72,7 +76,12 @@ public abstract class BaseCompoundExpression extends BaseExpression {
     public Determinism getDeterminism() {
         return determinism;
     }
-    
+
+    @Override
+    public boolean isCloneExpression() {
+        return this.cloneExpression;
+    }
+
     @Override
     public boolean isStateless() {
         return isStateless;
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/BaseExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/BaseExpression.java
index efdceac..ccb6073 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/BaseExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/BaseExpression.java
@@ -255,4 +255,15 @@ public abstract class BaseExpression implements Expression {
         return false;
     }
 
+    @Override
+    public boolean isCloneExpression()  {
+       return isCloneExpressionByDeterminism(this);
+    }
+
+    protected static boolean isCloneExpressionByDeterminism(BaseExpression expression) {
+        if(expression.getDeterminism() == Determinism.PER_INVOCATION) {
+            return true;
+        }
+        return false;
+    }
 }
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/BaseSingleExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/BaseSingleExpression.java
index fbe8040..c636319 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/BaseSingleExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/BaseSingleExpression.java
@@ -118,4 +118,9 @@ public abstract class BaseSingleExpression extends BaseExpression {
     public Determinism getDeterminism() {
         return children.get(0).getDeterminism();
     }
+
+    @Override
+    public boolean isCloneExpression() {
+        return children.get(0).isCloneExpression();
+    }
 }
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/DelegateExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/DelegateExpression.java
index 3ca93dd..fd783c3 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/DelegateExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/DelegateExpression.java
@@ -105,4 +105,9 @@ public class DelegateExpression implements Expression {
         return delegate.requiresFinalEvaluation();
     }
 
+    @Override
+    public boolean isCloneExpression() {
+        return delegate.isCloneExpression();
+    }
+
 }
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/Expression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/Expression.java
index aeea0c8..d490ced 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/Expression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/Expression.java
@@ -88,4 +88,10 @@ public interface Expression extends PDatum, Writable {
      * @return
      */
     boolean requiresFinalEvaluation();
+
+    /**
+     * Determines if expression needs to be cloned in {@link org.apache.phoenix.compile.RowProjector}
+     * @return
+     */
+    boolean isCloneExpression();
 }
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/ProjectedColumnExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/ProjectedColumnExpression.java
index f851efa..e60bf00 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/ProjectedColumnExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/ProjectedColumnExpression.java
@@ -153,12 +153,12 @@ public class ProjectedColumnExpression extends ColumnExpression {
     }
 
     @Override
-    public Determinism getDeterminism() {
-        return Determinism.PER_INVOCATION;
+    public ProjectedColumnExpression clone() {
+        return new ProjectedColumnExpression(this.column, this.columns, this.position, this.displayName);
     }
 
     @Override
-    public ProjectedColumnExpression clone() {
-        return new ProjectedColumnExpression(this.column, this.columns, this.position, this.displayName);
+    public boolean isCloneExpression() {
+        return true;
     }
 }
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RandomFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RandomFunction.java
index 01a4eed..d9048f4 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RandomFunction.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/RandomFunction.java
@@ -118,6 +118,11 @@ public class RandomFunction extends ScalarFunction {
     }
 
     @Override
+    public boolean isCloneExpression()  {
+        return isCloneExpressionByDeterminism(this);
+    }
+
+    @Override
     public boolean isStateless() {
         return true;
     }
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/CloneExpressionVisitor.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/CloneExpressionVisitor.java
index c6d7c9e..b7ea4ab 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/CloneExpressionVisitor.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/CloneExpressionVisitor.java
@@ -51,7 +51,7 @@ import org.apache.phoenix.expression.function.ScalarFunction;
 import org.apache.phoenix.expression.function.SingleAggregateFunction;
 import org.apache.phoenix.expression.function.UDFExpression;
 
-public abstract class CloneExpressionVisitor extends TraverseAllExpressionVisitor<Expression> {
+public class CloneExpressionVisitor extends TraverseAllExpressionVisitor<Expression> {
 
     public CloneExpressionVisitor() {
     }
@@ -215,5 +215,7 @@ public abstract class CloneExpressionVisitor extends TraverseAllExpressionVisito
         return isCloneNode(node, l) ? new ArrayElemRefExpression(l) : node;
     }
 
-    public abstract boolean isCloneNode(Expression node, List<Expression> children);
+    public boolean isCloneNode(Expression node, List<Expression> children) {
+        return node.isCloneExpression();
+    }
 }
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/CloneNonDeterministicExpressionVisitor.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/CloneNonDeterministicExpressionVisitor.java
deleted file mode 100644
index 9a56e36..0000000
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/CloneNonDeterministicExpressionVisitor.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * 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.expression.visitor;
-
-import java.util.List;
-
-import org.apache.phoenix.expression.Determinism;
-import org.apache.phoenix.expression.Expression;
-
-public class CloneNonDeterministicExpressionVisitor extends CloneExpressionVisitor {
-
-    @Override
-    public boolean isCloneNode(Expression node, List<Expression> children) {
-        return Determinism.PER_INVOCATION.compareTo(node.getDeterminism()) <= 0;
-    }
-}
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 881b0e1..e737721 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
@@ -10,12 +10,20 @@
 package org.apache.phoenix.util;
 
 import java.sql.SQLException;
+import java.util.Iterator;
 import java.util.List;
 
+import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.phoenix.expression.AndExpression;
+import org.apache.phoenix.expression.ColumnExpression;
+import org.apache.phoenix.expression.ComparisonExpression;
 import org.apache.phoenix.expression.Determinism;
 import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.IsNullExpression;
 import org.apache.phoenix.expression.LiteralExpression;
+import org.apache.phoenix.expression.visitor.StatelessTraverseAllExpressionVisitor;
+import org.apache.phoenix.expression.visitor.StatelessTraverseNoExpressionVisitor;
 import org.apache.phoenix.schema.ColumnRef;
 import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.TableRef;
@@ -27,10 +35,19 @@ public class ExpressionUtil {
 	}
 
 	public static boolean isConstant(Expression expression) {
-		return (expression.isStateless() && (expression.getDeterminism() == Determinism.ALWAYS
-				|| expression.getDeterminism() == Determinism.PER_STATEMENT));
+		return (expression.isStateless() && isContantForStatement(expression));
 	}
 
+   /**
+    * this method determines if expression is constant if all children of it are constants.
+    * @param expression
+    * @return
+    */
+    public static boolean isContantForStatement(Expression expression) {
+        return  (expression.getDeterminism() == Determinism.ALWAYS
+                || expression.getDeterminism() == Determinism.PER_STATEMENT);
+    }
+
     public static LiteralExpression getConstantExpression(Expression expression, ImmutableBytesWritable ptr)
             throws SQLException {
         Object value = null;
@@ -68,4 +85,143 @@ public class ExpressionUtil {
         return false;
     }
 
+    /**
+     * check the whereExpression to see if the columnExpression is constant.
+     * eg. for "where a =3 and b > 9", a is constant,but b is not.
+     * @param columnExpression
+     * @param whereExpression
+     * @return
+     */
+    public static boolean isColumnExpressionConstant(ColumnExpression columnExpression, Expression whereExpression) {
+        if(whereExpression == null) {
+            return false;
+        }
+        IsColumnConstantExpressionVisitor isColumnConstantExpressionVisitor =
+                new IsColumnConstantExpressionVisitor(columnExpression);
+        whereExpression.accept(isColumnConstantExpressionVisitor);
+        return isColumnConstantExpressionVisitor.isConstant();
+    }
+
+    private static class IsColumnConstantExpressionVisitor extends StatelessTraverseNoExpressionVisitor<Void> {
+        private final Expression columnExpression ;
+        private Expression firstRhsConstantExpression = null;
+        private int rhsConstantCount = 0;
+        private boolean isNullExpressionVisited = false;
+
+        public IsColumnConstantExpressionVisitor(Expression columnExpression) {
+            this.columnExpression = columnExpression;
+        }
+        /**
+         * only consider and,for "where a = 3 or b = 9", neither a or b is constant.
+         */
+        @Override
+        public Iterator<Expression> visitEnter(AndExpression andExpression) {
+            if(rhsConstantCount > 1) {
+                return null;
+            }
+            return andExpression.getChildren().iterator();
+        }
+        /**
+         * <pre>
+         * We just consider {@link ComparisonExpression} because:
+         * 1.for {@link InListExpression} as "a in ('2')", the {@link InListExpression} is rewritten to
+         *  {@link ComparisonExpression} in {@link InListExpression#create}
+         * 2.for {@link RowValueConstructorExpression} as "(a,b)=(1,2)",{@link RowValueConstructorExpression}
+         *   is rewritten to {@link ComparisonExpression} in {@link ComparisonExpression#create}
+         * 3.not consider {@link CoerceExpression}, because for "where cast(a as integer)=2", when a is double,
+         *   a is not constant.
+         * </pre>
+         */
+        @Override
+        public Iterator<Expression> visitEnter(ComparisonExpression comparisonExpression) {
+            if(rhsConstantCount > 1) {
+                return null;
+            }
+            if(comparisonExpression.getFilterOp() != CompareOp.EQUAL) {
+                return null;
+            }
+            Expression lhsExpresssion = comparisonExpression.getChildren().get(0);
+            if(!this.columnExpression.equals(lhsExpresssion)) {
+                return null;
+            }
+            Expression rhsExpression = comparisonExpression.getChildren().get(1);
+            if(rhsExpression == null) {
+                return null;
+            }
+            Boolean isConstant = rhsExpression.accept(new IsCompositeLiteralExpressionVisitor());
+            if(isConstant != null && isConstant.booleanValue()) {
+                checkConstantValue(rhsExpression);
+            }
+            return null;
+        }
+
+        public boolean isConstant() {
+            return this.rhsConstantCount == 1;
+        }
+
+        @Override
+        public Iterator<Expression> visitEnter(IsNullExpression isNullExpression) {
+            if(rhsConstantCount > 1) {
+                return null;
+            }
+            if(isNullExpression.isNegate()) {
+                return null;
+            }
+            Expression lhsExpresssion = isNullExpression.getChildren().get(0);
+            if(!this.columnExpression.equals(lhsExpresssion)) {
+                return null;
+            }
+            this.checkConstantValue(null);
+            return null;
+        }
+
+        private void checkConstantValue(Expression rhsExpression) {
+            if(!this.isNullExpressionVisited && this.firstRhsConstantExpression == null) {
+                this.firstRhsConstantExpression = rhsExpression;
+                rhsConstantCount++;
+                if(rhsExpression == null) {
+                    this.isNullExpressionVisited = true;
+                }
+                return;
+            }
+
+            if(!isExpressionEquals(this.isNullExpressionVisited ? null : this.firstRhsConstantExpression, rhsExpression)) {
+                rhsConstantCount++;
+                return;
+            }
+        }
+
+        private static boolean isExpressionEquals(Expression oldExpression,Expression newExpression) {
+            if(oldExpression == null) {
+                if(newExpression == null) {
+                    return true;
+                }
+                return ExpressionUtil.isNull(newExpression, new ImmutableBytesWritable());
+            }
+            if(newExpression == null) {
+                return ExpressionUtil.isNull(oldExpression, new ImmutableBytesWritable());
+            }
+            return oldExpression.equals(newExpression);
+        }
+    }
+
+    private static class IsCompositeLiteralExpressionVisitor extends StatelessTraverseAllExpressionVisitor<Boolean> {
+        @Override
+        public Boolean defaultReturn(Expression expression, List<Boolean> childResultValues) {
+            if (!ExpressionUtil.isContantForStatement(expression) ||
+                    childResultValues.size() < expression.getChildren().size()) {
+                return Boolean.FALSE;
+            }
+            for (Boolean childResultValue : childResultValues) {
+                if (!childResultValue) {
+                    return Boolean.FALSE;
+                }
+            }
+            return Boolean.TRUE;
+        }
+        @Override
+        public Boolean visit(LiteralExpression literalExpression) {
+            return Boolean.TRUE;
+        }
+    }
 }
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 154dd7a..68954b8 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
@@ -2951,6 +2951,129 @@ public class QueryCompilerTest extends BaseConnectionlessQueryTest {
             }
         }
     }
+
+    @Test
+    public void testOrderPreservingGroupByForNotPkColumns() throws Exception {
+         try (Connection conn= DriverManager.getConnection(getUrl())) {
+             conn.createStatement().execute("CREATE TABLE test (\n" +
+                    "            pk1 varchar, \n" +
+                    "            pk2 varchar, \n" +
+                    "            pk3 varchar, \n" +
+                    "            pk4 varchar, \n" +
+                    "            v1 varchar, \n" +
+                    "            v2 varchar,\n" +
+                    "            CONSTRAINT pk PRIMARY KEY (\n" +
+                    "               pk1,\n" +
+                    "               pk2,\n" +
+                    "               pk3,\n" +
+                    "               pk4\n" +
+                    "             )\n" +
+                    "         )");
+            String[] queries = new String[] {
+                    "SELECT pk3 FROM test WHERE v2 = 'a' GROUP BY substr(v2,0,1),pk3 ORDER BY pk3",
+                    "SELECT pk3 FROM test WHERE pk1 = 'c' and v2 = substr('abc',1,1) GROUP BY v2,pk3 ORDER BY pk3",
+                    "SELECT pk3 FROM test WHERE v1 = 'a' and v2 = 'b' GROUP BY length(v1)+length(v2),pk3 ORDER BY pk3",
+                    "SELECT pk3 FROM test WHERE pk1 = 'a' and v2 = 'b' GROUP BY length(pk1)+length(v2),pk3 ORDER BY pk3",
+                    "SELECT pk3 FROM test WHERE v1 = 'a' and v2 = substr('abc',2,1) GROUP BY pk4,CASE WHEN v1 > v2 THEN v1 ELSE v2 END,pk3 ORDER BY pk4,pk3",
+                    "SELECT pk3 FROM test WHERE pk1 = 'a' and v2 = substr('abc',2,1) GROUP BY pk4,CASE WHEN pk1 > v2 THEN pk1 ELSE v2 END,pk3 ORDER BY pk4,pk3",
+                    "SELECT pk3 FROM test WHERE pk1 = 'a' and pk2 = 'b' and v1 = 'c' GROUP BY CASE WHEN pk1 > pk2 THEN v1 WHEN pk1 = pk2 THEN pk1 ELSE pk2 END,pk3 ORDER BY pk3"
+            };
+            int index = 0;
+            for (String query : queries) {
+                QueryPlan plan = getQueryPlan(conn, query);
+                assertTrue((index + 1) + ") " + queries[index], plan.getOrderBy().getOrderByExpressions().isEmpty());
+                index++;
+            }
+        }
+    }
+
+    @Test
+    public void testOrderPreservingGroupByForClientAggregatePlan() throws Exception {
+        Connection conn = null;
+         try {
+             conn = DriverManager.getConnection(getUrl());
+             String tableName = "test_table";
+             String sql = "create table " + tableName + "( "+
+                     " pk1 varchar not null , " +
+                     " pk2 varchar not null, " +
+                     " pk3 varchar not null," +
+                     " v1 varchar, " +
+                     " v2 varchar, " +
+                     " CONSTRAINT TEST_PK PRIMARY KEY ( "+
+                        "pk1,"+
+                        "pk2,"+
+                        "pk3 ))";
+             conn.createStatement().execute(sql);
+
+             String[] queries = new String[] {
+                   "select a.ak3 "+
+                   "from (select substr(pk1,1,1) ak1,substr(pk2,1,1) ak2,substr(pk3,1,1) ak3,substr(v1,1,1) av1,substr(v2,1,1) av2 from "+tableName+" order by pk2,pk3 limit 10) a "+
+                   "group by a.ak3,a.av1 order by a.ak3,a.av1",
+
+                   "select a.ak3 "+
+                   "from (select substr(pk1,1,1) ak1,substr(pk2,1,1) ak2,substr(pk3,1,1) ak3,substr(v1,1,1) av1,substr(v2,1,1) av2 from "+tableName+" order by pk2,pk3 limit 10) a "+
+                   "where a.av2 = 'a' GROUP BY substr(a.av2,0,1),ak3 ORDER BY ak3",
+
+                   //for InListExpression
+                   "select a.ak3 "+
+                   "from (select substr(pk1,1,1) ak1,substr(pk2,1,1) ak2,substr(pk3,1,1) ak3,substr(v1,1,1) av1,substr(v2,1,1) av2 from "+tableName+" order by pk2,pk3 limit 10) a "+
+                   "where a.av2 in('a') GROUP BY substr(a.av2,0,1),ak3 ORDER BY ak3",
+
+                   "select a.ak3 "+
+                   "from (select substr(pk1,1,1) ak1,substr(pk2,1,1) ak2,substr(pk3,1,1) ak3,substr(v1,1,1) av1,substr(v2,1,1) av2 from "+tableName+" order by pk2,pk3 limit 10) a "+
+                   "where a.ak1 = 'c' and a.av2 = substr('abc',1,1) GROUP BY a.av2,a.ak3 ORDER BY a.ak3",
+
+                   //for RVC
+                   "select a.ak3 "+
+                   "from (select substr(pk1,1,1) ak1,substr(pk2,1,1) ak2,substr(pk3,1,1) ak3,substr(v1,1,1) av1,substr(v2,1,1) av2 from "+tableName+" order by pk2,pk3 limit 10) a "+
+                   "where (a.ak1,a.av2) = ('c', substr('abc',1,1)) GROUP BY a.av2,a.ak3 ORDER BY a.ak3",
+
+                   "select a.ak3 "+
+                   "from (select substr(pk1,1,1) ak1,substr(pk2,1,1) ak2,substr(pk3,1,1) ak3,substr(v1,1,1) av1,substr(v2,1,1) av2 from "+tableName+" order by pk2,pk3 limit 10) a "+
+                   "where a.av1 = 'a' and a.av2 = 'b' GROUP BY length(a.av1)+length(a.av2),a.ak3 ORDER BY a.ak3",
+
+                   "select a.ak3 "+
+                   "from (select substr(pk1,1,1) ak1,substr(pk2,1,1) ak2,substr(pk3,1,1) ak3,substr(v1,1,1) av1,substr(v2,1,1) av2 from "+tableName+" order by pk2,pk3 limit 10) a "+
+                   "where a.ak1 = 'a' and a.av2 = 'b' GROUP BY length(a.ak1)+length(a.av2),a.ak3 ORDER BY a.ak3",
+
+                   "select a.ak3 "+
+                   "from (select substr(pk1,1,1) ak1,substr(pk2,1,1) ak2,substr(pk3,1,1) ak3, coalesce(pk3,'1') ak4, substr(v1,1,1) av1,substr(v2,1,1) av2 from "+tableName+" order by pk2,pk3 limit 10) a "+
+                   "where a.av1 = 'a' and a.av2 = substr('abc',2,1) GROUP BY a.ak4,CASE WHEN a.av1 > a.av2 THEN a.av1 ELSE a.av2 END,a.ak3 ORDER BY a.ak4,a.ak3",
+
+                   "select a.ak3 "+
+                   "from (select rand() ak1,length(pk2) ak2,length(pk3) ak3,length(v1) av1,length(v2) av2 from "+tableName+" order by pk2,pk3 limit 10) a "+
+                   "where a.ak1 = 0.0 and a.av2 = (5+3*2) GROUP BY a.ak3,CASE WHEN a.ak1 > a.av2 THEN a.ak1 ELSE a.av2 END,a.av1 ORDER BY a.ak3,a.av1",
+
+                   "select a.ak3 "+
+                   "from (select rand() ak1,length(pk2) ak2,length(pk3) ak3,length(v1) av1,length(v2) av2 from "+tableName+" order by pk2,pk3 limit 10) a "+
+                   "where a.ak1 = 0.0 and a.av2 = length(substr('abc',1,1)) GROUP BY a.ak3,CASE WHEN a.ak1 > a.av2 THEN a.ak1 ELSE a.av2 END,a.av1 ORDER BY a.ak3,a.av1",
+
+                   "select a.ak3 "+
+                   "from (select rand() ak1,length(pk2) ak2,length(pk3) ak3,length(v1) av1,length(v2) av2 from "+tableName+" order by pk2,pk3 limit 10) a "+
+                   "where a.ak1 = 0.0 and a.av2 = length(substr('abc',1,1)) GROUP BY a.ak3,CASE WHEN coalesce(a.ak1,1) > coalesce(a.av2,2) THEN coalesce(a.ak1,1) ELSE coalesce(a.av2,2) END,a.av1 ORDER BY a.ak3,a.av1",
+
+                   //for IS NULL
+                   "select a.ak3 "+
+                   "from (select rand() ak1,length(pk2) ak2,length(pk3) ak3,length(v1) av1,length(v2) av2 from "+tableName+" order by pk2,pk3 limit 10) a "+
+                   "where a.ak1 is null and a.av2 = length(substr('abc',1,1)) GROUP BY a.ak3,CASE WHEN coalesce(a.ak1,1) > coalesce(a.av2,2) THEN coalesce(a.ak1,1) ELSE coalesce(a.av2,2) END,a.av1 ORDER BY a.ak3,a.av1",
+
+                   "select a.ak3 "+
+                   "from (select rand() ak1,length(pk2) ak2,length(pk3) ak3,length(v1) av1,length(v2) av2 from "+tableName+" order by pk2,pk3 limit 10) a "+
+                   "where a.ak1 = 0.0 and a.av2 is null GROUP BY a.ak3,CASE WHEN coalesce(a.ak1,1) > coalesce(a.av2,2) THEN coalesce(a.ak1,1) ELSE coalesce(a.av2,2) END,a.av1 ORDER BY a.ak3,a.av1",
+             };
+              int index = 0;
+             for (String query : queries) {
+                 QueryPlan plan =  TestUtil.getOptimizeQueryPlan(conn, query);
+                 assertTrue((index + 1) + ") " + queries[index], plan.getOrderBy()== OrderBy.FWD_ROW_KEY_ORDER_BY);
+                 index++;
+             }
+         }
+         finally {
+             if(conn != null) {
+                 conn.close();
+             }
+         }
+    }
     
     @Test
     public void testNotOrderPreservingGroupBy() throws Exception {
@@ -2988,6 +3111,207 @@ public class QueryCompilerTest extends BaseConnectionlessQueryTest {
     }
     
     @Test
+    public void testNotOrderPreservingGroupByForNotPkColumns() throws Exception {
+        try (Connection conn= DriverManager.getConnection(getUrl())) {
+            conn.createStatement().execute("CREATE TABLE test (\n" +
+                    "            pk1 varchar,\n" +
+                    "            pk2 varchar,\n" +
+                    "            pk3 varchar,\n" +
+                    "            pk4 varchar,\n" +
+                    "            v1 varchar,\n" +
+                    "            v2 varchar,\n" +
+                    "            CONSTRAINT pk PRIMARY KEY (\n" +
+                    "               pk1,\n" +
+                    "               pk2,\n" +
+                    "               pk3,\n" +
+                    "               pk4\n" +
+                    "             )\n" +
+                    "         )");
+             String[] queries = new String[] {
+                     "SELECT pk3 FROM test WHERE (pk1 = 'a' and pk2 = 'b') or v1 ='c' GROUP BY pk4,CASE WHEN pk1 > pk2 THEN coalesce(v1,'1') ELSE pk2 END,pk3 ORDER BY pk4,pk3",
+                     "SELECT pk3 FROM test WHERE pk1 = 'a' or pk2 = 'b' GROUP BY CASE WHEN pk1 > pk2 THEN v1 WHEN pk1 = pk2 THEN pk1 ELSE pk2 END,pk3 ORDER BY pk3",
+                     "SELECT pk3 FROM test WHERE pk1 = 'a' and (pk2 = 'b' or v1 = 'c') GROUP BY CASE WHEN pk1 > pk2 THEN v1 WHEN pk1 = pk2 THEN pk1 ELSE pk2 END,pk3 ORDER BY pk3",
+                     "SELECT v2 FROM test GROUP BY v1,v2 ORDER BY v2",
+                     "SELECT pk3 FROM test WHERE v1 = 'a' GROUP BY v1,v2,pk3 ORDER BY pk3",
+                     "SELECT length(pk3) FROM test WHERE v1 = 'a' GROUP BY RAND()+length(v1),length(v2),length(pk3) ORDER BY length(v2),length(pk3)",
+                     "SELECT length(pk3) FROM test WHERE v1 = 'a' and v2 = 'b' GROUP BY CASE WHEN v1 > v2 THEN length(v1) ELSE RAND(1) END,length(pk3) ORDER BY length(pk3)",
+             };
+             int index = 0;
+            for (String query : queries) {
+                QueryPlan plan = getQueryPlan(conn, query);
+                assertFalse((index + 1) + ") " + queries[index], plan.getOrderBy().getOrderByExpressions().isEmpty());
+                index++;
+            }
+        }
+    }
+
+    @Test
+    public void testNotOrderPreservingGroupByForClientAggregatePlan() throws Exception {
+        Connection conn = null;
+        try {
+            conn = DriverManager.getConnection(getUrl());
+            String tableName = "table_test";
+            String sql = "create table " + tableName + "( "+
+                    " pk1 varchar not null , " +
+                    " pk2 varchar not null, " +
+                    " pk3 varchar not null," +
+                    " v1 varchar, " +
+                    " v2 varchar, " +
+                    " CONSTRAINT TEST_PK PRIMARY KEY ( "+
+                    "pk1,"+
+                    "pk2,"+
+                    "pk3 ))";
+            conn.createStatement().execute(sql);
+
+            String[] queries = new String[] {
+                  "select a.ak3 "+
+                   "from (select substr(pk1,1,1) ak1,substr(pk2,1,1) ak2,substr(pk3,1,1) ak3,coalesce(pk3,'1') ak4, substr(v1,1,1) av1,substr(v2,1,1) av2 from "+tableName+" order by pk2,pk3 limit 10) a "+
+                   "where (a.ak1 = 'a' and a.ak2 = 'b') or a.av1 ='c' GROUP BY a.ak4,CASE WHEN a.ak1 > a.ak2 THEN coalesce(a.av1,'1') ELSE a.ak2 END,a.ak3 ORDER BY a.ak4,a.ak3",
+
+                   "select a.ak3 "+
+                   "from (select substr(pk1,1,1) ak1,substr(pk2,1,1) ak2,substr(pk3,1,1) ak3,coalesce(pk3,'1') ak4, substr(v1,1,1) av1,substr(v2,1,1) av2 from "+tableName+" order by pk2,pk3 limit 10) a "+
+                   "where a.ak1 = 'a' or a.ak2 = 'b' GROUP BY CASE WHEN a.ak1 > a.ak2 THEN a.av1 WHEN a.ak1 = a.ak2 THEN a.ak1 ELSE a.ak2 END,a.ak3 ORDER BY a.ak3",
+
+                   //for in
+                   "select a.ak3 "+
+                   "from (select substr(pk1,1,1) ak1,substr(pk2,1,1) ak2,substr(pk3,1,1) ak3,coalesce(pk3,'1') ak4, substr(v1,1,1) av1,substr(v2,1,1) av2 from "+tableName+" order by pk2,pk3 limit 10) a "+
+                   "where a.ak1 in ( 'a','b') GROUP BY CASE WHEN a.ak1 > a.ak2 THEN a.av1 WHEN a.ak1 = a.ak2 THEN a.ak1 ELSE a.ak2 END,a.ak3 ORDER BY a.ak3",
+
+                   "select a.ak3 "+
+                   "from (select substr(pk1,1,1) ak1,substr(pk2,1,1) ak2,substr(pk3,1,1) ak3,coalesce(pk3,'1') ak4, substr(v1,1,1) av1,substr(v2,1,1) av2 from "+tableName+" order by pk2,pk3 limit 10) a "+
+                   "where a.ak1 = 'a' and (a.ak2 = 'b' or a.av1 = 'c') GROUP BY CASE WHEN a.ak1 > a.ak2 THEN a.av1 WHEN a.ak1 = a.ak2 THEN a.ak1 ELSE a.ak2 END,a.ak3 ORDER BY a.ak3",
+
+                   "select a.av2 "+
+                   "from (select substr(pk1,1,1) ak1,substr(pk2,1,1) ak2,substr(pk3,1,1) ak3,coalesce(pk3,'1') ak4, substr(v1,1,1) av1,substr(v2,1,1) av2 from "+tableName+" order by pk2,pk3 limit 10) a "+
+                   "GROUP BY a.av1,a.av2 ORDER BY a.av2",
+
+                   "select a.ak3 "+
+                   "from (select substr(pk1,1,1) ak1,substr(pk2,1,1) ak2,substr(pk3,1,1) ak3,coalesce(pk3,'1') ak4, substr(v1,1,1) av1,substr(v2,1,1) av2 from "+tableName+" order by pk2,pk3 limit 10) a "+
+                   "where a.av1 = 'a' GROUP BY a.av1,a.av2,a.ak3 ORDER BY a.ak3",
+
+                   "select length(a.ak3) "+
+                   "from (select substr(pk1,1,1) ak1,substr(pk2,1,1) ak2,substr(pk3,1,1) ak3,coalesce(pk3,'1') ak4, substr(v1,1,1) av1,substr(v2,1,1) av2 from "+tableName+" order by pk2,pk3 limit 10) a "+
+                   "where a.av1 = 'a' GROUP BY RAND()+length(a.av1),length(a.av2),length(a.ak3) ORDER BY length(a.av2),length(a.ak3)",
+
+                   "select length(a.ak3) "+
+                   "from (select substr(pk1,1,1) ak1,substr(pk2,1,1) ak2,substr(pk3,1,1) ak3,coalesce(pk3,'1') ak4, substr(v1,1,1) av1,substr(v2,1,1) av2 from "+tableName+" order by pk2,pk3 limit 10) a "+
+                   "where a.av1 = 'a' and a.av2 = 'b' GROUP BY CASE WHEN a.av1 > a.av2 THEN length(a.av1) ELSE RAND(1) END,length(a.ak3) ORDER BY length(a.ak3)",
+
+                   "select a.ak3 "+
+                   "from (select rand() ak1,length(pk2) ak2,length(pk3) ak3,length(v1) av1,length(v2) av2 from "+tableName+" order by pk2,pk3 limit 10) a "+
+                   "where a.ak1 > 0.0 and a.av2 = (5+3*2) GROUP BY a.ak3,CASE WHEN a.ak1 > a.av2 THEN a.ak1 ELSE a.av2 END,a.av1 ORDER BY a.ak3,a.av1",
+
+                   //for CoerceExpression
+                   "select a.ak3 "+
+                   "from (select rand() ak1,length(pk2) ak2,length(pk3) ak3,length(v1) av1,length(v2) av2 from "+tableName+" order by pk2,pk3 limit 10) a "+
+                   "where CAST(a.ak1 AS INTEGER) = 0 and a.av2 = (5+3*2) GROUP BY a.ak3,a.ak1,a.av1 ORDER BY a.ak3,a.av1",
+
+                   "select a.ak3 "+
+                   "from (select rand() ak1,length(pk2) ak2,length(pk3) ak3,length(v1) av1,length(v2) av2 from "+tableName+" order by pk2,pk3 limit 10) a "+
+                   "where a.ak1 = 0.0 or a.av2 = length(substr('abc',1,1)) GROUP BY a.ak3,CASE WHEN coalesce(a.ak1,1) > coalesce(a.av2,2) THEN coalesce(a.ak1,1) ELSE coalesce(a.av2,2) END,a.av1 ORDER BY a.ak3,a.av1",
+
+                   //for IS NULL
+                   "select a.ak3 "+
+                   "from (select rand() ak1,length(pk2) ak2,length(pk3) ak3,length(v1) av1,length(v2) av2 from "+tableName+" order by pk2,pk3 limit 10) a "+
+                   "where a.ak1 is not null and a.av2 = length(substr('abc',1,1)) GROUP BY a.ak3,CASE WHEN coalesce(a.ak1,1) > coalesce(a.av2,2) THEN coalesce(a.ak1,1) ELSE coalesce(a.av2,2) END,a.av1 ORDER BY a.ak3,a.av1",
+
+                   "select a.ak3 "+
+                   "from (select rand() ak1,length(pk2) ak2,length(pk3) ak3,length(v1) av1,length(v2) av2 from "+tableName+" order by pk2,pk3 limit 10) a "+
+                   "where a.ak1 is null or a.av2 = length(substr('abc',1,1)) GROUP BY a.ak3,CASE WHEN coalesce(a.ak1,1) > coalesce(a.av2,2) THEN coalesce(a.ak1,1) ELSE coalesce(a.av2,2) END,a.av1 ORDER BY a.ak3,a.av1",
+
+                   "select a.ak3 "+
+                   "from (select rand() ak1,length(pk2) ak2,length(pk3) ak3,length(v1) av1,length(v2) av2 from "+tableName+" order by pk2,pk3 limit 10) a "+
+                   "where a.ak1 is null and a.av2 = length(substr('abc',1,1)) and a.ak1 = 0.0 GROUP BY a.ak3,CASE WHEN coalesce(a.ak1,1) > coalesce(a.av2,2) THEN coalesce(a.ak1,1) ELSE coalesce(a.av2,2) END,a.av1 ORDER BY a.ak3,a.av1",
+
+                   "select a.ak3 "+
+                   "from (select rand() ak1,length(pk2) ak2,length(pk3) ak3,length(v1) av1,length(v2) av2 from "+tableName+" order by pk2,pk3 limit 10) a "+
+                   "where a.ak1 is null and a.av2 = length(substr('abc',1,1)) or a.ak1 = 0.0 GROUP BY a.ak3,CASE WHEN coalesce(a.ak1,1) > coalesce(a.av2,2) THEN coalesce(a.ak1,1) ELSE coalesce(a.av2,2) END,a.av1 ORDER BY a.ak3,a.av1",
+             };
+            int index = 0;
+            for (String query : queries) {
+                QueryPlan plan = TestUtil.getOptimizeQueryPlan(conn, query);
+                assertTrue((index + 1) + ") " + queries[index], plan.getOrderBy().getOrderByExpressions().size() > 0);
+                index++;
+            }
+        }
+        finally {
+            if(conn != null) {
+                conn.close();
+            }
+        }
+    }
+
+    @Test
+    public void testOrderByOptimizeForClientAggregatePlanAndDesc() throws Exception {
+        Connection conn = null;
+        try {
+            conn = DriverManager.getConnection(getUrl());
+            String tableName = "test_table";
+            String sql = "create table " + tableName + "( "+
+                    " pk1 varchar not null, " +
+                    " pk2 varchar not null, " +
+                    " pk3 varchar not null, " +
+                    " v1 varchar, " +
+                    " v2 varchar, " +
+                    " CONSTRAINT TEST_PK PRIMARY KEY ( "+
+                    "pk1 desc,"+
+                    "pk2 desc,"+
+                    "pk3 desc))";
+            conn.createStatement().execute(sql);
+
+            String[] queries = new String[] {
+                     "select a.ak3 "+
+                     "from (select pk1 ak1,pk2 ak2,pk3 ak3, substr(v1,1,1) av1,substr(v2,1,1) av2 from "+tableName+" order by pk2,pk3 limit 10) a "+
+                     "group by a.ak3,a.av1 order by a.ak3 desc,a.av1",
+
+                     "select a.ak3 "+
+                     "from (select pk1 ak1,pk2 ak2,pk3 ak3,substr(v1,1,1) av1,substr(v2,1,1) av2 from "+tableName+" order by pk2,pk3 limit 10) a "+
+                     "where a.av1 = 'a' group by a.av1,a.ak3 order by a.ak3 desc",
+
+                     "select a.ak3 "+
+                     "from (select pk1 ak1,pk2 ak2,pk3 ak3,substr(v1,1,1) av1,substr(v2,1,1) av2 from "+tableName+" order by pk2,pk3 limit 10) a "+
+                     "where a.av1 = 'a' and a.av2= 'b' group by CASE WHEN a.av1 > a.av2 THEN a.av1 ELSE a.av2 END,a.ak3,a.ak2 order by a.ak3 desc,a.ak2 desc"
+            };
+
+            int index = 0;
+            for (String query : queries) {
+                QueryPlan plan =  TestUtil.getOptimizeQueryPlan(conn, query);
+                assertTrue((index + 1) + ") " + queries[index], plan.getOrderBy()== OrderBy.FWD_ROW_KEY_ORDER_BY);
+                index++;
+            }
+
+            queries = new String[] {
+                     "select a.ak3 "+
+                     "from (select pk1 ak1,pk2 ak2,pk3 ak3,substr(v1,1,1) av1,substr(v2,1,1) av2 from "+tableName+" order by pk2,pk3 limit 10) a "+
+                     "group by a.ak3,a.av1 order by a.ak3,a.av1",
+
+                     "select a.ak3 "+
+                     "from (select pk1 ak1,pk2 ak2,pk3 ak3,substr(v1,1,1) av1,substr(v2,1,1) av2 from "+tableName+" order by pk2,pk3 limit 10) a "+
+                     "where a.av1 = 'a' group by a.av1,a.ak3 order by a.ak3",
+
+                     "select a.ak3 "+
+                     "from (select pk1 ak1,pk2 ak2,pk3 ak3,substr(v1,1,1) av1,substr(v2,1,1) av2 from "+tableName+" order by pk2,pk3 limit 10) a "+
+                     "where a.av1 = 'a' and a.av2= 'b' group by CASE WHEN a.av1 > a.av2 THEN a.av1 ELSE a.av2 END,a.ak3,a.ak2 order by a.ak3,a.ak2",
+
+                     "select a.ak3 "+
+                     "from (select pk1 ak1,pk2 ak2,pk3 ak3,substr(v1,1,1) av1,substr(v2,1,1) av2 from "+tableName+" order by pk2,pk3 limit 10) a "+
+                     "where a.av1 = 'a' and a.av2= 'b' group by CASE WHEN a.av1 > a.av2 THEN a.av1 ELSE a.av2 END,a.ak3,a.ak2 order by a.ak3 asc,a.ak2 desc"
+            };
+            index = 0;
+            for (String query : queries) {
+                QueryPlan plan = TestUtil.getOptimizeQueryPlan(conn, query);
+                assertTrue((index + 1) + ") " + queries[index], plan.getOrderBy().getOrderByExpressions().size() > 0);
+                index++;
+            }
+        }
+        finally {
+            if(conn != null) {
+                conn.close();
+            }
+        }
+    }
+
+    @Test
     public void testGroupByDescColumnBug3451() throws Exception {
 
         try (Connection conn= DriverManager.getConnection(getUrl())) {
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/expression/ArithmeticOperationTest.java b/phoenix-core/src/test/java/org/apache/phoenix/expression/ArithmeticOperationTest.java
index 7876fce..1b830f2 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/expression/ArithmeticOperationTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/expression/ArithmeticOperationTest.java
@@ -30,7 +30,7 @@ import java.util.List;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.phoenix.exception.DataExceedsCapacityException;
 import org.apache.phoenix.expression.function.RandomFunction;
-import org.apache.phoenix.expression.visitor.CloneNonDeterministicExpressionVisitor;
+import org.apache.phoenix.expression.visitor.CloneExpressionVisitor;
 import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.types.PDecimal;
 import org.apache.phoenix.schema.types.PInteger;
@@ -273,7 +273,7 @@ public class ArithmeticOperationTest {
         e2 = new DoubleSubtractExpression(children);
         e3 = new DoubleAddExpression(Arrays.<Expression>asList(e1, e2));
         e4 = new DoubleAddExpression(Arrays.<Expression>asList(new RandomFunction(Arrays.<Expression>asList(LiteralExpression.newConstant(null))), e3));
-        CloneNonDeterministicExpressionVisitor visitor = new CloneNonDeterministicExpressionVisitor();
+        CloneExpressionVisitor visitor = new CloneExpressionVisitor();
         Expression clone = e4.accept(visitor);
         assertTrue(clone != e4);
         e4.evaluate(null, ptr1);
@@ -281,7 +281,7 @@ public class ArithmeticOperationTest {
         assertNotEquals(ptr1, ptr2);
         
         e4 = new DoubleAddExpression(Arrays.<Expression>asList(new RandomFunction(Arrays.<Expression>asList(LiteralExpression.newConstant(1))), e3));
-        visitor = new CloneNonDeterministicExpressionVisitor();
+        visitor = new CloneExpressionVisitor();
         clone = e4.accept(visitor);
         assertTrue(clone == e4);
         e4.evaluate(null, ptr1);
@@ -294,7 +294,7 @@ public class ArithmeticOperationTest {
         boolean evaluated = e.evaluate(null, ptr);
         assertTrue(evaluated);
         assertEquals(value, type.toObject(ptr.get()));
-        CloneNonDeterministicExpressionVisitor visitor = new CloneNonDeterministicExpressionVisitor();
+        CloneExpressionVisitor visitor = new CloneExpressionVisitor();
         Expression clone = e.accept(visitor);
         evaluated = clone.evaluate(null, ptr);
         assertTrue(evaluated);


[phoenix] 18/18: PHOENIX-5059 Use the Datasource v2 api in the spark connector

Posted by pb...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

pboado pushed a commit to branch 4.x-cdh5.16
in repository https://gitbox.apache.org/repos/asf/phoenix.git

commit b129be998de7eac2bd8b3bf78f5feb7812b8f642
Author: Thomas D'Silva <td...@apache.org>
AuthorDate: Tue Dec 11 22:59:39 2018 +0000

    PHOENIX-5059 Use the Datasource v2 api in the spark connector
---
 .../phoenix/end2end/salted/BaseSaltedTableIT.java  |   6 +-
 phoenix-spark/pom.xml                              |   8 +
 .../java/org/apache/phoenix/spark/OrderByIT.java   | 117 ++---
 .../java/org/apache/phoenix/spark/SparkUtil.java   |  45 +-
 phoenix-spark/src/it/resources/globalSetup.sql     |   6 +-
 .../phoenix/spark/AbstractPhoenixSparkIT.scala     |  12 +-
 .../org/apache/phoenix/spark/PhoenixSparkIT.scala  | 543 +++++++++++----------
 .../spark/PhoenixSparkITTenantSpecific.scala       |  18 +-
 .../spark/datasource/v2/PhoenixDataSource.java     |  82 ++++
 .../v2/reader/PhoenixDataSourceReadOptions.java    |  51 ++
 .../v2/reader/PhoenixDataSourceReader.java         | 201 ++++++++
 .../v2/reader/PhoenixInputPartition.java           |  44 ++
 .../v2/reader/PhoenixInputPartitionReader.java     | 168 +++++++
 .../v2/writer/PhoenixDataSourceWriteOptions.java   | 109 +++++
 .../datasource/v2/writer/PhoenixDataWriter.java    | 100 ++++
 .../v2/writer/PhoenixDataWriterFactory.java        |  19 +
 .../v2/writer/PhoenixDatasourceWriter.java         |  34 ++
 ...org.apache.spark.sql.sources.DataSourceRegister |   1 +
 .../apache/phoenix/spark/ConfigurationUtil.scala   |   1 +
 .../apache/phoenix/spark/DataFrameFunctions.scala  |   2 +-
 .../org/apache/phoenix/spark/DefaultSource.scala   |   1 +
 ...lation.scala => FilterExpressionCompiler.scala} | 109 ++---
 .../org/apache/phoenix/spark/PhoenixRDD.scala      |  61 +--
 .../phoenix/spark/PhoenixRecordWritable.scala      |   2 +-
 .../org/apache/phoenix/spark/PhoenixRelation.scala |  70 +--
 .../apache/phoenix/spark/ProductRDDFunctions.scala |   1 +
 .../phoenix/spark/SparkContextFunctions.scala      |   1 +
 .../org/apache/phoenix/spark/SparkSchemaUtil.scala |  84 ++++
 .../phoenix/spark/SparkSqlContextFunctions.scala   |   1 +
 .../datasources/jdbc/PhoenixJdbcDialect.scala      |  21 +
 .../execution/datasources/jdbc/SparkJdbcUtil.scala | 309 ++++++++++++
 31 files changed, 1664 insertions(+), 563 deletions(-)

diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/salted/BaseSaltedTableIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/salted/BaseSaltedTableIT.java
index 3051cd6..ef127ac 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/salted/BaseSaltedTableIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/salted/BaseSaltedTableIT.java
@@ -194,7 +194,7 @@ public abstract class BaseSaltedTableIT extends ParallelStatsDisabledIT  {
                 .setSelectColumns(
                         Lists.newArrayList("A_INTEGER", "A_STRING", "A_ID", "B_STRING", "B_INTEGER"))
                 .setFullTableName(tableName)
-                .setWhereClause("a_integer = 1 AND a_string >= 'ab' AND a_string < 'de' AND a_id = '123'");
+                .setWhereClause("A_INTEGER = 1 AND A_STRING >= 'ab' AND A_STRING < 'de' AND A_ID = '123'");
             rs = executeQuery(conn, queryBuilder);
             assertTrue(rs.next());
             assertEquals(1, rs.getInt(1));
@@ -205,7 +205,7 @@ public abstract class BaseSaltedTableIT extends ParallelStatsDisabledIT  {
             assertFalse(rs.next());
 
             // all single slots with one value.
-            queryBuilder.setWhereClause("a_integer = 1 AND a_string = 'ab' AND a_id = '123'");
+            queryBuilder.setWhereClause("A_INTEGER = 1 AND A_STRING = 'ab' AND A_ID = '123'");
             rs = executeQuery(conn, queryBuilder);
             assertTrue(rs.next());
             assertEquals(1, rs.getInt(1));
@@ -216,7 +216,7 @@ public abstract class BaseSaltedTableIT extends ParallelStatsDisabledIT  {
             assertFalse(rs.next());
 
             // all single slots with multiple values.
-            queryBuilder.setWhereClause("a_integer in (2, 4) AND a_string = 'abc' AND a_id = '123'");
+            queryBuilder.setWhereClause("A_INTEGER in (2, 4) AND A_STRING = 'abc' AND A_ID = '123'");
             rs = executeQuery(conn, queryBuilder);
 
             assertTrue(rs.next());
diff --git a/phoenix-spark/pom.xml b/phoenix-spark/pom.xml
index e2790bd..9cc3c3d 100644
--- a/phoenix-spark/pom.xml
+++ b/phoenix-spark/pom.xml
@@ -487,6 +487,14 @@
     <testSourceDirectory>src/it/scala</testSourceDirectory>
     <testResources><testResource><directory>src/it/resources</directory></testResource></testResources>
     <plugins>
+        <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-compiler-plugin</artifactId>
+            <configuration>
+                <source>1.8</source>
+                <target>1.8</target>
+            </configuration>
+        </plugin>
       <plugin>
         <groupId>org.codehaus.mojo</groupId>
         <artifactId>build-helper-maven-plugin</artifactId>
diff --git a/phoenix-spark/src/it/java/org/apache/phoenix/spark/OrderByIT.java b/phoenix-spark/src/it/java/org/apache/phoenix/spark/OrderByIT.java
index 83578ba..1257c43 100644
--- a/phoenix-spark/src/it/java/org/apache/phoenix/spark/OrderByIT.java
+++ b/phoenix-spark/src/it/java/org/apache/phoenix/spark/OrderByIT.java
@@ -16,12 +16,13 @@ import java.util.List;
 import java.util.Properties;
 
 import org.apache.phoenix.end2end.BaseOrderByIT;
+import org.apache.phoenix.spark.datasource.v2.PhoenixDataSource;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.QueryBuilder;
-import org.apache.spark.sql.DataFrame;
+import org.apache.spark.sql.Dataset;
 import org.apache.spark.sql.Row;
 import org.apache.spark.sql.SQLContext;
-import org.junit.Ignore;
+import org.apache.spark.sql.sources.v2.DataSourceOptions;
 import org.junit.Test;
 
 import com.google.common.collect.Lists;
@@ -31,28 +32,6 @@ import scala.collection.JavaConverters;
 
 public class OrderByIT extends BaseOrderByIT {
 
-    @Ignore(" || operator not supported in order by Spark 1.6 ")
-    @Test
-    @Override
-    public void testDescMultiOrderByExpr() throws Exception {
-        super.testDescMultiOrderByExpr();
-    }
-
-    @Ignore("NULLS FIRST|LAST not supported in Spark 1.6")
-    @Test
-    @Override
-    public void testNullsLastWithDesc() throws Exception {
-        super.testNullsLastWithDesc();
-    }
-
-    @Ignore("NULLS FIRST|LAST not supported in Spark 1.6")
-    @Test
-    @Override
-    public void testOrderByReverseOptimizationWithNullsLast() throws Exception {
-        super.testOrderByReverseOptimizationWithNullsLast();
-    }
-
-
     @Override
     protected ResultSet executeQueryThrowsException(Connection conn, QueryBuilder queryBuilder,
                                                     String expectedPhoenixExceptionMsg, String expectedSparkExceptionMsg) {
@@ -128,25 +107,20 @@ public class OrderByIT extends BaseOrderByIT {
 
             // create two PhoenixRDDs  using the table names and columns that are required for the JOIN query
             List<String> table1Columns = Lists.newArrayList("A_STRING", "CF1.A", "CF1.B", "COL1", "CF2.C", "CF2.D");
-            SQLContext sqlContext = SparkUtil.getSqlContext();
-            DataFrame phoenixDataSet =
-                    new PhoenixRDD(SparkUtil.getSparkContext(), tableName1,
-                            JavaConverters.asScalaBufferConverter(table1Columns).asScala().toSeq(),
-                            Option.apply((String) null), Option.apply(getUrl()), config, false,
-                            null).toDataFrame(sqlContext);
-            phoenixDataSet.registerTempTable(tableName1);
-            List<String> table2Columns = Lists.newArrayList("A_STRING", "COL1");
-            phoenixDataSet =
-                    new PhoenixRDD(SparkUtil.getSparkContext(), tableName2,
-                            JavaConverters.asScalaBufferConverter(table2Columns).asScala().toSeq(),
-                            Option.apply((String) null), Option.apply(getUrl()), config, false,
-                            null).toDataFrame(sqlContext);
-            phoenixDataSet.registerTempTable(tableName2);
+            SQLContext sqlContext = SparkUtil.getSparkSession().sqlContext();
+            Dataset phoenixDataSet = SparkUtil.getSparkSession().read().format("phoenix")
+                    .option(DataSourceOptions.TABLE_KEY, tableName1)
+                    .option(PhoenixDataSource.ZOOKEEPER_URL, getUrl()).load();
+            phoenixDataSet.createOrReplaceTempView(tableName1);
+            phoenixDataSet = SparkUtil.getSparkSession().read().format("phoenix")
+                    .option(DataSourceOptions.TABLE_KEY, tableName2)
+                    .option(PhoenixDataSource.ZOOKEEPER_URL, getUrl()).load();
+            phoenixDataSet.createOrReplaceTempView(tableName2);
 
             String query =
                     "SELECT T1.* FROM " + tableName1 + " T1 JOIN " + tableName2
                             + " T2 ON T1.A_STRING = T2.A_STRING ORDER BY T1.`CF1.B`";
-            DataFrame dataset =
+            Dataset<Row> dataset =
                     sqlContext.sql(query);
             List<Row> rows = dataset.collectAsList();
             ResultSet rs = new SparkResultSet(rows, dataset.columns());
@@ -175,8 +149,8 @@ public class OrderByIT extends BaseOrderByIT {
             assertFalse(rs.next());
 
             query =
-                    "select t1.a_string, t2.col1 from " + tableName1 + " t1 join " + tableName2
-                            + " t2 on t1.a_string = t2.a_string order by t2.col1";
+                    "SELECT T1.A_STRING, T2.COL1 FROM " + tableName1 + " T1 JOIN " + tableName2
+                            + " T2 ON T1.A_STRING = T2.A_STRING ORDER BY T2.COL1";
             dataset =  sqlContext.sql(query);
             rows = dataset.collectAsList();
             rs = new SparkResultSet(rows, dataset.columns());
@@ -193,7 +167,6 @@ public class OrderByIT extends BaseOrderByIT {
         }
     }
 
-    @Ignore("Not passing on CDH 4.15")
     @Test
     public void testOrderByWithUnionAll() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
@@ -249,26 +222,20 @@ public class OrderByIT extends BaseOrderByIT {
             conn.commit();
 
 
-            List<String> table1Columns = Lists.newArrayList("A_STRING", "CF1.A", "CF1.B", "COL1", "CF2.C", "CF2.D");
-            SQLContext sqlContext = SparkUtil.getSqlContext();
-            DataFrame phoenixDataSet =
-                    new PhoenixRDD(SparkUtil.getSparkContext(), tableName1,
-                            JavaConverters.asScalaBufferConverter(table1Columns).asScala().toSeq(),
-                            Option.apply((String) null), Option.apply(getUrl()), config, false,
-                            null).toDataFrame(sqlContext);
-            phoenixDataSet.registerTempTable(tableName1);
-            List<String> table2Columns = Lists.newArrayList("A_STRING", "COL1");
-            phoenixDataSet =
-                    new PhoenixRDD(SparkUtil.getSparkContext(), tableName2,
-                            JavaConverters.asScalaBufferConverter(table2Columns).asScala().toSeq(),
-                            Option.apply((String) null), Option.apply(getUrl()), config, false,
-                            null).toDataFrame(sqlContext);
-            phoenixDataSet.registerTempTable(tableName2);
+            SQLContext sqlContext = SparkUtil.getSparkSession().sqlContext();
+            Dataset phoenixDataSet = SparkUtil.getSparkSession().read().format("phoenix")
+                    .option(DataSourceOptions.TABLE_KEY, tableName1)
+                    .option(PhoenixDataSource.ZOOKEEPER_URL, getUrl()).load();
+            phoenixDataSet.createOrReplaceTempView(tableName1);
+            phoenixDataSet = SparkUtil.getSparkSession().read().format("phoenix")
+                    .option(DataSourceOptions.TABLE_KEY, tableName2)
+                    .option(PhoenixDataSource.ZOOKEEPER_URL, getUrl()).load();
+            phoenixDataSet.createOrReplaceTempView(tableName2);
 
             String query =
                     "select a_string, `cf2.d` from " + tableName1 + " union all select * from "
                             + tableName2 + " order by `cf2.d`";
-            DataFrame dataset =
+            Dataset<Row> dataset =
                     sqlContext.sql(query);
             List<Row> rows = dataset.collectAsList();
             ResultSet rs = new SparkResultSet(rows, dataset.columns());
@@ -330,15 +297,12 @@ public class OrderByIT extends BaseOrderByIT {
             stmt.execute();
             conn.commit();
 
-            SQLContext sqlContext = SparkUtil.getSqlContext();
-            DataFrame phoenixDataSet =
-                    new PhoenixRDD(SparkUtil.getSparkContext(), tableName,
-                            JavaConverters.asScalaBufferConverter(Lists.newArrayList("col1", "col2", "col4")).asScala().toSeq(),
-                            Option.apply((String) null), Option.apply(getUrl()), config, false,
-                            null).toDataFrame(sqlContext);
-
-            phoenixDataSet.registerTempTable(tableName);
-            DataFrame dataset =
+            SQLContext sqlContext = SparkUtil.getSparkSession().sqlContext();
+            Dataset phoenixDataSet = SparkUtil.getSparkSession().read().format("phoenix")
+                    .option(DataSourceOptions.TABLE_KEY, tableName)
+                    .option(PhoenixDataSource.ZOOKEEPER_URL, getUrl()).load();
+            phoenixDataSet.createOrReplaceTempView(tableName);
+            Dataset<Row> dataset =
                     sqlContext.sql("SELECT col1+col2, col4, a_string FROM " + tableName
                             + " ORDER BY col1+col2, col4");
             List<Row> rows = dataset.collectAsList();
@@ -395,19 +359,12 @@ public class OrderByIT extends BaseOrderByIT {
             conn.commit();
 
 
-            List<String> columns =
-                    Lists.newArrayList("A_STRING", "CF1.A", "CF1.B", "COL1", "CF2.C", "CF2.D",
-                        "COL2");
-
-            SQLContext sqlContext = SparkUtil.getSqlContext();
-            DataFrame phoenixDataSet =
-                    new PhoenixRDD(SparkUtil.getSparkContext(), tableName,
-                            JavaConverters.asScalaBufferConverter(columns).asScala().toSeq(),
-                            Option.apply((String) null), Option.apply(url), config, false, null)
-                                    .toDataFrame(sqlContext);
-
-            phoenixDataSet.registerTempTable(tableName);
-            DataFrame dataset =
+            SQLContext sqlContext = SparkUtil.getSparkSession().sqlContext();
+            Dataset phoenixDataSet = SparkUtil.getSparkSession().read().format("phoenix")
+                    .option(DataSourceOptions.TABLE_KEY, tableName)
+                    .option(PhoenixDataSource.ZOOKEEPER_URL, getUrl()).load();
+            phoenixDataSet.createOrReplaceTempView(tableName);
+            Dataset<Row> dataset =
                     sqlContext.sql("SELECT A_STRING, `CF1.A`, `CF1.B`, COL1, `CF2.C`, `CF2.D`, COL2 from "
                             + tableName + " ORDER BY `CF1.A`,`CF2.C`");
             List<Row> rows = dataset.collectAsList();
diff --git a/phoenix-spark/src/it/java/org/apache/phoenix/spark/SparkUtil.java b/phoenix-spark/src/it/java/org/apache/phoenix/spark/SparkUtil.java
index db2fe1a..668c3c8 100644
--- a/phoenix-spark/src/it/java/org/apache/phoenix/spark/SparkUtil.java
+++ b/phoenix-spark/src/it/java/org/apache/phoenix/spark/SparkUtil.java
@@ -22,13 +22,14 @@ import com.google.common.base.Joiner;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.spark.datasource.v2.PhoenixDataSource;
 import org.apache.phoenix.util.QueryBuilder;
-import org.apache.spark.SparkConf;
-import org.apache.spark.SparkContext;
-import org.apache.spark.sql.DataFrame;
+import org.apache.spark.sql.Dataset;
 import org.apache.spark.sql.Row;
 import org.apache.spark.sql.SQLContext;
+import org.apache.spark.sql.SparkSession;
 import org.apache.spark.sql.execution.SparkPlan;
+import org.apache.spark.sql.sources.v2.DataSourceOptions;
 import scala.Option;
 import scala.collection.JavaConverters;
 
@@ -42,33 +43,15 @@ public class SparkUtil {
     public static final String APP_NAME = "Java Spark Tests";
     public static final String NUM_EXECUTORS = "local[2]";
     public static final String UI_SHOW_CONSOLE_PROGRESS = "spark.ui.showConsoleProgress";
-    public static final String CASE_SENSITIVE_COLUMNS = "spark.sql.caseSensitive";
 
-    private static SparkContext sparkContext = null;
-    private static SQLContext sqlContext = null;
-
-    public static SparkContext getSparkContext() {
-        if (sparkContext == null) {
-            SparkConf conf = new SparkConf(true);
-            conf.setAppName(APP_NAME);
-            conf.setMaster(NUM_EXECUTORS);
-            conf.set(UI_SHOW_CONSOLE_PROGRESS, "false");
-            conf.set(CASE_SENSITIVE_COLUMNS, "false");
-            sparkContext = new SparkContext(conf);
-        }
-        return sparkContext;
-    }
-
-    public static SQLContext getSqlContext() {
-        if (sqlContext == null) {
-            sqlContext = new SQLContext(getSparkContext());
-        }
-        return sqlContext;
+    public static SparkSession getSparkSession() {
+        return SparkSession.builder().appName(APP_NAME).master(NUM_EXECUTORS)
+                .config(UI_SHOW_CONSOLE_PROGRESS, false).getOrCreate();
     }
 
     public static ResultSet executeQuery(Connection conn, QueryBuilder queryBuilder, String url, Configuration config)
             throws SQLException {
-        SQLContext sqlContext = SparkUtil.getSqlContext();
+        SQLContext sqlContext = getSparkSession().sqlContext();
 
         boolean forceRowKeyOrder =
                 conn.unwrap(PhoenixConnection.class).getQueryServices().getProps()
@@ -82,14 +65,12 @@ public class SparkUtil {
 
         // create PhoenixRDD using the table name and columns that are required by the query
         // since we don't set the predicate filtering is done after rows are returned from spark
-        DataFrame phoenixDataSet =
-                new PhoenixRDD(SparkUtil.getSparkContext(), queryBuilder.getFullTableName(),
-                        JavaConverters.asScalaBufferConverter(queryBuilder.getRequiredColumns()).asScala().toSeq(),
-                        Option.apply((String) null), Option.apply(url), config, false,
-                        null).toDataFrame(sqlContext);
+        Dataset phoenixDataSet = getSparkSession().read().format("phoenix")
+                .option(DataSourceOptions.TABLE_KEY, queryBuilder.getFullTableName())
+                .option(PhoenixDataSource.ZOOKEEPER_URL, url).load();
 
-        phoenixDataSet.registerTempTable(queryBuilder.getFullTableName());
-        DataFrame dataset = sqlContext.sql(queryBuilder.build());
+        phoenixDataSet.createOrReplaceTempView(queryBuilder.getFullTableName());
+        Dataset<Row> dataset = sqlContext.sql(queryBuilder.build());
         SparkPlan plan = dataset.queryExecution().executedPlan();
         List<Row> rows = dataset.collectAsList();
         queryBuilder.setOrderByClause(prevOrderBy);
diff --git a/phoenix-spark/src/it/resources/globalSetup.sql b/phoenix-spark/src/it/resources/globalSetup.sql
index 7ac0039..efdb8cb 100644
--- a/phoenix-spark/src/it/resources/globalSetup.sql
+++ b/phoenix-spark/src/it/resources/globalSetup.sql
@@ -26,9 +26,9 @@ UPSERT INTO table2 (id, table1_id, "t2col1") VALUES (3, 2, 'test_child_1')
 UPSERT INTO table2 (id, table1_id, "t2col1") VALUES (4, 2, 'test_child_2')
 UPSERT INTO table2 (id, table1_id, "t2col1") VALUES (5, 2, 'test_child_3')
 UPSERT INTO table2 (id, table1_id, "t2col1") VALUES (6, 2, 'test_child_4')
-CREATE TABLE "table3" ("id" BIGINT NOT NULL PRIMARY KEY, "col1" VARCHAR)
-UPSERT INTO "table3" ("id", "col1") VALUES (1, 'foo')
-UPSERT INTO "table3" ("id", "col1") VALUES (2, 'bar')
+CREATE TABLE "table4" ("id" BIGINT NOT NULL PRIMARY KEY, "col1" VARCHAR)
+UPSERT INTO "table4" ("id", "col1") VALUES (1, 'foo')
+UPSERT INTO "table4" ("id", "col1") VALUES (2, 'bar')
 CREATE TABLE ARRAY_TEST_TABLE (ID BIGINT NOT NULL PRIMARY KEY, VCARRAY VARCHAR[])
 UPSERT INTO ARRAY_TEST_TABLE (ID, VCARRAY) VALUES (1, ARRAY['String1', 'String2', 'String3'])
 CREATE TABLE ARRAYBUFFER_TEST_TABLE (ID BIGINT NOT NULL PRIMARY KEY, VCARRAY VARCHAR[], INTARRAY INTEGER[])
diff --git a/phoenix-spark/src/it/scala/org/apache/phoenix/spark/AbstractPhoenixSparkIT.scala b/phoenix-spark/src/it/scala/org/apache/phoenix/spark/AbstractPhoenixSparkIT.scala
index ca3470f..a9c2070 100644
--- a/phoenix-spark/src/it/scala/org/apache/phoenix/spark/AbstractPhoenixSparkIT.scala
+++ b/phoenix-spark/src/it/scala/org/apache/phoenix/spark/AbstractPhoenixSparkIT.scala
@@ -19,6 +19,7 @@ import java.util.Properties
 import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT
 import org.apache.phoenix.query.BaseTest
 import org.apache.phoenix.util.PhoenixRuntime
+import org.apache.spark.sql.{SQLContext, SparkSession}
 import org.apache.spark.{SparkConf, SparkContext}
 import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSuite, Matchers}
 
@@ -50,7 +51,7 @@ class AbstractPhoenixSparkIT extends FunSuite with Matchers with BeforeAndAfter
   final val TenantId = "theTenant"
 
   var conn: Connection = _
-  var sc: SparkContext = _
+  var spark: SparkSession = _
 
   lazy val hbaseConfiguration = {
     val conf = PhoenixSparkITHelper.getTestClusterConfig
@@ -99,12 +100,17 @@ class AbstractPhoenixSparkIT extends FunSuite with Matchers with BeforeAndAfter
       .setMaster("local[2]") // 2 threads, some parallelism
       .set("spark.ui.showConsoleProgress", "false") // Disable printing stage progress
 
-    sc = new SparkContext(conf)
+    spark = SparkSession
+      .builder()
+      .appName("PhoenixSparkIT")
+      .master("local[2]") // 2 threads, some parallelism
+      .config("spark.ui.showConsoleProgress", "false")
+      .getOrCreate()
   }
 
   override def afterAll() {
     conn.close()
-    sc.stop()
+    spark.stop()
     PhoenixSparkITHelper.cleanUpAfterTest()
     PhoenixSparkITHelper.doTeardown
   }
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 fb4bb64..6bef721 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
@@ -13,16 +13,16 @@
  */
 package org.apache.phoenix.spark
 
+import java.sql.DriverManager
 import java.util.Date
 
 import org.apache.phoenix.schema.types.PVarchar
+import org.apache.phoenix.spark.datasource.v2.PhoenixDataSource
 import org.apache.phoenix.util.{ColumnInfo, SchemaUtil}
 import org.apache.spark.sql.types._
-import org.apache.spark.sql.{Row, SQLContext, SaveMode}
-import org.joda.time.DateTime
-import org.apache.spark.{SparkConf, SparkContext}
+import org.apache.spark.sql.{Row, SaveMode}
+
 import scala.collection.mutable.ListBuffer
-import org.apache.hadoop.conf.Configuration
 
 /**
   * Note: If running directly from an IDE, these are the recommended VM parameters:
@@ -30,13 +30,20 @@ import org.apache.hadoop.conf.Configuration
   */
 class PhoenixSparkIT extends AbstractPhoenixSparkIT {
 
-  test("Can persist data with case senstive columns (like in avro schema) using 'DataFrame.saveToPhoenix'") {
-    val sqlContext = new SQLContext(sc)
-    val df = sqlContext.createDataFrame(
+  test("Can persist data with case sensitive columns (like in avro schema)") {
+    val df = spark.createDataFrame(
       Seq(
         (1, 1, "test_child_1"),
-        (2, 1, "test_child_2"))).toDF("ID", "TABLE3_ID", "t2col1")
-    df.saveToPhoenix("TABLE3", zkUrl = Some(quorumAddress),skipNormalizingIdentifier=true)
+        (2, 1, "test_child_2"))).
+    // column names are case sensitive
+      toDF("ID", "TABLE3_ID", "t2col1")
+    df.write
+      .format("phoenix")
+      .options(Map("table" -> "TABLE3",
+        PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress, PhoenixDataSource.SKIP_NORMALIZING_IDENTIFIER -> "true"))
+      .mode(SaveMode.Overwrite)
+      .save()
+
 
     // Verify results
     val stmt = conn.createStatement()
@@ -50,37 +57,55 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT {
     stmt.close()
 
     results.toList shouldEqual checkResults
+  }
+
+  // INSERT is not support using DataSource v2 api yet
+  ignore("Can use write data using spark SQL INSERT") {
+    val df1 = spark.sqlContext.read.format("phoenix")
+      .options( Map("table" -> "TABLE3", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load
+    df1.createOrReplaceTempView("TABLE3")
+
+    // Insert data
+    spark.sql("INSERT INTO TABLE3 VALUES(10, 10, 10)")
+    spark.sql("INSERT INTO TABLE3 VALUES(20, 20, 20)")
+
+    // Verify results
+    val stmt = conn.createStatement()
+    val rs = stmt.executeQuery("SELECT * FROM TABLE3 WHERE ID>=10")
+    val expectedResults = List((10, 10, "10"), (20, 20, "20"))
+    val results = ListBuffer[(Long, Long, String)]()
+    while (rs.next()) {
+      results.append((rs.getLong(1), rs.getLong(2), rs.getString(3)))
+    }
+    stmt.close()
 
+    results.toList shouldEqual expectedResults
   }
-  
+
   test("Can convert Phoenix schema") {
     val phoenixSchema = List(
       new ColumnInfo("varcharColumn", PVarchar.INSTANCE.getSqlType)
     )
 
-    val rdd = new PhoenixRDD(sc, "MyTable", Array("Foo", "Bar"),
-      conf = hbaseConfiguration)
+    val catalystSchema = SparkSchemaUtil.phoenixSchemaToCatalystSchema(phoenixSchema)
 
-    val catalystSchema = rdd.phoenixSchemaToCatalystSchema(phoenixSchema)
-
-    val expected = List(StructField("varcharColumn", StringType, nullable = true))
+    val expected = new StructType(List(StructField("varcharColumn", StringType, nullable = true)).toArray)
 
     catalystSchema shouldEqual expected
   }
 
   test("Can create schema RDD and execute query") {
-    val sqlContext = new SQLContext(sc)
-
-    val df1 = sqlContext.phoenixTableAsDataFrame("TABLE1", Array("ID", "COL1"), conf = hbaseConfiguration)
+    val df1 = spark.sqlContext.read.format("phoenix")
+      .options( Map("table" -> "TABLE1", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load
 
-    df1.registerTempTable("sql_table_1")
+    df1.createOrReplaceTempView("sql_table_1")
 
-    val df2 = sqlContext.phoenixTableAsDataFrame("TABLE2", Array("ID", "TABLE1_ID"),
-      conf = hbaseConfiguration)
+    val df2 = spark.sqlContext.read.format("phoenix")
+      .options( Map("table" -> "TABLE2", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load
 
-    df2.registerTempTable("sql_table_2")
+    df2.createOrReplaceTempView("sql_table_2")
 
-    val sqlRdd = sqlContext.sql(
+    val sqlRdd = spark.sql(
       """
         |SELECT t1.ID, t1.COL1, t2.ID, t2.TABLE1_ID FROM sql_table_1 AS t1
         |INNER JOIN sql_table_2 AS t2 ON (t2.TABLE1_ID = t1.ID)""".stripMargin
@@ -91,18 +116,49 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT {
     count shouldEqual 6L
   }
 
-  test("Can create schema RDD and execute query on case sensitive table (no config)") {
-    val sqlContext = new SQLContext(sc)
+  ignore("Ordering by pk columns should not require sorting") {
+    val df1 = spark.sqlContext.read.format("phoenix")
+      .options( Map("table" -> "TABLE1", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load
+    df1.createOrReplaceTempView("TABLE1")
+
+    val sqlRdd = spark.sql("SELECT * FROM TABLE1 ORDER BY ID, COL1")
+    val plan = sqlRdd.queryExecution.sparkPlan
+    // verify the spark plan doesn't have a sort
+    assert(!plan.toString.contains("Sort"))
 
+    val expectedResults = Array(Row.fromSeq(Seq(1, "test_row_1")), Row.fromSeq(Seq(2, "test_row_2")))
+    val actual = sqlRdd.collect()
 
-    val df1 = sqlContext.phoenixTableAsDataFrame(
-      SchemaUtil.getEscapedArgument("table3"),
-      Array("id", "col1"),
-      zkUrl = Some(quorumAddress))
+    actual shouldEqual expectedResults
+  }
 
-    df1.registerTempTable("table3")
+  test("Verify correct number of partitions are created") {
+    val conn = DriverManager.getConnection(PhoenixSparkITHelper.getUrl)
+    val ddl = "CREATE TABLE SPLIT_TABLE (id VARCHAR NOT NULL PRIMARY KEY, val VARCHAR) split on ('e','j','o')"
+    conn.createStatement.execute(ddl)
+    val keys = Array("a", "b", "c", "d", "e", "f", "g", "h", "i", "j", "k", "l", "m", "n", "o", "p", "q", "r", "s",
+      "t", "u", "v", "w", "x", "y", "z")
+    for (key <- keys) {
+      conn.createStatement.execute("UPSERT INTO SPLIT_TABLE VALUES('" + key + "', '" + key + "')")
+    }
+    conn.commit()
 
-    val sqlRdd = sqlContext.sql("SELECT * FROM table3")
+    val df1 = spark.sqlContext.read.format("phoenix")
+      .options( Map("table" -> "SPLIT_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load
+    df1.createOrReplaceTempView("SPLIT_TABLE")
+    val sqlRdd = spark.sql("SELECT * FROM SPLIT_TABLE")
+    val numPartitions = sqlRdd.rdd.partitions.size
+
+    numPartitions shouldEqual 4
+  }
+
+  test("Can create schema RDD and execute query on case sensitive table (no config)") {
+    val df1 = spark.sqlContext.read.format("phoenix")
+      .options( Map("table" -> SchemaUtil.getEscapedArgument("table4"), PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load
+
+    df1.createOrReplaceTempView("table4")
+
+    val sqlRdd = spark.sql("SELECT id FROM table4")
 
     val count = sqlRdd.count()
 
@@ -110,20 +166,17 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT {
   }
 
   test("Can create schema RDD and execute constrained query") {
-    val sqlContext = new SQLContext(sc)
-
-    val df1 = sqlContext.phoenixTableAsDataFrame("TABLE1", Array("ID", "COL1"),
-      conf = hbaseConfiguration)
+    val df1 = spark.sqlContext.read.format("phoenix")
+      .options( Map("table" -> "TABLE1", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load
 
-    df1.registerTempTable("sql_table_1")
+    df1.createOrReplaceTempView("sql_table_1")
 
-    val df2 = sqlContext.phoenixTableAsDataFrame("TABLE2", Array("ID", "TABLE1_ID"),
-      predicate = Some("\"ID\" = 1"),
-      conf = hbaseConfiguration)
+    val df2 = spark.sqlContext.read.format("phoenix")
+      .options( Map("table" -> "TABLE2", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load.filter("ID = 1")
 
-    df2.registerTempTable("sql_table_2")
+    df2.createOrReplaceTempView("sql_table_2")
 
-    val sqlRdd = sqlContext.sql(
+    val sqlRdd = spark.sql(
       """
         |SELECT t1.ID, t1.COL1, t2.ID, t2.TABLE1_ID FROM sql_table_1 AS t1
         |INNER JOIN sql_table_2 AS t2 ON (t2.TABLE1_ID = t1.ID)""".stripMargin
@@ -135,17 +188,12 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT {
   }
 
   test("Can create schema RDD with predicate that will never match") {
-    val sqlContext = new SQLContext(sc)
+    val df1 = spark.sqlContext.read.format("phoenix")
+      .options( Map("table" -> "TABLE1", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load.filter("ID = -1")
 
-    val df1 = sqlContext.phoenixTableAsDataFrame(
-      SchemaUtil.getEscapedArgument("table3"),
-      Array("id", "col1"),
-      predicate = Some("\"id\" = -1"),
-      conf = hbaseConfiguration)
+    df1.createOrReplaceTempView("table3")
 
-    df1.registerTempTable("table3")
-
-    val sqlRdd = sqlContext.sql("SELECT * FROM table3")
+    val sqlRdd = spark.sql("SELECT * FROM table3")
 
     val count = sqlRdd.count()
 
@@ -153,21 +201,17 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT {
   }
 
   test("Can create schema RDD with complex predicate") {
-    val sqlContext = new SQLContext(sc)
-
-    val df1 = sqlContext.phoenixTableAsDataFrame(
-      "DATE_PREDICATE_TEST_TABLE",
-      Array("ID", "TIMESERIES_KEY"),
-      predicate = Some(
-        """
-          |ID > 0 AND TIMESERIES_KEY BETWEEN
-          |CAST(TO_DATE('1990-01-01 00:00:01', 'yyyy-MM-dd HH:mm:ss') AS TIMESTAMP) AND
-          |CAST(TO_DATE('1990-01-30 00:00:01', 'yyyy-MM-dd HH:mm:ss') AS TIMESTAMP)""".stripMargin),
-      conf = hbaseConfiguration)
+    val predicate = "ID > 0 AND TIMESERIES_KEY BETWEEN " +
+      "CAST(TO_DATE('1990-01-01 00:00:01', 'yyyy-MM-dd HH:mm:ss') AS TIMESTAMP) AND " +
+      "CAST(TO_DATE('1990-01-30 00:00:01', 'yyyy-MM-dd HH:mm:ss') AS TIMESTAMP)"
+    val df1 = spark.sqlContext.read.format("phoenix")
+      .options(Map("table" -> "DATE_PREDICATE_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress))
+      .load
+      .filter(predicate)
 
-    df1.registerTempTable("date_predicate_test_table")
+    df1.createOrReplaceTempView("date_predicate_test_table")
 
-    val sqlRdd = df1.sqlContext.sql("SELECT * FROM date_predicate_test_table")
+    val sqlRdd = spark.sqlContext.sql("SELECT * FROM date_predicate_test_table")
 
     val count = sqlRdd.count()
 
@@ -175,14 +219,12 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT {
   }
 
   test("Can query an array table") {
-    val sqlContext = new SQLContext(sc)
-
-    val df1 = sqlContext.phoenixTableAsDataFrame("ARRAY_TEST_TABLE", Array("ID", "VCARRAY"),
-      conf = hbaseConfiguration)
+    val df1 = spark.sqlContext.read.format("phoenix")
+      .options( Map("table" -> "ARRAY_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load
 
-    df1.registerTempTable("ARRAY_TEST_TABLE")
+    df1.createOrReplaceTempView("ARRAY_TEST_TABLE")
 
-    val sqlRdd = sqlContext.sql("SELECT * FROM ARRAY_TEST_TABLE")
+    val sqlRdd = spark.sql("SELECT * FROM ARRAY_TEST_TABLE")
 
     val count = sqlRdd.count()
 
@@ -195,12 +237,12 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT {
   }
 
   test("Can read a table as an RDD") {
-    val rdd1 = sc.phoenixTableAsRDD("ARRAY_TEST_TABLE", Seq("ID", "VCARRAY"),
-      conf = hbaseConfiguration)
+    val rdd1 = spark.sqlContext.read.format("phoenix")
+      .options( Map("table" -> "ARRAY_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load
 
     val count = rdd1.count()
 
-    val arrayValues = rdd1.take(1)(0)("VCARRAY")
+    val arrayValues = rdd1.take(1)(0)(1)
 
     arrayValues should equal(Array("String1", "String2", "String3"))
 
@@ -208,24 +250,30 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT {
   }
 
   test("Can save to phoenix table") {
-    val sqlContext = new SQLContext(sc)
+    val dataSet = List(Row(1L, "1", 1), Row(2L, "2", 2), Row(3L, "3", 3))
 
-    val dataSet = List((1L, "1", 1), (2L, "2", 2), (3L, "3", 3))
+    val schema = StructType(
+      Seq(StructField("ID", LongType, nullable = false),
+        StructField("COL1", StringType),
+        StructField("COL2", IntegerType)))
 
-    sc
-      .parallelize(dataSet)
-      .saveToPhoenix(
-        "OUTPUT_TEST_TABLE",
-        Seq("ID", "COL1", "COL2"),
-        hbaseConfiguration
-      )
+    val rowRDD = spark.sparkContext.parallelize(dataSet)
+
+    // Apply the schema to the RDD.
+    val df = spark.sqlContext.createDataFrame(rowRDD, schema)
+
+    df.write
+      .format("phoenix")
+      .options(Map("table" -> "OUTPUT_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress))
+      .mode(SaveMode.Overwrite)
+      .save()
 
     // Load the results back
     val stmt = conn.createStatement()
     val rs = stmt.executeQuery("SELECT ID, COL1, COL2 FROM OUTPUT_TEST_TABLE")
-    val results = ListBuffer[(Long, String, Int)]()
+    val results = ListBuffer[Row]()
     while (rs.next()) {
-      results.append((rs.getLong(1), rs.getString(2), rs.getInt(3)))
+      results.append(Row(rs.getLong(1), rs.getString(2), rs.getInt(3)))
     }
 
     // Verify they match
@@ -234,18 +282,29 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT {
     }
   }
 
-  test("Can save Java and Joda dates to Phoenix (no config)") {
-    val dt = new DateTime()
-    val date = new Date()
+  test("Can save dates to Phoenix using java.sql.Date") {
+    val date = java.sql.Date.valueOf("2016-09-30")
 
-    val dataSet = List((1L, "1", 1, dt), (2L, "2", 2, date))
-    sc
-      .parallelize(dataSet)
-      .saveToPhoenix(
-        "OUTPUT_TEST_TABLE",
-        Seq("ID", "COL1", "COL2", "COL3"),
-        zkUrl = Some(quorumAddress)
-      )
+    // Since we are creating a Row we have to use java.sql.date
+    // java.util.date or joda.DateTime is not supported
+    val dataSet = Seq(Row(1L, "1", 1, date), Row(2L, "2", 2, date))
+
+    val schema = StructType(
+      Seq(StructField("ID", LongType, nullable = false),
+        StructField("COL1", StringType),
+        StructField("COL2", IntegerType),
+        StructField("COL3", DateType)))
+
+    val rowRDD = spark.sparkContext.parallelize(dataSet)
+
+    // Apply the schema to the RDD.
+    val df = spark.sqlContext.createDataFrame(rowRDD, schema)
+
+    df.write
+      .format("phoenix")
+      .options(Map("table" -> "OUTPUT_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress))
+      .mode(SaveMode.Overwrite)
+      .save()
 
     // Load the results back
     val stmt = conn.createStatement()
@@ -256,94 +315,56 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT {
     }
 
     // Verify the epochs are equal
-    results(0).getTime shouldEqual dt.getMillis
+    results(0).getTime shouldEqual date.getTime
     results(1).getTime shouldEqual date.getTime
   }
 
   test("Can infer schema without defining columns") {
-    val sqlContext = new SQLContext(sc)
-    val df = sqlContext.phoenixTableAsDataFrame("TABLE2", Seq(), conf = hbaseConfiguration)
+    val df = spark.sqlContext.read.format("phoenix")
+      .options( Map("table" -> "TABLE2", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load()
     df.schema("ID").dataType shouldEqual LongType
     df.schema("TABLE1_ID").dataType shouldEqual LongType
     df.schema("t2col1").dataType shouldEqual StringType
   }
 
   test("Spark SQL can use Phoenix as a data source with no schema specified") {
-    val sqlContext = new SQLContext(sc)
-    val df = sqlContext.load("org.apache.phoenix.spark", Map("table" -> "TABLE1",
-      "zkUrl" -> quorumAddress))
+    val df = spark.sqlContext.read.format("phoenix")
+      .options( Map("table" -> "TABLE1", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load
     df.count() shouldEqual 2
     df.schema("ID").dataType shouldEqual LongType
     df.schema("COL1").dataType shouldEqual StringType
   }
 
-  ignore("Spark SQL can use Phoenix as a data source with PrunedFilteredScan") {
-    val sqlContext = new SQLContext(sc)
-    val df = sqlContext.load("org.apache.phoenix.spark", Map("table" -> "TABLE1",
-      "zkUrl" -> quorumAddress))
+  test("Datasource v2 pushes down filters") {
+    val df = spark.sqlContext.read.format("phoenix")
+      .options( Map("table" -> "TABLE1", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load
     val res = df.filter(df("COL1") === "test_row_1" && df("ID") === 1L).select(df("ID"))
 
     // Make sure we got the right value back
     assert(res.first().getLong(0) == 1L)
 
     val plan = res.queryExecution.sparkPlan
-    // filters should be pushed into phoenix relation
-    assert(plan.toString.contains("PushedFilters: [*IsNotNull(COL1), *IsNotNull(ID), " +
-      "*EqualTo(COL1,test_row_1), *EqualTo(ID,1)]"))
-    // spark should run the filters on the rows returned by Phoenix
-    assert(!plan.toString.matches(".*Filter (((isnotnull(COL1.*) && isnotnull(ID.*)) "
-      + " && (COL1.* = test_row_1)) && (ID.* = 1)).*"))
+    // filters should be pushed into scan
+    assert(".*ScanV2 phoenix.*Filters.*ID.*COL1.*".r.findFirstIn(plan.toString).isDefined)
+    // spark should not do post scan filtering
+    assert(".*Filter .*ID.*COL1.*".r.findFirstIn(plan.toString).isEmpty)
   }
 
-  test("Can persist a dataframe using 'DataFrame.saveToPhoenix'") {
+  test("Can persist a dataframe") {
     // Load from TABLE1
-    val sqlContext = new SQLContext(sc)
-    val df = sqlContext.load("org.apache.phoenix.spark", Map("table" -> "TABLE1",
-      "zkUrl" -> quorumAddress))
+    val df = spark.sqlContext.read.format("phoenix").options( Map("table" -> "TABLE1", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load
 
     // Save to TABLE1_COPY
-    df.saveToPhoenix("TABLE1_COPY", zkUrl = Some(quorumAddress))
-
-    // Verify results
-    val stmt = conn.createStatement()
-    val rs = stmt.executeQuery("SELECT * FROM TABLE1_COPY")
-
-    val checkResults = List((1L, "test_row_1"), (2, "test_row_2"))
-    val results = ListBuffer[(Long, String)]()
-    while (rs.next()) {
-      results.append((rs.getLong(1), rs.getString(2)))
-    }
-    stmt.close()
-
-    results.toList shouldEqual checkResults
-  }
-
-  test("Can persist a dataframe using 'DataFrame.save()") {
-    // Clear TABLE1_COPY
-    var stmt = conn.createStatement()
-    stmt.executeUpdate("DELETE FROM TABLE1_COPY")
-    stmt.close()
-
-    // Load TABLE1, save as TABLE1_COPY
-    val sqlContext = new SQLContext(sc)
-    val df = sqlContext
-      .read
-      .format("org.apache.phoenix.spark")
-      .option("table", "TABLE1")
-      .option("zkUrl", quorumAddress)
-      .load()
-
-    // Save to TABLE21_COPY
     df
       .write
-      .format("org.apache.phoenix.spark")
+      .format("phoenix")
       .mode(SaveMode.Overwrite)
       .option("table", "TABLE1_COPY")
-      .option("zkUrl", quorumAddress)
+      .option(PhoenixDataSource.ZOOKEEPER_URL, quorumAddress)
       .save()
 
     // Verify results
-    stmt = conn.createStatement()
+    val stmt = conn.createStatement()
     val rs = stmt.executeQuery("SELECT * FROM TABLE1_COPY")
 
     val checkResults = List((1L, "test_row_1"), (2, "test_row_2"))
@@ -357,15 +378,22 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT {
   }
 
   test("Can save arrays back to phoenix") {
-    val dataSet = List((2L, Array("String1", "String2", "String3")))
+    val dataSet = List(Row(2L, Array("String1", "String2", "String3")))
+    val schema = StructType(Seq(
+      StructField("ID", LongType, nullable = false),
+      StructField("VCARRAY", ArrayType(StringType, true))
+    ))
 
-    sc
-      .parallelize(dataSet)
-      .saveToPhoenix(
-        "ARRAY_TEST_TABLE",
-        Seq("ID", "VCARRAY"),
-        zkUrl = Some(quorumAddress)
-      )
+    val rowRDD = spark.sparkContext.parallelize(dataSet)
+
+    // Apply the schema to the RDD.
+    val df = spark.sqlContext.createDataFrame(rowRDD, schema)
+
+    df.write
+      .format("phoenix")
+      .options(Map("table" -> "ARRAY_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress))
+      .mode(SaveMode.Overwrite)
+      .save()
 
     // Load the results back
     val stmt = conn.createStatement()
@@ -374,57 +402,54 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT {
     val sqlArray = rs.getArray(1).getArray().asInstanceOf[Array[String]]
 
     // Verify the arrays are equal
-    sqlArray shouldEqual dataSet(0)._2
+    sqlArray shouldEqual dataSet(0).get(1)
   }
 
   test("Can read from table with schema and escaped table name") {
     // Manually escape
-    val rdd1 = sc.phoenixTableAsRDD(
-      "CUSTOM_ENTITY.\"z02\"",
-      Seq("ID"),
-      conf = hbaseConfiguration)
+    val df1 = spark.sqlContext.read.format("phoenix")
+      .options(Map("table" -> "CUSTOM_ENTITY.\"z02\"", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load()
 
-    var count = rdd1.count()
+    var count = df1.count()
 
     count shouldEqual 1L
 
     // Use SchemaUtil
-    val rdd2 = sc.phoenixTableAsRDD(
-      SchemaUtil.getEscapedFullTableName("CUSTOM_ENTITY.z02"),
-      Seq("ID"),
-      conf = hbaseConfiguration)
+    val df2 = spark.sqlContext.read.format("phoenix")
+      .options(
+        Map("table" -> SchemaUtil.getEscapedFullTableName("CUSTOM_ENTITY.z02"), PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress))
+      .load()
 
-    count = rdd2.count()
+    count = df2.count()
 
     count shouldEqual 1L
-
   }
 
   test("Ensure DataFrame field normalization (PHOENIX-2196)") {
-    val rdd1 = sc
+    val rdd1 = spark.sparkContext
       .parallelize(Seq((1L, 1L, "One"), (2L, 2L, "Two")))
       .map(p => Row(p._1, p._2, p._3))
 
-    val sqlContext = new SQLContext(sc)
-
     val schema = StructType(Seq(
       StructField("id", LongType, nullable = false),
       StructField("table1_id", LongType, nullable = true),
       StructField("\"t2col1\"", StringType, nullable = true)
     ))
 
-    val df = sqlContext.createDataFrame(rdd1, schema)
+    val df = spark.sqlContext.createDataFrame(rdd1, schema)
 
-    df.saveToPhoenix("TABLE2", zkUrl = Some(quorumAddress))
+    df.write
+      .format("phoenix")
+      .options(Map("table" -> "TABLE2", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress))
+      .mode(SaveMode.Overwrite)
+      .save()
   }
 
   test("Ensure Dataframe supports LIKE and IN filters (PHOENIX-2328)") {
-    val sqlContext = new SQLContext(sc)
-    val df = sqlContext.load("org.apache.phoenix.spark", Map("table" -> "TABLE1",
-      "zkUrl" -> quorumAddress))
-
+    val df = spark.sqlContext.read.format("phoenix").options(Map("table" -> "TABLE1", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load()
     // Prefix match
     val res1 = df.filter("COL1 like 'test_row_%'")
+    val plan = res1.groupBy().count().queryExecution.sparkPlan
     res1.count() shouldEqual 2
 
     // Suffix match
@@ -463,14 +488,14 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT {
   }
 
   test("Can load decimal types with accurate precision and scale (PHOENIX-2288)") {
-    val sqlContext = new SQLContext(sc)
-    val df = sqlContext.load("org.apache.phoenix.spark", Map("table" -> "TEST_DECIMAL", "zkUrl" -> quorumAddress))
+    val df = spark.sqlContext.read.format("phoenix")
+      .options(Map("table" -> "TEST_DECIMAL", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load()
     assert(df.select("COL1").first().getDecimal(0) == BigDecimal("123.456789").bigDecimal)
   }
 
-  test("Can load small and tiny integeger types (PHOENIX-2426)") {
-    val sqlContext = new SQLContext(sc)
-    val df = sqlContext.load("org.apache.phoenix.spark", Map("table" -> "TEST_SMALL_TINY", "zkUrl" -> quorumAddress))
+  test("Can load small and tiny integer types (PHOENIX-2426)") {
+    val df = spark.sqlContext.read.format("phoenix")
+      .options(Map("table" -> "TEST_SMALL_TINY", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress)).load()
     assert(df.select("COL1").first().getShort(0).toInt == 32767)
     assert(df.select("COL2").first().getByte(0).toInt == 127)
   }
@@ -478,21 +503,19 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT {
   test("Can save arrays from custom dataframes back to phoenix") {
     val dataSet = List(Row(2L, Array("String1", "String2", "String3"), Array(1, 2, 3)))
 
-    val sqlContext = new SQLContext(sc)
-
     val schema = StructType(
       Seq(StructField("ID", LongType, nullable = false),
         StructField("VCARRAY", ArrayType(StringType)),
         StructField("INTARRAY", ArrayType(IntegerType))))
 
-    val rowRDD = sc.parallelize(dataSet)
+    val rowRDD = spark.sparkContext.parallelize(dataSet)
 
     // Apply the schema to the RDD.
-    val df = sqlContext.createDataFrame(rowRDD, schema)
+    val df = spark.sqlContext.createDataFrame(rowRDD, schema)
 
     df.write
-      .format("org.apache.phoenix.spark")
-      .options(Map("table" -> "ARRAYBUFFER_TEST_TABLE", "zkUrl" -> quorumAddress))
+      .format("phoenix")
+      .options(Map("table" -> "ARRAYBUFFER_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress))
       .mode(SaveMode.Overwrite)
       .save()
 
@@ -509,15 +532,23 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT {
   }
 
   test("Can save arrays of AnyVal type back to phoenix") {
-    val dataSet = List((2L, Array(1, 2, 3), Array(1L, 2L, 3L)))
+    val dataSet = List(Row(2L, Array(1, 2, 3), Array(1L, 2L, 3L)))
 
-    sc
-      .parallelize(dataSet)
-      .saveToPhoenix(
-        "ARRAY_ANYVAL_TEST_TABLE",
-        Seq("ID", "INTARRAY", "BIGINTARRAY"),
-        zkUrl = Some(quorumAddress)
-      )
+    val schema = StructType(
+      Seq(StructField("ID", LongType, nullable = false),
+        StructField("INTARRAY", ArrayType(IntegerType)),
+        StructField("BIGINTARRAY", ArrayType(LongType))))
+
+    val rowRDD = spark.sparkContext.parallelize(dataSet)
+
+    // Apply the schema to the RDD.
+    val df = spark.sqlContext.createDataFrame(rowRDD, schema)
+
+    df.write
+      .format("phoenix")
+      .options(Map("table" -> "ARRAY_ANYVAL_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress))
+      .mode(SaveMode.Overwrite)
+      .save()
 
     // Load the results back
     val stmt = conn.createStatement()
@@ -527,20 +558,27 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT {
     val longArray = rs.getArray(2).getArray().asInstanceOf[Array[Long]]
 
     // Verify the arrays are equal
-    intArray shouldEqual dataSet(0)._2
-    longArray shouldEqual dataSet(0)._3
+    intArray shouldEqual dataSet(0).get(1)
+    longArray shouldEqual dataSet(0).get(2)
   }
 
   test("Can save arrays of Byte type back to phoenix") {
-    val dataSet = List((2L, Array(1.toByte, 2.toByte, 3.toByte)))
+    val dataSet = List(Row(2L, Array(1.toByte, 2.toByte, 3.toByte)))
 
-    sc
-      .parallelize(dataSet)
-      .saveToPhoenix(
-        "ARRAY_BYTE_TEST_TABLE",
-        Seq("ID", "BYTEARRAY"),
-        zkUrl = Some(quorumAddress)
-      )
+    val schema = StructType(
+      Seq(StructField("ID", LongType, nullable = false),
+        StructField("BYTEARRAY", ArrayType(ByteType))))
+
+    val rowRDD = spark.sparkContext.parallelize(dataSet)
+
+    // Apply the schema to the RDD.
+    val df = spark.sqlContext.createDataFrame(rowRDD, schema)
+
+    df.write
+      .format("phoenix")
+      .options(Map("table" -> "ARRAY_BYTE_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress))
+      .mode(SaveMode.Overwrite)
+      .save()
 
     // Load the results back
     val stmt = conn.createStatement()
@@ -549,19 +587,28 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT {
     val byteArray = rs.getArray(1).getArray().asInstanceOf[Array[Byte]]
 
     // Verify the arrays are equal
-    byteArray shouldEqual dataSet(0)._2
+    byteArray shouldEqual dataSet(0).get(1)
   }
 
   test("Can save binary types back to phoenix") {
-    val dataSet = List((2L, Array[Byte](1), Array[Byte](1, 2, 3), Array[Array[Byte]](Array[Byte](1), Array[Byte](2))))
+    val dataSet = List(Row(2L, Array[Byte](1), Array[Byte](1, 2, 3), Array[Array[Byte]](Array[Byte](1), Array[Byte](2))))
+
+    val schema = StructType(
+      Seq(StructField("ID", LongType, false),
+        StructField("BIN", BinaryType),
+        StructField("VARBIN", BinaryType),
+        StructField("BINARRAY", ArrayType(BinaryType))))
+
+    val rowRDD = spark.sparkContext.parallelize(dataSet)
+
+    // Apply the schema to the RDD.
+    val df = spark.sqlContext.createDataFrame(rowRDD, schema)
 
-    sc
-      .parallelize(dataSet)
-      .saveToPhoenix(
-        "VARBINARY_TEST_TABLE",
-        Seq("ID", "BIN", "VARBIN", "BINARRAY"),
-        zkUrl = Some(quorumAddress)
-      )
+    df.write
+      .format("phoenix")
+      .options(Map("table" -> "VARBINARY_TEST_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress))
+      .mode(SaveMode.Overwrite)
+      .save()
 
     // Load the results back
     val stmt = conn.createStatement()
@@ -572,16 +619,15 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT {
     val varByteArray = rs.getArray("BINARRAY").getArray().asInstanceOf[Array[Array[Byte]]]
 
     // Verify the arrays are equal
-    byte shouldEqual dataSet(0)._2
-    varByte shouldEqual dataSet(0)._3
-    varByteArray shouldEqual dataSet(0)._4
+    byte shouldEqual dataSet(0).get(1)
+    varByte shouldEqual dataSet(0).get(2)
+    varByteArray shouldEqual dataSet(0).get(3)
   }
 
   test("Can load Phoenix DATE columns through DataFrame API") {
-    val sqlContext = new SQLContext(sc)
-    val df = sqlContext.read
-      .format("org.apache.phoenix.spark")
-      .options(Map("table" -> "DATE_TEST", "zkUrl" -> quorumAddress))
+    val df = spark.sqlContext.read
+      .format("phoenix")
+      .options(Map("table" -> "DATE_TEST", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress))
       .load
     val dt = df.select("COL1").first().getDate(0).getTime
     val epoch = new Date().getTime
@@ -595,37 +641,37 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT {
   }
 
   test("Filter operation doesn't work for column names containing a white space (PHOENIX-2547)") {
-    val sqlContext = new SQLContext(sc)
-    val df = sqlContext.load("org.apache.phoenix.spark", Map("table" -> SchemaUtil.getEscapedArgument("space"),
-      "zkUrl" -> quorumAddress))
+    val df = spark.sqlContext.read.format("phoenix")
+      .options(Map("table" -> SchemaUtil.getEscapedArgument("space"), PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress))
+      .load
     val res = df.filter(df.col("first name").equalTo("xyz"))
     // Make sure we got the right value back
     assert(res.collectAsList().size() == 1L)
   }
 
   test("Spark Phoenix cannot recognize Phoenix view fields (PHOENIX-2290)") {
-    val sqlContext = new SQLContext(sc)
-    val df = sqlContext.load("org.apache.phoenix.spark", Map("table" -> SchemaUtil.getEscapedArgument("small"),
-      "zkUrl" -> quorumAddress))
-    df.registerTempTable("temp")
+    val df = spark.sqlContext.read.format("phoenix")
+      .options(Map("table" -> SchemaUtil.getEscapedArgument("small"), PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress))
+      .load
+    df.createOrReplaceTempView("temp")
 
     // limitation: filter / where expressions are not allowed with "double quotes", instead of that pass it as column expressions
     // reason: if the expression contains "double quotes" then spark sql parser, ignoring evaluating .. giving to next level to handle
 
-    val res1 = sqlContext.sql("select * from temp where salary = '10000' ")
+    val res1 = spark.sql("select * from temp where salary = '10000' ")
     assert(res1.collectAsList().size() == 1L)
 
-    val res2 = sqlContext.sql("select * from temp where \"salary\" = '10000' ")
+    val res2 = spark.sql("select * from temp where \"salary\" = '10000' ")
     assert(res2.collectAsList().size() == 0L)
 
-    val res3 = sqlContext.sql("select * from temp where salary > '10000' ")
+    val res3 = spark.sql("select * from temp where salary > '10000' ")
     assert(res3.collectAsList().size() == 2L)
   }
 
   test("Queries with small case column-names return empty result-set when working with Spark Datasource Plugin (PHOENIX-2336)") {
-    val sqlContext = new SQLContext(sc)
-    val df = sqlContext.load("org.apache.phoenix.spark", Map("table" -> SchemaUtil.getEscapedArgument("small"),
-      "zkUrl" -> quorumAddress))
+    val df = spark.sqlContext.read.format("phoenix")
+      .options(Map("table" -> SchemaUtil.getEscapedArgument("small"), PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress))
+      .load
 
     // limitation: filter / where expressions are not allowed with "double quotes", instead of that pass it as column expressions
     // reason: if the expression contains "double quotes" then spark sql parser, ignoring evaluating .. giving to next level to handle
@@ -644,10 +690,9 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT {
   }
 
   test("Can coerce Phoenix DATE columns to TIMESTAMP through DataFrame API") {
-    val sqlContext = new SQLContext(sc)
-    val df = sqlContext.read
-      .format("org.apache.phoenix.spark")
-      .options(Map("table" -> "DATE_TEST", "zkUrl" -> quorumAddress, "dateAsTimestamp" -> "true"))
+    val df = spark.sqlContext.read
+      .format("phoenix")
+      .options(Map("table" -> "DATE_TEST", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress, "dateAsTimestamp" -> "true"))
       .load
     val dtRes = df.select("COL1").first()
     val ts = dtRes.getTimestamp(0).getTime
@@ -657,10 +702,9 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT {
   }
 
   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))
+    val df = spark.sqlContext.read
+      .format("phoenix")
+      .options(Map("table" -> "TIME_TEST", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress))
       .load
     val time = df.select("COL1").first().getTimestamp(0).getTime
     val epoch = new Date().getTime
@@ -668,13 +712,14 @@ class PhoenixSparkIT extends AbstractPhoenixSparkIT {
   }
 
   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))
+    val df = spark.sqlContext.read
+      .format("phoenix")
+      .options(Map("table" -> "GIGANTIC_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress))
+      .load
 
     df.write
-      .format("org.apache.phoenix.spark")
-      .options(Map("table" -> "OUTPUT_GIGANTIC_TABLE", "zkUrl" -> quorumAddress))
+      .format("phoenix")
+      .options(Map("table" -> "OUTPUT_GIGANTIC_TABLE", PhoenixDataSource.ZOOKEEPER_URL -> quorumAddress))
       .mode(SaveMode.Overwrite)
       .save()
 
diff --git a/phoenix-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkITTenantSpecific.scala b/phoenix-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkITTenantSpecific.scala
index 77b41af..291ea2a 100644
--- a/phoenix-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkITTenantSpecific.scala
+++ b/phoenix-spark/src/it/scala/org/apache/phoenix/spark/PhoenixSparkITTenantSpecific.scala
@@ -64,8 +64,7 @@ class PhoenixSparkITTenantSpecific extends AbstractPhoenixSparkIT {
   /*****************/
 
   test("Can read from tenant-specific table as DataFrame") {
-    val sqlContext = new SQLContext(sc)
-    val df = sqlContext.phoenixTableAsDataFrame(
+    val df = spark.sqlContext.phoenixTableAsDataFrame(
       TenantTable,
       Seq(OrgIdCol, TenantOnlyCol),
       zkUrl = Some(quorumAddress),
@@ -78,7 +77,7 @@ class PhoenixSparkITTenantSpecific extends AbstractPhoenixSparkIT {
   }
 
   test("Can read from tenant-specific table as RDD") {
-    val rdd = sc.phoenixTableAsRDD(
+    val rdd = spark.sparkContext.phoenixTableAsRDD(
       TenantTable,
       Seq(OrgIdCol, TenantOnlyCol),
       zkUrl = Some(quorumAddress),
@@ -95,23 +94,23 @@ class PhoenixSparkITTenantSpecific extends AbstractPhoenixSparkIT {
   /*****************/
 
   test("Can write a DataFrame using 'DataFrame.saveToPhoenix' to tenant-specific view") {
-    val sqlContext = new SQLContext(sc)
+    val sqlContext = spark.sqlContext
     import sqlContext.implicits._
 
-    val df = sc.parallelize(TestDataSet).toDF(OrgIdCol, TenantOnlyCol)
+    val df = spark.sparkContext.parallelize(TestDataSet).toDF(OrgIdCol, TenantOnlyCol)
     df.saveToPhoenix(TenantTable, zkUrl = Some(quorumAddress), tenantId = Some(TenantId))
 
     verifyResults
   }
 
   test("Can write a DataFrame using 'DataFrame.write' to tenant-specific view") {
-    val sqlContext = new SQLContext(sc)
+    val sqlContext = spark.sqlContext
     import sqlContext.implicits._
 
-    val df = sc.parallelize(TestDataSet).toDF(OrgIdCol, TenantOnlyCol)
+    val df = spark.sparkContext.parallelize(TestDataSet).toDF(OrgIdCol, TenantOnlyCol)
 
     df.write
-      .format("org.apache.phoenix.spark")
+      .format("phoenix")
       .mode("overwrite")
       .option("table", TenantTable)
       .option(PhoenixRuntime.TENANT_ID_ATTRIB, TenantId)
@@ -122,8 +121,7 @@ class PhoenixSparkITTenantSpecific extends AbstractPhoenixSparkIT {
   }
 
   test("Can write an RDD to Phoenix tenant-specific view") {
-    val sqlContext = new SQLContext(sc)
-    sc
+    spark.sparkContext
       .parallelize(TestDataSet)
       .saveToPhoenix(
         TenantTable,
diff --git a/phoenix-spark/src/main/java/org/apache/phoenix/spark/datasource/v2/PhoenixDataSource.java b/phoenix-spark/src/main/java/org/apache/phoenix/spark/datasource/v2/PhoenixDataSource.java
new file mode 100644
index 0000000..ad79d1c
--- /dev/null
+++ b/phoenix-spark/src/main/java/org/apache/phoenix/spark/datasource/v2/PhoenixDataSource.java
@@ -0,0 +1,82 @@
+/*
+ * 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.spark.datasource.v2;
+
+import java.util.Optional;
+
+import org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil;
+import org.apache.phoenix.spark.datasource.v2.reader.PhoenixDataSourceReader;
+import org.apache.phoenix.spark.datasource.v2.writer.PhoenixDataSourceWriteOptions;
+import org.apache.phoenix.spark.datasource.v2.writer.PhoenixDatasourceWriter;
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.apache.spark.sql.SaveMode;
+import org.apache.spark.sql.sources.DataSourceRegister;
+import org.apache.spark.sql.sources.v2.DataSourceOptions;
+import org.apache.spark.sql.sources.v2.DataSourceV2;
+import org.apache.spark.sql.sources.v2.ReadSupport;
+import org.apache.spark.sql.sources.v2.WriteSupport;
+import org.apache.spark.sql.sources.v2.reader.DataSourceReader;
+import org.apache.spark.sql.sources.v2.writer.DataSourceWriter;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * Implements the DataSourceV2 api to read and write from Phoenix tables
+ */
+public class PhoenixDataSource  implements DataSourceV2,  ReadSupport, WriteSupport, DataSourceRegister {
+
+    public static final String SKIP_NORMALIZING_IDENTIFIER = "skipNormalizingIdentifier";
+    public static final String ZOOKEEPER_URL = "zkUrl";
+
+    @Override
+    public DataSourceReader createReader(DataSourceOptions options) {
+        return new PhoenixDataSourceReader(options);
+    }
+
+    @Override
+    public Optional<DataSourceWriter> createWriter(String writeUUID, StructType schema, SaveMode mode,
+            DataSourceOptions options) {
+        if (!mode.equals(SaveMode.Overwrite)) {
+            throw new RuntimeException("SaveMode other than SaveMode.OverWrite is not supported");
+        }
+        if (!options.tableName().isPresent()) {
+            throw new RuntimeException("No Phoenix option " + DataSourceOptions.TABLE_KEY + " defined");
+        }
+        if (!options.get(PhoenixDataSource.ZOOKEEPER_URL).isPresent()) {
+            throw new RuntimeException("No Phoenix option " + PhoenixDataSource.ZOOKEEPER_URL + " defined");
+        }
+
+        PhoenixDataSourceWriteOptions writeOptions = createPhoenixDataSourceWriteOptions(options, schema);
+        return Optional.of(new PhoenixDatasourceWriter(writeOptions));
+    }
+
+    private PhoenixDataSourceWriteOptions createPhoenixDataSourceWriteOptions(DataSourceOptions options,
+            StructType schema) {
+        String scn = options.get(PhoenixConfigurationUtil.CURRENT_SCN_VALUE).orElse(null);
+        String tenantId = options.get(PhoenixRuntime.TENANT_ID_ATTRIB).orElse(null);
+        String zkUrl = options.get(ZOOKEEPER_URL).get();
+        boolean skipNormalizingIdentifier = options.getBoolean(SKIP_NORMALIZING_IDENTIFIER, false);
+        return new PhoenixDataSourceWriteOptions.Builder().setTableName(options.tableName().get())
+                .setZkUrl(zkUrl).setScn(scn).setTenantId(tenantId).setSchema(schema)
+                .setSkipNormalizingIdentifier(skipNormalizingIdentifier).build();
+    }
+
+    @Override
+    public String shortName() {
+        return "phoenix";
+    }
+}
diff --git a/phoenix-spark/src/main/java/org/apache/phoenix/spark/datasource/v2/reader/PhoenixDataSourceReadOptions.java b/phoenix-spark/src/main/java/org/apache/phoenix/spark/datasource/v2/reader/PhoenixDataSourceReadOptions.java
new file mode 100644
index 0000000..8c2fdb1
--- /dev/null
+++ b/phoenix-spark/src/main/java/org/apache/phoenix/spark/datasource/v2/reader/PhoenixDataSourceReadOptions.java
@@ -0,0 +1,51 @@
+/*
+ * 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.spark.datasource.v2.reader;
+
+import java.io.Serializable;
+
+public class PhoenixDataSourceReadOptions implements Serializable {
+
+    private final String tenantId;
+    private final String zkUrl;
+    private final String scn;
+    private final String selectStatement;
+
+    public PhoenixDataSourceReadOptions(String zkUrl, String scn, String tenantId, String selectStatement) {
+        this.zkUrl = zkUrl;
+        this.scn = scn;
+        this.tenantId = tenantId;
+        this.selectStatement = selectStatement;
+    }
+
+    public String getSelectStatement() {
+        return selectStatement;
+    }
+
+    public String getScn() {
+        return scn;
+    }
+
+    public String getZkUrl() {
+        return zkUrl;
+    }
+
+    public String getTenantId() {
+        return tenantId;
+    }
+}
diff --git a/phoenix-spark/src/main/java/org/apache/phoenix/spark/datasource/v2/reader/PhoenixDataSourceReader.java b/phoenix-spark/src/main/java/org/apache/phoenix/spark/datasource/v2/reader/PhoenixDataSourceReader.java
new file mode 100644
index 0000000..446d96f
--- /dev/null
+++ b/phoenix-spark/src/main/java/org/apache/phoenix/spark/datasource/v2/reader/PhoenixDataSourceReader.java
@@ -0,0 +1,201 @@
+/*
+ * 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.spark.datasource.v2.reader;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionLocator;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.util.RegionSizeCalculator;
+import org.apache.phoenix.compile.QueryPlan;
+import org.apache.phoenix.iterate.MapReduceParallelScanGrouper;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixStatement;
+import org.apache.phoenix.mapreduce.PhoenixInputSplit;
+import org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil;
+import org.apache.phoenix.query.KeyRange;
+import org.apache.phoenix.spark.FilterExpressionCompiler;
+import org.apache.phoenix.spark.SparkSchemaUtil;
+import org.apache.phoenix.spark.datasource.v2.PhoenixDataSource;
+import org.apache.phoenix.util.ColumnInfo;
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.apache.phoenix.util.QueryUtil;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.sources.Filter;
+import org.apache.spark.sql.sources.v2.DataSourceOptions;
+import org.apache.spark.sql.sources.v2.reader.DataSourceReader;
+import org.apache.spark.sql.sources.v2.reader.InputPartition;
+import org.apache.spark.sql.sources.v2.reader.SupportsPushDownFilters;
+import org.apache.spark.sql.sources.v2.reader.SupportsPushDownRequiredColumns;
+import org.apache.spark.sql.types.StructType;
+import scala.Tuple3;
+import scala.collection.JavaConverters;
+import scala.collection.Seq;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import java.util.Properties;
+
+public class PhoenixDataSourceReader implements DataSourceReader, SupportsPushDownFilters,
+        SupportsPushDownRequiredColumns {
+
+    private final DataSourceOptions options;
+    private final String tableName;
+    private final String zkUrl;
+    private final boolean dateAsTimestamp;
+
+    private StructType schema;
+    private Filter[] pushedFilters = new Filter[]{};
+    // derived from pushedFilters
+    private String whereClause;
+
+    public PhoenixDataSourceReader(DataSourceOptions options) {
+        if (!options.tableName().isPresent()) {
+            throw new RuntimeException("No Phoenix option " + DataSourceOptions.TABLE_KEY + " defined");
+        }
+        if (!options.get(PhoenixDataSource.ZOOKEEPER_URL).isPresent()) {
+            throw new RuntimeException("No Phoenix option " + PhoenixDataSource.ZOOKEEPER_URL + " defined");
+        }
+        this.options = options;
+        this.tableName = options.tableName().get();
+        this.zkUrl = options.get("zkUrl").get();
+        this.dateAsTimestamp = options.getBoolean("dateAsTimestamp", false);
+        setSchema();
+    }
+
+    /**
+     * Sets the schema using all the table columns before any column pruning has been done
+     */
+    private void setSchema() {
+        try (Connection conn = DriverManager.getConnection("jdbc:phoenix:" + zkUrl)) {
+            List<ColumnInfo> columnInfos = PhoenixRuntime.generateColumnInfo(conn, tableName, null);
+            Seq<ColumnInfo> columnInfoSeq = JavaConverters.asScalaIteratorConverter(columnInfos.iterator()).asScala().toSeq();
+            schema = SparkSchemaUtil.phoenixSchemaToCatalystSchema(columnInfoSeq, dateAsTimestamp);
+        }
+        catch (SQLException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Override
+    public StructType readSchema() {
+        return schema;
+    }
+
+    @Override
+    public Filter[] pushFilters(Filter[] filters) {
+        Tuple3<String, Filter[], Filter[]> tuple3 = new FilterExpressionCompiler().pushFilters(filters);
+        whereClause = tuple3._1();
+        pushedFilters = tuple3._3();
+        return tuple3._2();
+    }
+
+    @Override
+    public List<InputPartition<InternalRow>> planInputPartitions() {
+        Optional<String> currentScnValue = options.get(PhoenixConfigurationUtil.CURRENT_SCN_VALUE);
+        Optional<String> tenantId = options.get(PhoenixConfigurationUtil.MAPREDUCE_TENANT_ID);
+        // Generate splits based off statistics, or just region splits?
+        boolean splitByStats = options.getBoolean(
+                PhoenixConfigurationUtil.MAPREDUCE_SPLIT_BY_STATS, PhoenixConfigurationUtil.DEFAULT_SPLIT_BY_STATS);
+        Properties overridingProps = new Properties();
+        if(currentScnValue.isPresent()) {
+            overridingProps.put(PhoenixRuntime.CURRENT_SCN_ATTRIB, currentScnValue.get());
+        }
+        if (tenantId.isPresent()){
+            overridingProps.put(PhoenixRuntime.TENANT_ID_ATTRIB, tenantId.get());
+        }
+        try (Connection conn = DriverManager.getConnection("jdbc:phoenix:" + zkUrl, overridingProps)) {
+            List<ColumnInfo> columnInfos = PhoenixRuntime.generateColumnInfo(conn, tableName, Lists.newArrayList(schema.names()));
+            final Statement statement = conn.createStatement();
+            final String selectStatement = QueryUtil.constructSelectStatement(tableName, columnInfos, whereClause);
+            Preconditions.checkNotNull(selectStatement);
+
+            final PhoenixStatement pstmt = statement.unwrap(PhoenixStatement.class);
+            // Optimize the query plan so that we potentially use secondary indexes
+            final QueryPlan queryPlan = pstmt.optimizeQuery(selectStatement);
+            final Scan scan = queryPlan.getContext().getScan();
+
+            // setting the snapshot configuration
+            Optional<String> snapshotName = options.get(PhoenixConfigurationUtil.SNAPSHOT_NAME_KEY);
+            if (snapshotName.isPresent())
+                PhoenixConfigurationUtil.setSnapshotNameKey(queryPlan.getContext().getConnection().
+                        getQueryServices().getConfiguration(), snapshotName.get());
+
+            // Initialize the query plan so it sets up the parallel scans
+            queryPlan.iterator(MapReduceParallelScanGrouper.getInstance());
+
+            List<KeyRange> allSplits = queryPlan.getSplits();
+            // Get the RegionSizeCalculator
+            PhoenixConnection phxConn = conn.unwrap(PhoenixConnection.class);
+            org.apache.hadoop.hbase.client.Connection connection =
+                    phxConn.getQueryServices().getAdmin().getConnection();
+            RegionLocator regionLocator = connection.getRegionLocator(TableName.valueOf(queryPlan
+                    .getTableRef().getTable().getPhysicalName().toString()));
+            RegionSizeCalculator sizeCalculator = new RegionSizeCalculator(regionLocator, connection
+                    .getAdmin());
+
+            final List<InputPartition<InternalRow>> partitions = Lists.newArrayListWithExpectedSize(allSplits.size());
+            for (List<Scan> scans : queryPlan.getScans()) {
+                // Get the region location
+                HRegionLocation location = regionLocator.getRegionLocation(
+                        scans.get(0).getStartRow(),
+                        false
+                );
+
+                String regionLocation = location.getHostname();
+
+                // Get the region size
+                long regionSize = sizeCalculator.getRegionSize(
+                        location.getRegionInfo().getRegionName()
+                );
+
+                PhoenixDataSourceReadOptions phoenixDataSourceOptions = new PhoenixDataSourceReadOptions(zkUrl,
+                        currentScnValue.orElse(null), tenantId.orElse(null), selectStatement);
+                if (splitByStats) {
+                    for (Scan aScan : scans) {
+                        partitions.add(new PhoenixInputPartition(phoenixDataSourceOptions, schema,
+                                new PhoenixInputSplit(Collections.singletonList(aScan), regionSize, regionLocation)));
+                    }
+                } else {
+                    partitions.add(new PhoenixInputPartition(phoenixDataSourceOptions, schema,
+                            new PhoenixInputSplit(scans, regionSize, regionLocation)));
+                }
+            }
+            return partitions;
+        } catch (Exception e) {
+            throw new RuntimeException("Unable to plan query", e);
+        }
+    }
+
+    @Override
+    public Filter[] pushedFilters() {
+        return pushedFilters;
+    }
+
+    @Override
+    public void pruneColumns(StructType schema) {
+        this.schema = schema;
+    }
+}
diff --git a/phoenix-spark/src/main/java/org/apache/phoenix/spark/datasource/v2/reader/PhoenixInputPartition.java b/phoenix-spark/src/main/java/org/apache/phoenix/spark/datasource/v2/reader/PhoenixInputPartition.java
new file mode 100644
index 0000000..624ff0f
--- /dev/null
+++ b/phoenix-spark/src/main/java/org/apache/phoenix/spark/datasource/v2/reader/PhoenixInputPartition.java
@@ -0,0 +1,44 @@
+/*
+ * 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.spark.datasource.v2.reader;
+
+import org.apache.phoenix.mapreduce.PhoenixInputSplit;
+import org.apache.spark.SerializableWritable;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.sources.v2.reader.InputPartition;
+import org.apache.spark.sql.sources.v2.reader.InputPartitionReader;
+import org.apache.spark.sql.types.StructType;
+
+public class PhoenixInputPartition implements InputPartition<InternalRow> {
+
+    private SerializableWritable<PhoenixInputSplit> phoenixInputSplit;
+    private StructType schema;
+    private PhoenixDataSourceReadOptions options;
+
+    public PhoenixInputPartition(PhoenixDataSourceReadOptions options, StructType schema, PhoenixInputSplit phoenixInputSplit) {
+        this.phoenixInputSplit = new SerializableWritable<>(phoenixInputSplit);
+        this.schema = schema;
+        this.options = options;
+    }
+
+    @Override
+    public InputPartitionReader<InternalRow> createPartitionReader() {
+        return new PhoenixInputPartitionReader(options, schema, phoenixInputSplit);
+    }
+
+}
diff --git a/phoenix-spark/src/main/java/org/apache/phoenix/spark/datasource/v2/reader/PhoenixInputPartitionReader.java b/phoenix-spark/src/main/java/org/apache/phoenix/spark/datasource/v2/reader/PhoenixInputPartitionReader.java
new file mode 100644
index 0000000..30e84db
--- /dev/null
+++ b/phoenix-spark/src/main/java/org/apache/phoenix/spark/datasource/v2/reader/PhoenixInputPartitionReader.java
@@ -0,0 +1,168 @@
+/*
+ * 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.spark.datasource.v2.reader;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.List;
+import java.util.Properties;
+
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.compile.QueryPlan;
+import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.coprocessor.BaseScannerRegionObserver;
+import org.apache.phoenix.iterate.ConcatResultIterator;
+import org.apache.phoenix.iterate.LookAheadResultIterator;
+import org.apache.phoenix.iterate.MapReduceParallelScanGrouper;
+import org.apache.phoenix.iterate.PeekingResultIterator;
+import org.apache.phoenix.iterate.ResultIterator;
+import org.apache.phoenix.iterate.RoundRobinResultIterator;
+import org.apache.phoenix.iterate.SequenceResultIterator;
+import org.apache.phoenix.iterate.TableResultIterator;
+import org.apache.phoenix.jdbc.PhoenixResultSet;
+import org.apache.phoenix.jdbc.PhoenixStatement;
+import org.apache.phoenix.mapreduce.PhoenixInputSplit;
+import org.apache.phoenix.monitoring.ReadMetricQueue;
+import org.apache.phoenix.monitoring.ScanMetricsHolder;
+import org.apache.phoenix.query.ConnectionQueryServices;
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.apache.spark.SerializableWritable;
+import org.apache.spark.executor.InputMetrics;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.execution.datasources.SparkJdbcUtil;
+import org.apache.spark.sql.sources.v2.reader.InputPartitionReader;
+import org.apache.spark.sql.types.StructType;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+import scala.collection.Iterator;
+
+public class PhoenixInputPartitionReader implements InputPartitionReader<InternalRow>  {
+
+    private SerializableWritable<PhoenixInputSplit> phoenixInputSplit;
+    private StructType schema;
+    private Iterator<InternalRow> iterator;
+    private PhoenixResultSet resultSet;
+    private InternalRow currentRow;
+    private PhoenixDataSourceReadOptions options;
+
+    public PhoenixInputPartitionReader(PhoenixDataSourceReadOptions options, StructType schema, SerializableWritable<PhoenixInputSplit> phoenixInputSplit) {
+        this.options = options;
+        this.phoenixInputSplit = phoenixInputSplit;
+        this.schema = schema;
+        initialize();
+    }
+
+    private QueryPlan getQueryPlan() throws SQLException {
+        String scn = options.getScn();
+        String tenantId = options.getTenantId();
+        String zkUrl = options.getZkUrl();
+        Properties overridingProps = new Properties();
+        if (scn != null) {
+            overridingProps.put(PhoenixRuntime.CURRENT_SCN_ATTRIB, scn);
+        }
+        if (tenantId != null) {
+            overridingProps.put(PhoenixRuntime.TENANT_ID_ATTRIB, tenantId);
+        }
+        try (Connection conn = DriverManager.getConnection("jdbc:phoenix:" + zkUrl, overridingProps)) {
+            final Statement statement = conn.createStatement();
+            final String selectStatement = options.getSelectStatement();
+            Preconditions.checkNotNull(selectStatement);
+
+            final PhoenixStatement pstmt = statement.unwrap(PhoenixStatement.class);
+            // Optimize the query plan so that we potentially use secondary indexes
+            final QueryPlan queryPlan = pstmt.optimizeQuery(selectStatement);
+            return queryPlan;
+        }
+    }
+
+    private void initialize() {
+        try {
+            final QueryPlan queryPlan = getQueryPlan();
+            final List<Scan> scans = phoenixInputSplit.value().getScans();
+            List<PeekingResultIterator> iterators = Lists.newArrayListWithExpectedSize(scans.size());
+            StatementContext ctx = queryPlan.getContext();
+            ReadMetricQueue readMetrics = ctx.getReadMetricsQueue();
+            String tableName = queryPlan.getTableRef().getTable().getPhysicalName().getString();
+
+            // Clear the table region boundary cache to make sure long running jobs stay up to date
+            byte[] tableNameBytes = queryPlan.getTableRef().getTable().getPhysicalName().getBytes();
+            ConnectionQueryServices services = queryPlan.getContext().getConnection().getQueryServices();
+            services.clearTableRegionCache(tableNameBytes);
+
+            long renewScannerLeaseThreshold = queryPlan.getContext().getConnection().getQueryServices().getRenewLeaseThresholdMilliSeconds();
+            for (Scan scan : scans) {
+                // For MR, skip the region boundary check exception if we encounter a split. ref: PHOENIX-2599
+                scan.setAttribute(BaseScannerRegionObserver.SKIP_REGION_BOUNDARY_CHECK, Bytes.toBytes(true));
+
+                PeekingResultIterator peekingResultIterator;
+                ScanMetricsHolder scanMetricsHolder =
+                        ScanMetricsHolder.getInstance(readMetrics, tableName, scan,
+                                queryPlan.getContext().getConnection().getLogLevel());
+                final TableResultIterator tableResultIterator =
+                        new TableResultIterator(
+                                queryPlan.getContext().getConnection().getMutationState(), scan,
+                                scanMetricsHolder, renewScannerLeaseThreshold, queryPlan,
+                                MapReduceParallelScanGrouper.getInstance());
+                peekingResultIterator = LookAheadResultIterator.wrap(tableResultIterator);
+                iterators.add(peekingResultIterator);
+            }
+            ResultIterator iterator = queryPlan.useRoundRobinIterator() ? RoundRobinResultIterator.newIterator(iterators, queryPlan) : ConcatResultIterator.newIterator(iterators);
+            if (queryPlan.getContext().getSequenceManager().getSequenceCount() > 0) {
+                iterator = new SequenceResultIterator(iterator, queryPlan.getContext().getSequenceManager());
+            }
+            // Clone the row projector as it's not thread safe and would be used simultaneously by
+            // multiple threads otherwise.
+            this.resultSet = new PhoenixResultSet(iterator, queryPlan.getProjector().cloneIfNecessary(), queryPlan.getContext());
+            this.iterator = SparkJdbcUtil.resultSetToSparkInternalRows(resultSet, schema, new InputMetrics());
+        }
+        catch (SQLException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Override
+    public boolean next() {
+        if (!iterator.hasNext()) {
+            return false;
+        }
+        currentRow = iterator.next();
+        return true;
+    }
+
+    @Override
+    public InternalRow get() {
+        return currentRow;
+    }
+
+    @Override
+    public void close() throws IOException {
+        if(resultSet != null) {
+            try {
+                resultSet.close();
+            } catch (SQLException e) {
+                throw new IOException(e);
+            }
+        }
+    }
+}
diff --git a/phoenix-spark/src/main/java/org/apache/phoenix/spark/datasource/v2/writer/PhoenixDataSourceWriteOptions.java b/phoenix-spark/src/main/java/org/apache/phoenix/spark/datasource/v2/writer/PhoenixDataSourceWriteOptions.java
new file mode 100644
index 0000000..781d1c8
--- /dev/null
+++ b/phoenix-spark/src/main/java/org/apache/phoenix/spark/datasource/v2/writer/PhoenixDataSourceWriteOptions.java
@@ -0,0 +1,109 @@
+/*
+ * 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.spark.datasource.v2.writer;
+
+import org.apache.spark.sql.types.StructType;
+
+import java.io.Serializable;
+
+public class PhoenixDataSourceWriteOptions implements Serializable {
+
+    private final String tableName;
+    private final String zkUrl;
+    private final String tenantId;
+    private final String scn;
+    private final StructType schema;
+    private final boolean skipNormalizingIdentifier;
+
+    private PhoenixDataSourceWriteOptions(String tableName, String zkUrl, String scn, String tenantId,
+                                          StructType schema, boolean skipNormalizingIdentifier) {
+        this.tableName = tableName;
+        this.zkUrl = zkUrl;
+        this.scn = scn;
+        this.tenantId = tenantId;
+        this.schema = schema;
+        this.skipNormalizingIdentifier = skipNormalizingIdentifier;
+    }
+
+    public String getScn() {
+        return scn;
+    }
+
+    public String getZkUrl() {
+        return zkUrl;
+    }
+
+    public String getTenantId() {
+        return tenantId;
+    }
+
+    public StructType getSchema() {
+        return schema;
+    }
+
+    public String getTableName() {
+        return tableName;
+    }
+
+    public boolean skipNormalizingIdentifier() {
+        return skipNormalizingIdentifier;
+    }
+
+    public static class Builder {
+        private String tableName;
+        private String zkUrl;
+        private String scn;
+        private String tenantId;
+        private StructType schema;
+        private boolean skipNormalizingIdentifier;
+
+        public Builder setTableName(String tableName) {
+            this.tableName = tableName;
+            return this;
+        }
+
+        public Builder setZkUrl(String zkUrl) {
+            this.zkUrl = zkUrl;
+            return this;
+        }
+
+        public Builder setScn(String scn) {
+            this.scn = scn;
+            return this;
+        }
+
+        public Builder setTenantId(String tenantId) {
+            this.tenantId = tenantId;
+            return this;
+        }
+
+        public Builder setSchema(StructType schema) {
+            this.schema = schema;
+            return this;
+        }
+
+        public Builder setSkipNormalizingIdentifier(boolean skipNormalizingIdentifier) {
+            this.skipNormalizingIdentifier = skipNormalizingIdentifier;
+            return this;
+        }
+
+        public PhoenixDataSourceWriteOptions build() {
+            return new PhoenixDataSourceWriteOptions(tableName, zkUrl, scn, tenantId, schema, skipNormalizingIdentifier);
+        }
+    }
+}
\ No newline at end of file
diff --git a/phoenix-spark/src/main/java/org/apache/phoenix/spark/datasource/v2/writer/PhoenixDataWriter.java b/phoenix-spark/src/main/java/org/apache/phoenix/spark/datasource/v2/writer/PhoenixDataWriter.java
new file mode 100644
index 0000000..cf42aa5
--- /dev/null
+++ b/phoenix-spark/src/main/java/org/apache/phoenix/spark/datasource/v2/writer/PhoenixDataWriter.java
@@ -0,0 +1,100 @@
+package org.apache.phoenix.spark.datasource.v2.writer;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.Properties;
+import java.util.stream.Collectors;
+
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.apache.phoenix.util.QueryUtil;
+import org.apache.phoenix.util.SchemaUtil;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.execution.datasources.SparkJdbcUtil;
+import org.apache.spark.sql.execution.datasources.jdbc.PhoenixJdbcDialect$;
+import org.apache.spark.sql.sources.v2.writer.DataWriter;
+import org.apache.spark.sql.sources.v2.writer.WriterCommitMessage;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import com.google.common.collect.Lists;
+
+public class PhoenixDataWriter implements DataWriter<InternalRow> {
+
+    private final StructType schema;
+    private final Connection conn;
+    private final PreparedStatement statement;
+
+    public PhoenixDataWriter(PhoenixDataSourceWriteOptions options) {
+        String scn = options.getScn();
+        String tenantId = options.getTenantId();
+        String zkUrl = options.getZkUrl();
+        Properties overridingProps = new Properties();
+        if (scn != null) {
+            overridingProps.put(PhoenixRuntime.CURRENT_SCN_ATTRIB, scn);
+        }
+        if (tenantId != null) {
+            overridingProps.put(PhoenixRuntime.TENANT_ID_ATTRIB, tenantId);
+        }
+        this.schema = options.getSchema();
+        try {
+            this.conn = DriverManager.getConnection("jdbc:phoenix:" + zkUrl, overridingProps);
+            List<String> colNames = Lists.newArrayList(options.getSchema().names());
+            if (!options.skipNormalizingIdentifier()){
+                colNames = colNames.stream().map(colName -> SchemaUtil.normalizeIdentifier(colName)).collect(Collectors.toList());
+            }
+            String upsertSql = QueryUtil.constructUpsertStatement(options.getTableName(), colNames, null);
+            this.statement = this.conn.prepareStatement(upsertSql);
+        } catch (SQLException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Override
+    public void write(InternalRow internalRow) throws IOException {
+        try {
+            int i=0;
+            for (StructField field : schema.fields()) {
+                DataType dataType = field.dataType();
+                if (internalRow.isNullAt(i)) {
+                    statement.setNull(i + 1, SparkJdbcUtil.getJdbcType(dataType,
+                            PhoenixJdbcDialect$.MODULE$).jdbcNullType());
+                } else {
+                    Row row = SparkJdbcUtil.toRow(schema, internalRow);
+                    SparkJdbcUtil.makeSetter(conn, PhoenixJdbcDialect$.MODULE$, dataType).apply(statement, row, i);
+                }
+                ++i;
+            }
+            statement.execute();
+        } catch (SQLException e) {
+            throw new IOException("Exception while executing Phoenix prepared statement", e);
+        }
+    }
+
+    @Override
+    public WriterCommitMessage commit() throws IOException {
+        try {
+            conn.commit();
+        } catch (SQLException e) {
+            throw new RuntimeException(e);
+        } finally {
+            try {
+                statement.close();
+                conn.close();
+            }
+            catch (SQLException ex) {
+                throw new RuntimeException(ex);
+            }
+        }
+        return null;
+    }
+
+    @Override
+    public void abort() throws IOException {
+    }
+}
diff --git a/phoenix-spark/src/main/java/org/apache/phoenix/spark/datasource/v2/writer/PhoenixDataWriterFactory.java b/phoenix-spark/src/main/java/org/apache/phoenix/spark/datasource/v2/writer/PhoenixDataWriterFactory.java
new file mode 100644
index 0000000..751fdfa
--- /dev/null
+++ b/phoenix-spark/src/main/java/org/apache/phoenix/spark/datasource/v2/writer/PhoenixDataWriterFactory.java
@@ -0,0 +1,19 @@
+package org.apache.phoenix.spark.datasource.v2.writer;
+
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.sources.v2.writer.DataWriter;
+import org.apache.spark.sql.sources.v2.writer.DataWriterFactory;
+
+public class PhoenixDataWriterFactory implements DataWriterFactory<InternalRow> {
+
+    private final PhoenixDataSourceWriteOptions options;
+
+    public PhoenixDataWriterFactory(PhoenixDataSourceWriteOptions options) {
+        this.options = options;
+    }
+
+    @Override
+    public DataWriter<InternalRow> createDataWriter(int partitionId, long taskId, long epochId) {
+        return new PhoenixDataWriter(options);
+    }
+}
diff --git a/phoenix-spark/src/main/java/org/apache/phoenix/spark/datasource/v2/writer/PhoenixDatasourceWriter.java b/phoenix-spark/src/main/java/org/apache/phoenix/spark/datasource/v2/writer/PhoenixDatasourceWriter.java
new file mode 100644
index 0000000..7847609
--- /dev/null
+++ b/phoenix-spark/src/main/java/org/apache/phoenix/spark/datasource/v2/writer/PhoenixDatasourceWriter.java
@@ -0,0 +1,34 @@
+package org.apache.phoenix.spark.datasource.v2.writer;
+
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.sources.v2.DataSourceOptions;
+import org.apache.spark.sql.sources.v2.writer.DataSourceWriter;
+import org.apache.spark.sql.sources.v2.writer.DataWriterFactory;
+import org.apache.spark.sql.sources.v2.writer.WriterCommitMessage;
+
+public class PhoenixDatasourceWriter implements DataSourceWriter {
+
+    private final PhoenixDataSourceWriteOptions options;
+
+    public PhoenixDatasourceWriter(PhoenixDataSourceWriteOptions options) {
+        this.options = options;
+    }
+
+    @Override
+    public DataWriterFactory<InternalRow> createWriterFactory() {
+        return new PhoenixDataWriterFactory(options);
+    }
+
+    @Override
+    public boolean useCommitCoordinator() {
+        return false;
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+    }
+
+    @Override
+    public void abort(WriterCommitMessage[] messages) {
+    }
+}
diff --git a/phoenix-spark/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister b/phoenix-spark/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
new file mode 100644
index 0000000..6eff1af
--- /dev/null
+++ b/phoenix-spark/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister
@@ -0,0 +1 @@
+org.apache.phoenix.spark.datasource.v2.PhoenixDataSource
\ No newline at end of file
diff --git a/phoenix-spark/src/main/scala/org/apache/phoenix/spark/ConfigurationUtil.scala b/phoenix-spark/src/main/scala/org/apache/phoenix/spark/ConfigurationUtil.scala
index ca476e7..d555954 100644
--- a/phoenix-spark/src/main/scala/org/apache/phoenix/spark/ConfigurationUtil.scala
+++ b/phoenix-spark/src/main/scala/org/apache/phoenix/spark/ConfigurationUtil.scala
@@ -21,6 +21,7 @@ import org.apache.phoenix.util.{ColumnInfo, PhoenixRuntime}
 
 import scala.collection.JavaConversions._
 
+@deprecated("Use the DataSource V2 API implementation (see PhoenixDataSource)")
 object ConfigurationUtil extends Serializable {
 
   def getOutputConfiguration(tableName: String, columns: Seq[String], zkUrl: Option[String], tenantId: Option[String] = None, conf: Option[Configuration] = None): Configuration = {
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 be4a32b..3b0289d 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
@@ -22,7 +22,7 @@ import org.apache.spark.sql.DataFrame
 
 import scala.collection.JavaConversions._
 
-
+@deprecated("Use the DataSource V2 API implementation (see PhoenixDataSource)")
 class DataFrameFunctions(data: DataFrame) extends Serializable {
   def saveToPhoenix(parameters: Map[String, String]): Unit = {
   		saveToPhoenix(parameters("table"), zkUrl = parameters.get("zkUrl"), tenantId = parameters.get("TenantId"), 
diff --git a/phoenix-spark/src/main/scala/org/apache/phoenix/spark/DefaultSource.scala b/phoenix-spark/src/main/scala/org/apache/phoenix/spark/DefaultSource.scala
index e000b74..ccdf595 100644
--- a/phoenix-spark/src/main/scala/org/apache/phoenix/spark/DefaultSource.scala
+++ b/phoenix-spark/src/main/scala/org/apache/phoenix/spark/DefaultSource.scala
@@ -20,6 +20,7 @@ package org.apache.phoenix.spark
 import org.apache.spark.sql.sources.{BaseRelation, CreatableRelationProvider, RelationProvider}
 import org.apache.spark.sql.{DataFrame, SQLContext, SaveMode}
 
+@deprecated("Use the DataSource V2 API implementation (see PhoenixDataSource)")
 class DefaultSource extends RelationProvider with CreatableRelationProvider {
 
   // Override 'RelationProvider.createRelation', this enables DataFrame.load()
diff --git a/phoenix-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRelation.scala b/phoenix-spark/src/main/scala/org/apache/phoenix/spark/FilterExpressionCompiler.scala
similarity index 60%
copy from phoenix-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRelation.scala
copy to phoenix-spark/src/main/scala/org/apache/phoenix/spark/FilterExpressionCompiler.scala
index 38bf29a..74ff67e 100644
--- a/phoenix-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRelation.scala
+++ b/phoenix-spark/src/main/scala/org/apache/phoenix/spark/FilterExpressionCompiler.scala
@@ -17,55 +17,27 @@
  */
 package org.apache.phoenix.spark
 
-import org.apache.hadoop.conf.Configuration
-import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.types.StructType
-import org.apache.spark.sql.{Row, SQLContext}
-import org.apache.spark.sql.sources._
+import java.sql.Timestamp
+import java.text.Format
+
+import org.apache.phoenix.util.{DateUtil, SchemaUtil}
 import org.apache.phoenix.util.StringUtil.escapeStringConstant
-import org.apache.phoenix.util.SchemaUtil
-
-case class PhoenixRelation(tableName: String, zkUrl: String, dateAsTimestamp: Boolean = false)(@transient val sqlContext: SQLContext)
-    extends BaseRelation with PrunedFilteredScan {
-
-  /*
-    This is the buildScan() implementing Spark's PrunedFilteredScan.
-    Spark SQL queries with columns or predicates specified will be pushed down
-    to us here, and we can pass that on to Phoenix. According to the docs, this
-    is an optimization, and the filtering/pruning will be re-evaluated again,
-    but this prevents having to load the whole table into Spark first.
-  */
-  override def buildScan(requiredColumns: Array[String], filters: Array[Filter]): RDD[Row] = {
-    val(pushedFilters, unhandledFilters) = buildFilter(filters)
-    new PhoenixRDD(
-      sqlContext.sparkContext,
-      tableName,
-      requiredColumns,
-      Some(pushedFilters),
-      Some(zkUrl),
-      new Configuration(),
-      dateAsTimestamp
-    ).toDataFrame(sqlContext).rdd
-  }
+import org.apache.spark.sql.sources._
 
-  // Required by BaseRelation, this will return the full schema for a table
-  override def schema: StructType = {
-    new PhoenixRDD(
-      sqlContext.sparkContext,
-      tableName,
-      Seq(),
-      None,
-      Some(zkUrl),
-      new Configuration(),
-      dateAsTimestamp
-    ).toDataFrame(sqlContext).schema
-  }
+class FilterExpressionCompiler() {
+
+  val timeformatter:Format = DateUtil.getTimestampFormatter(DateUtil.DEFAULT_TIME_FORMAT, DateUtil.DEFAULT_TIME_ZONE_ID)
 
-  // Attempt to create Phoenix-accepted WHERE clauses from Spark filters,
-  // mostly inspired from Spark SQL JDBCRDD and the couchbase-spark-connector
-  private def buildFilter(filters: Array[Filter]): (String, Array[Filter]) = {
+  /**
+    * Attempt to create Phoenix-accepted WHERE clause from Spark filters,
+    * mostly inspired from Spark SQL JDBCRDD and the couchbase-spark-connector
+    *
+    * @return tuple representing where clause (derived from supported filters),
+    *         array of unsupported filters and array of supported filters
+    */
+  def pushFilters(filters: Array[Filter]): (String, Array[Filter], Array[Filter]) = {
     if (filters.isEmpty) {
-      return ("" , Array[Filter]())
+      return ("" , Array[Filter](), Array[Filter]())
     }
 
     val filter = new StringBuilder("")
@@ -80,9 +52,30 @@ case class PhoenixRelation(tableName: String, zkUrl: String, dateAsTimestamp: Bo
 
       f match {
         // Spark 1.3.1+ supported filters
-        case And(leftFilter, rightFilter) => filter.append(buildFilter(Array(leftFilter, rightFilter)))
-        case Or(leftFilter, rightFilter) => filter.append(buildFilter(Array(leftFilter)) + " OR " + buildFilter(Array(rightFilter)))
-        case Not(aFilter) => filter.append(" NOT " + buildFilter(Array(aFilter)))
+        case And(leftFilter, rightFilter)  => {
+          val(whereClause, currUnsupportedFilters, _) = pushFilters(Array(leftFilter, rightFilter))
+          if (currUnsupportedFilters.isEmpty)
+            filter.append(whereClause)
+          else
+            unsupportedFilters :+ f
+        }
+        case Or(leftFilter, rightFilter) => {
+          val(whereLeftClause, leftUnsupportedFilters, _) = pushFilters(Array(leftFilter))
+          val(whereRightClause, rightUnsupportedFilters, _) = pushFilters(Array(rightFilter))
+          if (leftUnsupportedFilters.isEmpty && rightUnsupportedFilters.isEmpty) {
+            filter.append(whereLeftClause + " OR " + whereRightClause)
+          }
+          else {
+            unsupportedFilters :+ f
+          }
+        }
+        case Not(aFilter) => {
+          val(whereClause, currUnsupportedFilters, _) = pushFilters(Array(aFilter))
+          if (currUnsupportedFilters.isEmpty)
+            filter.append(" NOT " + whereClause)
+          else
+            unsupportedFilters :+ f
+        }
         case EqualTo(attr, value) => filter.append(s" ${escapeKey(attr)} = ${compileValue(value)}")
         case GreaterThan(attr, value) => filter.append(s" ${escapeKey(attr)} > ${compileValue(value)}")
         case GreaterThanOrEqual(attr, value) => filter.append(s" ${escapeKey(attr)} >= ${compileValue(value)}")
@@ -100,21 +93,15 @@ case class PhoenixRelation(tableName: String, zkUrl: String, dateAsTimestamp: Bo
       i = i + 1
     })
 
-    (filter.toString(), unsupportedFilters)
-  }
-
-  override def unhandledFilters(filters: Array[Filter]): Array[Filter] = {
-    val(pushedFilters, unhandledFilters) = buildFilter(filters)
-    unhandledFilters
+    (filter.toString(), unsupportedFilters, filters diff unsupportedFilters)
   }
 
-  // Helper function to escape column key to work with SQL queries
-  private def escapeKey(key: String): String = SchemaUtil.getEscapedArgument(key)
-
   // Helper function to escape string values in SQL queries
   private def compileValue(value: Any): Any = value match {
     case stringValue: String => s"'${escapeStringConstant(stringValue)}'"
 
+    case timestampValue: Timestamp => getTimestampString(timestampValue)
+
     // Borrowed from 'elasticsearch-hadoop', support these internal UTF types across Spark versions
     // Spark 1.4
     case utf if (isClass(utf, "org.apache.spark.sql.types.UTF8String")) => s"'${escapeStringConstant(utf.toString)}'"
@@ -125,6 +112,14 @@ case class PhoenixRelation(tableName: String, zkUrl: String, dateAsTimestamp: Bo
     case _ => value
   }
 
+  private def getTimestampString(timestampValue: Timestamp): String = {
+    "TO_TIMESTAMP('%s', '%s', '%s')".format(timeformatter.format(timestampValue),
+      DateUtil.DEFAULT_TIME_FORMAT, DateUtil.DEFAULT_TIME_ZONE_ID)
+  }
+
+  // Helper function to escape column key to work with SQL queries
+  private def escapeKey(key: String): String = SchemaUtil.getEscapedFullColumnName(key)
+
   private def isClass(obj: Any, className: String) = {
     className.equals(obj.getClass().getName())
   }
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 d604e0e..7331a5f 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
@@ -21,9 +21,6 @@ import org.apache.hadoop.io.NullWritable
 import org.apache.phoenix.jdbc.PhoenixDriver
 import org.apache.phoenix.mapreduce.PhoenixInputFormat
 import org.apache.phoenix.mapreduce.util.PhoenixConfigurationUtil
-import org.apache.phoenix.query.QueryConstants
-import org.apache.phoenix.schema.types._
-import org.apache.phoenix.util.{ColumnInfo, SchemaUtil}
 import org.apache.spark._
 import org.apache.spark.annotation.DeveloperApi
 import org.apache.spark.rdd.RDD
@@ -32,6 +29,7 @@ import org.apache.spark.sql.{DataFrame, Row, SQLContext}
 
 import scala.collection.JavaConverters._
 
+@deprecated("Use the DataSource V2 API implementation (see PhoenixDataSource)")
 class PhoenixRDD(sc: SparkContext, table: String, columns: Seq[String],
                  predicate: Option[String] = None,
                  zkUrl: Option[String] = None,
@@ -126,7 +124,7 @@ class PhoenixRDD(sc: SparkContext, table: String, columns: Seq[String],
 
 
     // Lookup the Spark catalyst types from the Phoenix schema
-    val structFields = phoenixSchemaToCatalystSchema(columnInfoList).toArray
+    val structType = SparkSchemaUtil.phoenixSchemaToCatalystSchema(columnInfoList, dateAsTimestamp)
 
     // Create the data frame from the converted Spark schema
     sqlContext.createDataFrame(map(pr => {
@@ -146,60 +144,7 @@ class PhoenixRDD(sc: SparkContext, table: String, columns: Seq[String],
 
       // Create a Spark Row from the sequence
       Row.fromSeq(rowSeq)
-    }), new StructType(structFields))
+    }), structType)
   }
 
-  def normalizeColumnName(columnName: String) = {
-    val unescapedColumnName = SchemaUtil.getUnEscapedFullColumnName(columnName)
-    var normalizedColumnName = ""
-    if (unescapedColumnName.indexOf(QueryConstants.NAME_SEPARATOR) < 0) {
-      normalizedColumnName = unescapedColumnName
-    }
-    else {
-      // split by separator to get the column family and column name
-      val tokens = unescapedColumnName.split(QueryConstants.NAME_SEPARATOR_REGEX)
-      normalizedColumnName = if (tokens(0) == "0") tokens(1) else unescapedColumnName
-    }
-    normalizedColumnName
-  }
-
-  def phoenixSchemaToCatalystSchema(columnList: Seq[ColumnInfo]) = columnList.map(ci => {
-    val structType = phoenixTypeToCatalystType(ci)
-    StructField(normalizeColumnName(ci.getColumnName), structType)
-  })
-
-
-  // Lookup table for Phoenix types to Spark catalyst types
-  def phoenixTypeToCatalystType(columnInfo: ColumnInfo): DataType = columnInfo.getPDataType match {
-    case t if t.isInstanceOf[PVarchar] || t.isInstanceOf[PChar] => StringType
-    case t if t.isInstanceOf[PLong] || t.isInstanceOf[PUnsignedLong] => LongType
-    case t if t.isInstanceOf[PInteger] || t.isInstanceOf[PUnsignedInt] => IntegerType
-    case t if t.isInstanceOf[PSmallint] || t.isInstanceOf[PUnsignedSmallint] => ShortType
-    case t if t.isInstanceOf[PTinyint] || t.isInstanceOf[PUnsignedTinyint] => ByteType
-    case t if t.isInstanceOf[PFloat] || t.isInstanceOf[PUnsignedFloat] => FloatType
-    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 == 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
-    case t if (t.isInstanceOf[PDate] || t.isInstanceOf[PUnsignedDate]) && dateAsTimestamp => TimestampType
-    case t if t.isInstanceOf[PBoolean] => BooleanType
-    case t if t.isInstanceOf[PVarbinary] || t.isInstanceOf[PBinary] => BinaryType
-    case t if t.isInstanceOf[PIntegerArray] || t.isInstanceOf[PUnsignedIntArray] => ArrayType(IntegerType, containsNull = true)
-    case t if t.isInstanceOf[PBooleanArray] => ArrayType(BooleanType, containsNull = true)
-    case t if t.isInstanceOf[PVarcharArray] || t.isInstanceOf[PCharArray] => ArrayType(StringType, containsNull = true)
-    case t if t.isInstanceOf[PVarbinaryArray] || t.isInstanceOf[PBinaryArray] => ArrayType(BinaryType, containsNull = true)
-    case t if t.isInstanceOf[PLongArray] || t.isInstanceOf[PUnsignedLongArray] => ArrayType(LongType, containsNull = true)
-    case t if t.isInstanceOf[PSmallintArray] || t.isInstanceOf[PUnsignedSmallintArray] => ArrayType(IntegerType, containsNull = true)
-    case t if t.isInstanceOf[PTinyintArray] || t.isInstanceOf[PUnsignedTinyintArray] => ArrayType(ByteType, containsNull = true)
-    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 == 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)
-  }
 }
diff --git a/phoenix-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRecordWritable.scala b/phoenix-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRecordWritable.scala
index c35cc54..6d4c4cc 100644
--- a/phoenix-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRecordWritable.scala
+++ b/phoenix-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRecordWritable.scala
@@ -20,7 +20,7 @@ import org.apache.phoenix.util.ColumnInfo
 import org.joda.time.DateTime
 import scala.collection.{mutable, immutable}
 
-
+@deprecated("Use the DataSource V2 API implementation (see PhoenixDataSource)")
 class PhoenixRecordWritable(columnMetaDataList: List[ColumnInfo]) extends DBWritable {
   val upsertValues = mutable.ArrayBuffer[Any]()
   val resultMap = mutable.Map[String, AnyRef]()
diff --git a/phoenix-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRelation.scala b/phoenix-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRelation.scala
index 38bf29a..2f6ea8c 100644
--- a/phoenix-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRelation.scala
+++ b/phoenix-spark/src/main/scala/org/apache/phoenix/spark/PhoenixRelation.scala
@@ -19,12 +19,11 @@ package org.apache.phoenix.spark
 
 import org.apache.hadoop.conf.Configuration
 import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.sources._
 import org.apache.spark.sql.types.StructType
 import org.apache.spark.sql.{Row, SQLContext}
-import org.apache.spark.sql.sources._
-import org.apache.phoenix.util.StringUtil.escapeStringConstant
-import org.apache.phoenix.util.SchemaUtil
 
+@deprecated("Use the DataSource V2 API implementation (see PhoenixDataSource)")
 case class PhoenixRelation(tableName: String, zkUrl: String, dateAsTimestamp: Boolean = false)(@transient val sqlContext: SQLContext)
     extends BaseRelation with PrunedFilteredScan {
 
@@ -36,7 +35,7 @@ case class PhoenixRelation(tableName: String, zkUrl: String, dateAsTimestamp: Bo
     but this prevents having to load the whole table into Spark first.
   */
   override def buildScan(requiredColumns: Array[String], filters: Array[Filter]): RDD[Row] = {
-    val(pushedFilters, unhandledFilters) = buildFilter(filters)
+    val(pushedFilters, _, _) = new FilterExpressionCompiler().pushFilters(filters)
     new PhoenixRDD(
       sqlContext.sparkContext,
       tableName,
@@ -61,71 +60,10 @@ case class PhoenixRelation(tableName: String, zkUrl: String, dateAsTimestamp: Bo
     ).toDataFrame(sqlContext).schema
   }
 
-  // Attempt to create Phoenix-accepted WHERE clauses from Spark filters,
-  // mostly inspired from Spark SQL JDBCRDD and the couchbase-spark-connector
-  private def buildFilter(filters: Array[Filter]): (String, Array[Filter]) = {
-    if (filters.isEmpty) {
-      return ("" , Array[Filter]())
-    }
-
-    val filter = new StringBuilder("")
-    val unsupportedFilters = Array[Filter]();
-    var i = 0
-
-    filters.foreach(f => {
-      // Assume conjunction for multiple filters, unless otherwise specified
-      if (i > 0) {
-        filter.append(" AND")
-      }
-
-      f match {
-        // Spark 1.3.1+ supported filters
-        case And(leftFilter, rightFilter) => filter.append(buildFilter(Array(leftFilter, rightFilter)))
-        case Or(leftFilter, rightFilter) => filter.append(buildFilter(Array(leftFilter)) + " OR " + buildFilter(Array(rightFilter)))
-        case Not(aFilter) => filter.append(" NOT " + buildFilter(Array(aFilter)))
-        case EqualTo(attr, value) => filter.append(s" ${escapeKey(attr)} = ${compileValue(value)}")
-        case GreaterThan(attr, value) => filter.append(s" ${escapeKey(attr)} > ${compileValue(value)}")
-        case GreaterThanOrEqual(attr, value) => filter.append(s" ${escapeKey(attr)} >= ${compileValue(value)}")
-        case LessThan(attr, value) => filter.append(s" ${escapeKey(attr)} < ${compileValue(value)}")
-        case LessThanOrEqual(attr, value) => filter.append(s" ${escapeKey(attr)} <= ${compileValue(value)}")
-        case IsNull(attr) => filter.append(s" ${escapeKey(attr)} IS NULL")
-        case IsNotNull(attr) => filter.append(s" ${escapeKey(attr)} IS NOT NULL")
-        case In(attr, values) => filter.append(s" ${escapeKey(attr)} IN ${values.map(compileValue).mkString("(", ",", ")")}")
-        case StringStartsWith(attr, value) => filter.append(s" ${escapeKey(attr)} LIKE ${compileValue(value + "%")}")
-        case StringEndsWith(attr, value) => filter.append(s" ${escapeKey(attr)} LIKE ${compileValue("%" + value)}")
-        case StringContains(attr, value) => filter.append(s" ${escapeKey(attr)} LIKE ${compileValue("%" + value + "%")}")
-        case _ => unsupportedFilters :+ f
-      }
-
-      i = i + 1
-    })
-
-    (filter.toString(), unsupportedFilters)
-  }
 
   override def unhandledFilters(filters: Array[Filter]): Array[Filter] = {
-    val(pushedFilters, unhandledFilters) = buildFilter(filters)
+    val (_, unhandledFilters, _) = new FilterExpressionCompiler().pushFilters(filters)
     unhandledFilters
   }
 
-  // Helper function to escape column key to work with SQL queries
-  private def escapeKey(key: String): String = SchemaUtil.getEscapedArgument(key)
-
-  // Helper function to escape string values in SQL queries
-  private def compileValue(value: Any): Any = value match {
-    case stringValue: String => s"'${escapeStringConstant(stringValue)}'"
-
-    // Borrowed from 'elasticsearch-hadoop', support these internal UTF types across Spark versions
-    // Spark 1.4
-    case utf if (isClass(utf, "org.apache.spark.sql.types.UTF8String")) => s"'${escapeStringConstant(utf.toString)}'"
-    // Spark 1.5
-    case utf if (isClass(utf, "org.apache.spark.unsafe.types.UTF8String")) => s"'${escapeStringConstant(utf.toString)}'"
-
-    // Pass through anything else
-    case _ => value
-  }
-
-  private def isClass(obj: Any, className: String) = {
-    className.equals(obj.getClass().getName())
-  }
 }
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 1b33e6e..b073521 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
@@ -21,6 +21,7 @@ import org.apache.spark.rdd.RDD
 
 import scala.collection.JavaConversions._
 
+@deprecated("Use the DataSource V2 API implementation (see PhoenixDataSource)")
 class ProductRDDFunctions[A <: Product](data: RDD[A]) extends Serializable {
 
   def saveToPhoenix(tableName: String, cols: Seq[String],
diff --git a/phoenix-spark/src/main/scala/org/apache/phoenix/spark/SparkContextFunctions.scala b/phoenix-spark/src/main/scala/org/apache/phoenix/spark/SparkContextFunctions.scala
index 476ce8a..1b377ab 100644
--- a/phoenix-spark/src/main/scala/org/apache/phoenix/spark/SparkContextFunctions.scala
+++ b/phoenix-spark/src/main/scala/org/apache/phoenix/spark/SparkContextFunctions.scala
@@ -17,6 +17,7 @@ import org.apache.hadoop.conf.Configuration
 import org.apache.spark.SparkContext
 import org.apache.spark.rdd.RDD
 
+@deprecated("Use the DataSource V2 API implementation (see PhoenixDataSource)")
 class SparkContextFunctions(@transient val sc: SparkContext) extends Serializable {
 
   /*
diff --git a/phoenix-spark/src/main/scala/org/apache/phoenix/spark/SparkSchemaUtil.scala b/phoenix-spark/src/main/scala/org/apache/phoenix/spark/SparkSchemaUtil.scala
new file mode 100644
index 0000000..f69e988
--- /dev/null
+++ b/phoenix-spark/src/main/scala/org/apache/phoenix/spark/SparkSchemaUtil.scala
@@ -0,0 +1,84 @@
+/*
+ * 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.spark
+
+import org.apache.phoenix.query.QueryConstants
+import org.apache.phoenix.schema.types._
+import org.apache.phoenix.util.{ColumnInfo, SchemaUtil}
+import org.apache.spark.sql.types._
+
+object SparkSchemaUtil {
+
+  def phoenixSchemaToCatalystSchema(columnList: Seq[ColumnInfo], dateAsTimestamp: Boolean = false) : StructType = {
+    val structFields = columnList.map(ci => {
+      val structType = phoenixTypeToCatalystType(ci, dateAsTimestamp)
+      StructField(normalizeColumnName(ci.getColumnName), structType)
+    })
+    new StructType(structFields.toArray)
+  }
+
+  def normalizeColumnName(columnName: String) = {
+    val unescapedColumnName = SchemaUtil.getUnEscapedFullColumnName(columnName)
+    var normalizedColumnName = ""
+    if (unescapedColumnName.indexOf(QueryConstants.NAME_SEPARATOR) < 0) {
+      normalizedColumnName = unescapedColumnName
+    }
+    else {
+      // split by separator to get the column family and column name
+      val tokens = unescapedColumnName.split(QueryConstants.NAME_SEPARATOR_REGEX)
+      normalizedColumnName = if (tokens(0) == "0") tokens(1) else unescapedColumnName
+    }
+    normalizedColumnName
+  }
+
+
+  // Lookup table for Phoenix types to Spark catalyst types
+  def phoenixTypeToCatalystType(columnInfo: ColumnInfo, dateAsTimestamp: Boolean): DataType = columnInfo.getPDataType match {
+    case t if t.isInstanceOf[PVarchar] || t.isInstanceOf[PChar] => StringType
+    case t if t.isInstanceOf[PLong] || t.isInstanceOf[PUnsignedLong] => LongType
+    case t if t.isInstanceOf[PInteger] || t.isInstanceOf[PUnsignedInt] => IntegerType
+    case t if t.isInstanceOf[PSmallint] || t.isInstanceOf[PUnsignedSmallint] => ShortType
+    case t if t.isInstanceOf[PTinyint] || t.isInstanceOf[PUnsignedTinyint] => ByteType
+    case t if t.isInstanceOf[PFloat] || t.isInstanceOf[PUnsignedFloat] => FloatType
+    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 == 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
+    case t if (t.isInstanceOf[PDate] || t.isInstanceOf[PUnsignedDate]) && dateAsTimestamp => TimestampType
+    case t if t.isInstanceOf[PBoolean] => BooleanType
+    case t if t.isInstanceOf[PVarbinary] || t.isInstanceOf[PBinary] => BinaryType
+    case t if t.isInstanceOf[PIntegerArray] || t.isInstanceOf[PUnsignedIntArray] => ArrayType(IntegerType, containsNull = true)
+    case t if t.isInstanceOf[PBooleanArray] => ArrayType(BooleanType, containsNull = true)
+    case t if t.isInstanceOf[PVarcharArray] || t.isInstanceOf[PCharArray] => ArrayType(StringType, containsNull = true)
+    case t if t.isInstanceOf[PVarbinaryArray] || t.isInstanceOf[PBinaryArray] => ArrayType(BinaryType, containsNull = true)
+    case t if t.isInstanceOf[PLongArray] || t.isInstanceOf[PUnsignedLongArray] => ArrayType(LongType, containsNull = true)
+    case t if t.isInstanceOf[PSmallintArray] || t.isInstanceOf[PUnsignedSmallintArray] => ArrayType(IntegerType, containsNull = true)
+    case t if t.isInstanceOf[PTinyintArray] || t.isInstanceOf[PUnsignedTinyintArray] => ArrayType(ByteType, containsNull = true)
+    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 == 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)
+  }
+
+}
diff --git a/phoenix-spark/src/main/scala/org/apache/phoenix/spark/SparkSqlContextFunctions.scala b/phoenix-spark/src/main/scala/org/apache/phoenix/spark/SparkSqlContextFunctions.scala
index a0842c9..f9154ad 100644
--- a/phoenix-spark/src/main/scala/org/apache/phoenix/spark/SparkSqlContextFunctions.scala
+++ b/phoenix-spark/src/main/scala/org/apache/phoenix/spark/SparkSqlContextFunctions.scala
@@ -16,6 +16,7 @@ package org.apache.phoenix.spark
 import org.apache.hadoop.conf.Configuration
 import org.apache.spark.sql.{DataFrame, SQLContext}
 
+@deprecated("Use the DataSource V2 API implementation (see PhoenixDataSource)")
 class SparkSqlContextFunctions(@transient val sqlContext: SQLContext) extends Serializable {
 
   /*
diff --git a/phoenix-spark/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/PhoenixJdbcDialect.scala b/phoenix-spark/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/PhoenixJdbcDialect.scala
new file mode 100644
index 0000000..712ec2d
--- /dev/null
+++ b/phoenix-spark/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/PhoenixJdbcDialect.scala
@@ -0,0 +1,21 @@
+package org.apache.spark.sql.execution.datasources.jdbc
+
+import org.apache.spark.sql.jdbc.{JdbcDialect, JdbcType}
+import org.apache.spark.sql.types._
+
+private object PhoenixJdbcDialect  extends JdbcDialect {
+
+  override def canHandle(url: String): Boolean = url.startsWith("jdbc:phoenix")
+
+  /**
+    * This is only called for ArrayType (see JdbcUtils.makeSetter)
+    */
+  override def getJDBCType(dt: DataType): Option[JdbcType] = dt match {
+    case StringType => Some(JdbcType("VARCHAR", java.sql.Types.VARCHAR))
+    case BinaryType => Some(JdbcType("BINARY(" + dt.defaultSize + ")", java.sql.Types.BINARY))
+    case ByteType => Some(JdbcType("TINYINT", java.sql.Types.TINYINT))
+    case _ => None
+  }
+
+
+}
\ No newline at end of file
diff --git a/phoenix-spark/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/SparkJdbcUtil.scala b/phoenix-spark/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/SparkJdbcUtil.scala
new file mode 100644
index 0000000..eac483a
--- /dev/null
+++ b/phoenix-spark/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/SparkJdbcUtil.scala
@@ -0,0 +1,309 @@
+/*
+ * 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.spark.sql.execution.datasources
+
+import java.sql.{Connection, PreparedStatement, ResultSet}
+import java.util.Locale
+
+import org.apache.spark.executor.InputMetrics
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.encoders.RowEncoder
+import org.apache.spark.sql.catalyst.expressions.SpecificInternalRow
+import org.apache.spark.sql.catalyst.util.{DateTimeUtils, GenericArrayData}
+import org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils
+import org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils._
+import org.apache.spark.sql.jdbc.{JdbcDialect, JdbcType}
+import org.apache.spark.sql.types._
+import org.apache.spark.unsafe.types.UTF8String
+import org.apache.spark.util.NextIterator
+
+object SparkJdbcUtil {
+
+  def toRow(schema: StructType, internalRow: InternalRow) : Row = {
+    val encoder = RowEncoder(schema).resolveAndBind()
+    encoder.fromRow(internalRow)
+  }
+
+  // A `JDBCValueGetter` is responsible for getting a value from `ResultSet` into a field
+  // for `MutableRow`. The last argument `Int` means the index for the value to be set in
+  // the row and also used for the value in `ResultSet`.
+  private type JDBCValueGetter = (ResultSet, InternalRow, Int) => Unit
+
+  private def nullSafeConvert[T](input: T, f: T => Any): Any = {
+    if (input == null) {
+      null
+    } else {
+      f(input)
+    }
+  }
+
+  /**
+    * Creates `JDBCValueGetter`s according to [[StructType]], which can set
+    * each value from `ResultSet` to each field of [[InternalRow]] correctly.
+    */
+  private def makeGetters(schema: StructType): Array[JDBCValueGetter] =
+    schema.fields.map(sf => makeGetter(sf.dataType, sf.metadata))
+
+  private def makeGetter(dt: DataType, metadata: Metadata): JDBCValueGetter = dt match {
+    case BooleanType =>
+      (rs: ResultSet, row: InternalRow, pos: Int) =>
+        row.setBoolean(pos, rs.getBoolean(pos + 1))
+
+    case DateType =>
+      (rs: ResultSet, row: InternalRow, pos: Int) =>
+        // DateTimeUtils.fromJavaDate does not handle null value, so we need to check it.
+        val dateVal = rs.getDate(pos + 1)
+        if (dateVal != null) {
+          row.setInt(pos, DateTimeUtils.fromJavaDate(dateVal))
+        } else {
+          row.update(pos, null)
+        }
+
+    // When connecting with Oracle DB through JDBC, the precision and scale of BigDecimal
+    // object returned by ResultSet.getBigDecimal is not correctly matched to the table
+    // schema reported by ResultSetMetaData.getPrecision and ResultSetMetaData.getScale.
+    // If inserting values like 19999 into a column with NUMBER(12, 2) type, you get through
+    // a BigDecimal object with scale as 0. But the dataframe schema has correct type as
+    // DecimalType(12, 2). Thus, after saving the dataframe into parquet file and then
+    // retrieve it, you will get wrong result 199.99.
+    // So it is needed to set precision and scale for Decimal based on JDBC metadata.
+    case DecimalType.Fixed(p, s) =>
+      (rs: ResultSet, row: InternalRow, pos: Int) =>
+        val decimal =
+          nullSafeConvert[java.math.BigDecimal](rs.getBigDecimal(pos + 1), d => Decimal(d, p, s))
+        row.update(pos, decimal)
+
+    case DoubleType =>
+      (rs: ResultSet, row: InternalRow, pos: Int) =>
+        row.setDouble(pos, rs.getDouble(pos + 1))
+
+    case FloatType =>
+      (rs: ResultSet, row: InternalRow, pos: Int) =>
+        row.setFloat(pos, rs.getFloat(pos + 1))
+
+    case IntegerType =>
+      (rs: ResultSet, row: InternalRow, pos: Int) =>
+        row.setInt(pos, rs.getInt(pos + 1))
+
+    case LongType if metadata.contains("binarylong") =>
+      (rs: ResultSet, row: InternalRow, pos: Int) =>
+        val bytes = rs.getBytes(pos + 1)
+        var ans = 0L
+        var j = 0
+        while (j < bytes.length) {
+          ans = 256 * ans + (255 & bytes(j))
+          j = j + 1
+        }
+        row.setLong(pos, ans)
+
+    case LongType =>
+      (rs: ResultSet, row: InternalRow, pos: Int) =>
+        row.setLong(pos, rs.getLong(pos + 1))
+
+    case ShortType =>
+      (rs: ResultSet, row: InternalRow, pos: Int) =>
+        row.setShort(pos, rs.getShort(pos + 1))
+
+    case StringType =>
+      (rs: ResultSet, row: InternalRow, pos: Int) =>
+        // TODO(davies): use getBytes for better performance, if the encoding is UTF-8
+        row.update(pos, UTF8String.fromString(rs.getString(pos + 1)))
+
+    case TimestampType =>
+      (rs: ResultSet, row: InternalRow, pos: Int) =>
+        val t = rs.getTimestamp(pos + 1)
+        if (t != null) {
+          row.setLong(pos, DateTimeUtils.fromJavaTimestamp(t))
+        } else {
+          row.update(pos, null)
+        }
+
+    case BinaryType =>
+      (rs: ResultSet, row: InternalRow, pos: Int) =>
+        row.update(pos, rs.getBytes(pos + 1))
+
+    case ByteType =>
+      (rs: ResultSet, row: InternalRow, pos: Int) =>
+        row.update(pos, rs.getByte(pos + 1))
+
+    case ArrayType(et, _) =>
+      val elementConversion = et match {
+        case TimestampType =>
+          (array: Object) =>
+            array.asInstanceOf[Array[java.sql.Timestamp]].map { timestamp =>
+              nullSafeConvert(timestamp, DateTimeUtils.fromJavaTimestamp)
+            }
+
+        case StringType =>
+          (array: Object) =>
+            // some underling types are not String such as uuid, inet, cidr, etc.
+            array.asInstanceOf[Array[java.lang.Object]]
+              .map(obj => if (obj == null) null else UTF8String.fromString(obj.toString))
+
+        case DateType =>
+          (array: Object) =>
+            array.asInstanceOf[Array[java.sql.Date]].map { date =>
+              nullSafeConvert(date, DateTimeUtils.fromJavaDate)
+            }
+
+        case dt: DecimalType =>
+          (array: Object) =>
+            array.asInstanceOf[Array[java.math.BigDecimal]].map { decimal =>
+              nullSafeConvert[java.math.BigDecimal](
+                decimal, d => Decimal(d, dt.precision, dt.scale))
+            }
+
+        case LongType if metadata.contains("binarylong") =>
+          throw new IllegalArgumentException(s"Unsupported array element " +
+            s"type ${dt.catalogString} based on binary")
+
+        case ArrayType(_, _) =>
+          throw new IllegalArgumentException("Nested arrays unsupported")
+
+        case _ => (array: Object) => array.asInstanceOf[Array[Any]]
+      }
+
+      (rs: ResultSet, row: InternalRow, pos: Int) =>
+        val array = nullSafeConvert[java.sql.Array](
+          input = rs.getArray(pos + 1),
+          array => new GenericArrayData(elementConversion.apply(array.getArray)))
+        row.update(pos, array)
+
+    case _ => throw new IllegalArgumentException(s"Unsupported type ${dt.catalogString}")
+  }
+
+  // TODO just use JdbcUtils.resultSetToSparkInternalRows in Spark 3.0 (see SPARK-26499)
+  def resultSetToSparkInternalRows(
+                                    resultSet: ResultSet,
+                                    schema: StructType,
+                                    inputMetrics: InputMetrics): Iterator[InternalRow] = {
+    // JdbcUtils.resultSetToSparkInternalRows(resultSet, schema, inputMetrics)
+    new NextIterator[InternalRow] {
+      private[this] val rs = resultSet
+      private[this] val getters: Array[JDBCValueGetter] = makeGetters(schema)
+      private[this] val mutableRow = new SpecificInternalRow(schema.fields.map(x => x.dataType))
+
+      override protected def close(): Unit = {
+        try {
+          rs.close()
+        } catch {
+          case e: Exception =>
+        }
+      }
+
+      override protected def getNext(): InternalRow = {
+        if (rs.next()) {
+          inputMetrics.incRecordsRead(1)
+          var i = 0
+          while (i < getters.length) {
+            getters(i).apply(rs, mutableRow, i)
+            if (rs.wasNull) mutableRow.setNullAt(i)
+            i = i + 1
+          }
+          mutableRow
+        } else {
+          finished = true
+          null.asInstanceOf[InternalRow]
+        }
+      }
+    }
+  }
+
+  // A `JDBCValueSetter` is responsible for setting a value from `Row` into a field for
+  // `PreparedStatement`. The last argument `Int` means the index for the value to be set
+  // in the SQL statement and also used for the value in `Row`.
+  private type JDBCValueSetter = (PreparedStatement, Row, Int) => Unit
+
+  // take from Spark JdbcUtils.scala, cannot be used directly because the method is private
+  def makeSetter(
+                  conn: Connection,
+                  dialect: JdbcDialect,
+                  dataType: DataType): JDBCValueSetter = dataType match {
+    case IntegerType =>
+      (stmt: PreparedStatement, row: Row, pos: Int) =>
+        stmt.setInt(pos + 1, row.getInt(pos))
+
+    case LongType =>
+      (stmt: PreparedStatement, row: Row, pos: Int) =>
+        stmt.setLong(pos + 1, row.getLong(pos))
+
+    case DoubleType =>
+      (stmt: PreparedStatement, row: Row, pos: Int) =>
+        stmt.setDouble(pos + 1, row.getDouble(pos))
+
+    case FloatType =>
+      (stmt: PreparedStatement, row: Row, pos: Int) =>
+        stmt.setFloat(pos + 1, row.getFloat(pos))
+
+    case ShortType =>
+      (stmt: PreparedStatement, row: Row, pos: Int) =>
+        stmt.setInt(pos + 1, row.getShort(pos))
+
+    case ByteType =>
+      (stmt: PreparedStatement, row: Row, pos: Int) =>
+        stmt.setInt(pos + 1, row.getByte(pos))
+
+    case BooleanType =>
+      (stmt: PreparedStatement, row: Row, pos: Int) =>
+        stmt.setBoolean(pos + 1, row.getBoolean(pos))
+
+    case StringType =>
+      (stmt: PreparedStatement, row: Row, pos: Int) =>
+        stmt.setString(pos + 1, row.getString(pos))
+
+    case BinaryType =>
+      (stmt: PreparedStatement, row: Row, pos: Int) =>
+        stmt.setBytes(pos + 1, row.getAs[Array[Byte]](pos))
+
+    case TimestampType =>
+      (stmt: PreparedStatement, row: Row, pos: Int) =>
+        stmt.setTimestamp(pos + 1, row.getAs[java.sql.Timestamp](pos))
+
+    case DateType =>
+      (stmt: PreparedStatement, row: Row, pos: Int) =>
+        stmt.setDate(pos + 1, row.getAs[java.sql.Date](pos))
+
+    case t: DecimalType =>
+      (stmt: PreparedStatement, row: Row, pos: Int) =>
+        stmt.setBigDecimal(pos + 1, row.getDecimal(pos))
+
+    case ArrayType(et, _) =>
+      // remove type length parameters from end of type name
+      val typeName = getJdbcType(et, dialect).databaseTypeDefinition
+        .toLowerCase(Locale.ROOT).split("\\(")(0)
+      (stmt: PreparedStatement, row: Row, pos: Int) =>
+        val array = conn.createArrayOf(
+          typeName,
+          row.getSeq[AnyRef](pos).toArray)
+        stmt.setArray(pos + 1, array)
+
+    case _ =>
+      (_: PreparedStatement, _: Row, pos: Int) =>
+        throw new IllegalArgumentException(
+          s"Can't translate non-null value for field $pos")
+  }
+
+  // taken from Spark JdbcUtils
+  def getJdbcType(dt: DataType, dialect: JdbcDialect): JdbcType = {
+    dialect.getJDBCType(dt).orElse(getCommonJDBCType(dt)).getOrElse(
+      throw new IllegalArgumentException(s"Can't get JDBC type for ${dt.catalogString}"))
+  }
+
+}


[phoenix] 12/18: PHOENIX-5074 DropTableWithViewsIT.testDropTableWithChildViews is flapping

Posted by pb...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

pboado pushed a commit to branch 4.x-cdh5.16
in repository https://gitbox.apache.org/repos/asf/phoenix.git

commit 3f17a89e6c70b8f47023175e585e216dcfec5731
Author: Kadir <ko...@salesforce.com>
AuthorDate: Thu Dec 20 19:38:44 2018 +0000

    PHOENIX-5074 DropTableWithViewsIT.testDropTableWithChildViews is flapping
---
 .../phoenix/end2end/DropTableWithViewsIT.java      | 56 ++++++++++++----------
 1 file changed, 30 insertions(+), 26 deletions(-)

diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DropTableWithViewsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DropTableWithViewsIT.java
index 9502218..a4cd354 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DropTableWithViewsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DropTableWithViewsIT.java
@@ -20,7 +20,6 @@ package org.apache.phoenix.end2end;
 import static org.apache.phoenix.util.PhoenixRuntime.TENANT_ID_ATTRIB;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.fail;
 
 import java.sql.Connection;
 import java.sql.DriverManager;
@@ -30,14 +29,16 @@ import java.util.Collection;
 
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.phoenix.coprocessor.TableViewFinderResult;
+import org.apache.phoenix.coprocessor.TaskRegionObserver;
 import org.apache.phoenix.coprocessor.ViewFinder;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 
-import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.util.SchemaUtil;
+import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
@@ -50,6 +51,20 @@ public class DropTableWithViewsIT extends SplitSystemCatalogIT {
     private final boolean columnEncoded;
     private final String TENANT_SPECIFIC_URL1 = getUrl() + ';' + TENANT_ID_ATTRIB + "=" + TENANT1;
 
+    private static RegionCoprocessorEnvironment TaskRegionEnvironment;
+
+    @BeforeClass
+    public static void doSetup() throws Exception {
+        SplitSystemCatalogIT.doSetup();
+        TaskRegionEnvironment =
+                getUtility()
+                        .getRSForFirstRegionInTable(
+                                PhoenixDatabaseMetaData.SYSTEM_TASK_HBASE_TABLE_NAME)
+                        .getRegions(PhoenixDatabaseMetaData.SYSTEM_TASK_HBASE_TABLE_NAME)
+                        .get(0).getCoprocessorHost()
+                        .findCoprocessorEnvironment(TaskRegionObserver.class.getName());
+    }
+
     public DropTableWithViewsIT(boolean isMultiTenant, boolean columnEncoded) {
         this.isMultiTenant = isMultiTenant;
         this.columnEncoded = columnEncoded;
@@ -108,30 +123,19 @@ public class DropTableWithViewsIT extends SplitSystemCatalogIT {
             // Drop the base table
             String dropTable = String.format("DROP TABLE IF EXISTS %s CASCADE", baseTable);
             conn.createStatement().execute(dropTable);
-
-            // Wait for the tasks for dropping child views to complete. The depth of the view tree is 2, so we expect that
-            // this will be done in two task handling runs, i.e., in tree task handling interval at most in general
-            // by assuming that each non-root level will be processed in one interval. To be on the safe side, we will
-            // wait at most 10 intervals.
-            long halfTimeInterval = config.getLong(QueryServices.TASK_HANDLING_INTERVAL_MS_ATTRIB,
-                    QueryServicesOptions.DEFAULT_TASK_HANDLING_INTERVAL_MS)/2;
-            ResultSet rs = null;
-            boolean timedOut = true;
-            Thread.sleep(3 * halfTimeInterval);
-            for (int i = 3; i < 20; i++) {
-                rs = conn.createStatement().executeQuery("SELECT * " +
-                                " FROM " + PhoenixDatabaseMetaData.SYSTEM_TASK_NAME +
-                                " WHERE " + PhoenixDatabaseMetaData.TASK_TYPE + " = " +
-                                PTable.TaskType.DROP_CHILD_VIEWS.getSerializedValue());
-                Thread.sleep(halfTimeInterval);
-                if (!rs.next()) {
-                    timedOut = false;
-                    break;
-                }
-            }
-            if (timedOut) {
-                fail("Drop child view task execution timed out!");
-            }
+            // Run DropChildViewsTask to complete the tasks for dropping child views. The depth of the view tree is 2,
+            // so we expect that this will be done in two task handling runs as each non-root level will be processed
+            // in one run
+            TaskRegionObserver.DropChildViewsTask task =
+                    new TaskRegionObserver.DropChildViewsTask(
+                            TaskRegionEnvironment, QueryServicesOptions.DEFAULT_TASK_HANDLING_MAX_INTERVAL_MS);
+            task.run();
+            task.run();
+            ResultSet rs = conn.createStatement().executeQuery("SELECT * " +
+                    " FROM " + PhoenixDatabaseMetaData.SYSTEM_TASK_NAME +
+                    " WHERE " + PhoenixDatabaseMetaData.TASK_TYPE + " = " +
+                    PTable.TaskType.DROP_CHILD_VIEWS.getSerializedValue());
+            assertFalse(rs.next());
             // Views should be dropped by now
             TableName linkTable = TableName.valueOf(PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_NAME_BYTES);
             TableViewFinderResult childViewsResult = new TableViewFinderResult();


[phoenix] 05/18: PHOENIX-5025 Tool to clean up orphan views

Posted by pb...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

pboado pushed a commit to branch 4.x-cdh5.16
in repository https://gitbox.apache.org/repos/asf/phoenix.git

commit dff179b6c184bfeb4d28c090241cf08577ec4d85
Author: Kadir <ko...@salesforce.com>
AuthorDate: Tue Nov 13 06:24:10 2018 +0000

    PHOENIX-5025 Tool to clean up orphan views
---
 .../apache/phoenix/end2end/OrphanViewToolIT.java   | 472 +++++++++++
 .../apache/phoenix/mapreduce/OrphanViewTool.java   | 879 +++++++++++++++++++++
 2 files changed, 1351 insertions(+)

diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/OrphanViewToolIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/OrphanViewToolIT.java
new file mode 100644
index 0000000..f9a1785
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/OrphanViewToolIT.java
@@ -0,0 +1,472 @@
+/*
+ * 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.jdbc.PhoenixDatabaseMetaData.LINK_TYPE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_NAME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_SCHEM;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_TYPE;
+import static org.apache.phoenix.util.PhoenixRuntime.TENANT_ID_ATTRIB;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+import com.google.common.collect.Lists;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.phoenix.mapreduce.OrphanViewTool;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTableType;
+import org.apache.phoenix.util.SchemaUtil;
+import org.junit.AfterClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@RunWith(Parameterized.class)
+public class OrphanViewToolIT extends ParallelStatsDisabledIT {
+    private static final Logger LOG = LoggerFactory.getLogger(OrphanViewToolIT.class);
+
+    private final boolean isMultiTenant;
+    private final boolean columnEncoded;
+
+    private static final long fanout = 2;
+    private static final long childCount = fanout;
+    private static final long grandChildCount = fanout * fanout;
+    private static final long grandGrandChildCount = fanout * fanout * fanout;
+
+    private static final String filePath = "/tmp/";
+    private static final String viewFileName = "/tmp/" + OrphanViewTool.fileName[OrphanViewTool.VIEW];
+    private static final String physicalLinkFileName = "/tmp/" + OrphanViewTool.fileName[OrphanViewTool.PHYSICAL_TABLE_LINK];
+    private static final String parentLinkFileName = "/tmp/" + OrphanViewTool.fileName[OrphanViewTool.PARENT_TABLE_LINK];
+    private static final String childLinkFileName = "/tmp/" + OrphanViewTool.fileName[OrphanViewTool.CHILD_TABLE_LINK];
+
+    protected static String SCHEMA1 = "SCHEMA1";
+    protected static String SCHEMA2 = "SCHEMA2";
+    protected static String SCHEMA3 = "SCHEMA3";
+    protected static String SCHEMA4 = "SCHEMA4";
+
+    private final String TENANT_SPECIFIC_URL = getUrl() + ';' + TENANT_ID_ATTRIB + "=tenant";
+
+    private static final String createBaseTableFirstPartDDL = "CREATE TABLE IF NOT EXISTS %s";
+    private static final String createBaseTableSecondPartDDL = "(%s PK2 VARCHAR NOT NULL, V1 VARCHAR, V2 VARCHAR " +
+            " CONSTRAINT NAME_PK PRIMARY KEY (%s PK2)) %s";
+    private static final String deleteTableRows = "DELETE FROM " + SYSTEM_CATALOG_NAME +
+            " WHERE " + TABLE_SCHEM + " %s AND " +
+            TABLE_TYPE + " = '" + PTableType.TABLE.getSerializedValue() + "'";
+
+    private static final String createViewDDL = "CREATE VIEW %s AS SELECT * FROM %s";
+    private static final String countAllViewsQuery = "SELECT COUNT(*) FROM " + SYSTEM_CATALOG_NAME +
+            " WHERE " + TABLE_TYPE + " = '" + PTableType.VIEW.getSerializedValue() + "'";
+    private static final String countViewsQuery = "SELECT COUNT(*) FROM " + SYSTEM_CATALOG_NAME +
+            " WHERE " + TABLE_SCHEM + " %s AND " +
+            TABLE_TYPE + " = '" + PTableType.VIEW.getSerializedValue() + "'";
+    private static final String deleteViewRows = "DELETE FROM " + SYSTEM_CATALOG_NAME +
+            " WHERE " + TABLE_SCHEM + " %s AND " +
+            TABLE_TYPE + " = '" + PTableType.VIEW.getSerializedValue() + "'";
+
+    private static final String countChildLinksQuery = "SELECT COUNT(*) FROM " + SYSTEM_CHILD_LINK_NAME +
+            " WHERE " + TABLE_SCHEM + " %s AND " +
+            LINK_TYPE + " = " + PTable.LinkType.CHILD_TABLE.getSerializedValue();
+    private static final String deleteChildLinks = "DELETE FROM " + SYSTEM_CHILD_LINK_NAME +
+            " WHERE " + TABLE_SCHEM + " %s AND " +
+            LINK_TYPE + " = " + PTable.LinkType.CHILD_TABLE.getSerializedValue();
+
+    private static final String countParentLinksQuery = "SELECT COUNT(*) FROM " + SYSTEM_CATALOG_NAME +
+            " WHERE " + TABLE_SCHEM + " %s AND " +
+            LINK_TYPE + " = " + PTable.LinkType.PARENT_TABLE.getSerializedValue();
+    private static final String deleteParentLinks = "DELETE FROM " + SYSTEM_CATALOG_NAME +
+            " WHERE " + TABLE_SCHEM + " %s AND " +
+            LINK_TYPE + " = " + PTable.LinkType.PARENT_TABLE.getSerializedValue();
+
+    private static final String countPhysicalLinksQuery = "SELECT COUNT(*) FROM " + SYSTEM_CATALOG_NAME +
+            " WHERE " + TABLE_SCHEM + " %s AND " +
+            LINK_TYPE + " = " + PTable.LinkType.PHYSICAL_TABLE.getSerializedValue();
+    private static final String deletePhysicalLinks = "DELETE FROM " + SYSTEM_CATALOG_NAME +
+            " WHERE " + TABLE_SCHEM + " %s AND " +
+            LINK_TYPE + " = " + PTable.LinkType.PHYSICAL_TABLE.getSerializedValue();
+
+    private static final String deleteSchemaRows = "DELETE FROM %s WHERE " + TABLE_SCHEM + " %s";
+
+    public OrphanViewToolIT(boolean isMultiTenant, boolean columnEncoded) {
+        this.isMultiTenant = isMultiTenant;
+        this.columnEncoded = columnEncoded;
+    }
+
+    @Parameters(name="OrphanViewToolIT_multiTenant={0}, columnEncoded={1}") // name is used by failsafe as file name in reports
+    public static Collection<Boolean[]> data() {
+        return Arrays.asList(new Boolean[][] {
+                { false, false }, { false, true },
+                { true, false }, { true, true } });
+    }
+
+    @AfterClass
+    public static void cleanUp() {
+        for (int i = OrphanViewTool.VIEW; i < OrphanViewTool.ORPHAN_TYPE_COUNT; i++) {
+            File file = new File(filePath + OrphanViewTool.fileName[i]);
+            if (file.exists()) {
+                file.delete();
+            }
+        }
+    }
+
+    private String generateDDL(String format) {
+        return generateDDL("", format);
+    }
+
+    private String generateDDL(String options, String format) {
+        StringBuilder optionsBuilder = new StringBuilder(options);
+        if (!columnEncoded) {
+            if (optionsBuilder.length()!=0)
+                optionsBuilder.append(",");
+            optionsBuilder.append("COLUMN_ENCODED_BYTES=0");
+        }
+        if (isMultiTenant) {
+            if (optionsBuilder.length()!=0)
+                optionsBuilder.append(",");
+            optionsBuilder.append("MULTI_TENANT=true");
+        }
+        return String.format(format, isMultiTenant ? "TENANT_ID VARCHAR NOT NULL, " : "",
+                isMultiTenant ? "TENANT_ID, " : "", optionsBuilder.toString());
+    }
+
+    private void deleteRowsFrom(Connection connection, String systemTableName, String baseTableSchema,
+                                String childViewSchemaName,
+                                String grandchildViewSchemaName, String grandGrandChildViewSchemaName)
+            throws SQLException {
+        connection.createStatement().execute(String.format(deleteSchemaRows, systemTableName,
+                baseTableSchema == null ? "IS NULL" : " = '" + baseTableSchema + "'"));
+        connection.createStatement().execute(String.format(deleteSchemaRows, systemTableName,
+                childViewSchemaName == null ? "IS NULL" : " = '" + childViewSchemaName + "'"));
+        connection.createStatement().execute(String.format(deleteSchemaRows, systemTableName,
+                grandchildViewSchemaName == null ? "IS NULL" : " = '" + grandchildViewSchemaName + "'"));
+        connection.createStatement().execute(String.format(deleteSchemaRows, systemTableName,
+                grandGrandChildViewSchemaName == null ? "IS NULL" : " = '" + grandGrandChildViewSchemaName + "'"));
+    }
+
+    private void deleteAllRows(Connection connection, String baseTableSchema,
+                            String childViewSchemaName,
+                            String grandchildViewSchemaName, String grandGrandChildViewSchemaName) throws SQLException {
+        deleteRowsFrom(connection, SYSTEM_CATALOG_NAME, baseTableSchema, childViewSchemaName,
+                grandchildViewSchemaName, grandGrandChildViewSchemaName);
+        deleteRowsFrom(connection, SYSTEM_CHILD_LINK_NAME, baseTableSchema, childViewSchemaName,
+                grandchildViewSchemaName, grandGrandChildViewSchemaName);
+        connection.commit();
+    }
+
+    private void createBaseTableAndViews(Connection baseTableConnection, String baseTableFullName,
+                                         Connection viewConnection, String childViewSchemaName,
+                                         String grandchildViewSchemaName, String grandGrandChildViewSchemaName)
+            throws SQLException {
+        baseTableConnection.createStatement().execute(generateDDL(String.format(createBaseTableFirstPartDDL,
+                baseTableFullName) + createBaseTableSecondPartDDL));
+        // Create a view tree (i.e., tree of views) with depth of 3
+        for (int i = 0; i < fanout; i++) {
+            String childView = SchemaUtil.getTableName(childViewSchemaName, generateUniqueName());
+            viewConnection.createStatement().execute(String.format(createViewDDL, childView, baseTableFullName));
+            for (int j = 0; j < fanout; j++) {
+                String grandchildView = SchemaUtil.getTableName(grandchildViewSchemaName, generateUniqueName());
+                viewConnection.createStatement().execute(String.format(createViewDDL, grandchildView, childView));
+                for (int k = 0; k < fanout; k++) {
+                    viewConnection.createStatement().execute(String.format(createViewDDL,
+                            SchemaUtil.getTableName(grandGrandChildViewSchemaName, generateUniqueName()),
+                            grandchildView));
+                }
+            }
+        }
+    }
+
+    private void verifyLineCount(String fileName, long lineCount) throws IOException {
+        if (Files.lines(Paths.get(fileName)).count() != lineCount)
+            LOG.debug(Files.lines(Paths.get(fileName)).count() + " != " + lineCount);
+        assertTrue(Files.lines(Paths.get(fileName)).count() == lineCount);
+    }
+
+    private void verifyCountQuery(Connection connection, String query, String schemaName, long count)
+            throws SQLException {
+        ResultSet rs = connection.createStatement().executeQuery(String.format(query,
+                schemaName == null ? "IS NULL" : "= '" + schemaName + "'"));
+        assertTrue(rs.next());
+        assertTrue(rs.getLong(1) == count);
+    }
+
+    @Test
+    public void testCreateTableAndViews() throws Exception {
+        String baseTableName = generateUniqueName();
+        String baseTableFullName = SchemaUtil.getTableName(SCHEMA1, baseTableName);
+        try (Connection connection = DriverManager.getConnection(getUrl());
+             Connection viewConnection =
+                     isMultiTenant ? DriverManager.getConnection(TENANT_SPECIFIC_URL) : connection) {
+            createBaseTableAndViews(connection, baseTableFullName, viewConnection, SCHEMA2, SCHEMA3, SCHEMA4);
+            // Run the orphan view tool to drop orphan views but no view should be dropped
+            runOrphanViewTool(true, false, true, false);
+            verifyOrphanFileLineCounts(0, 0, 0, 0);
+            // Verify that the views we have created are still in the system catalog table
+            ResultSet rs = connection.createStatement().executeQuery(countAllViewsQuery);
+            assertTrue(rs.next());
+            assertTrue(rs.getLong(1) == childCount + grandChildCount + grandGrandChildCount);
+            deleteAllRows(connection, SCHEMA1, SCHEMA2, SCHEMA3, SCHEMA4);
+        }
+    }
+
+
+    private void verifyNoChildLink(Connection connection, String viewSchemaName) throws Exception {
+        // Verify that there there is no link in the system child link table
+        verifyCountQuery(connection, countChildLinksQuery, viewSchemaName, 0);
+    }
+
+    private void verifyNoViewNoLinkInSystemCatalog(Connection connection, String viewSchemaName) throws Exception {
+        // Verify that the views and links have been removed from the system catalog table
+        verifyCountQuery(connection, countViewsQuery, viewSchemaName, 0);
+        verifyCountQuery(connection, countParentLinksQuery, viewSchemaName, 0);
+        verifyCountQuery(connection, countPhysicalLinksQuery, viewSchemaName, 0);
+    }
+
+    private void verifyOrphanFileLineCounts(long viewCount, long parentLinkCount,
+                                            long physicalLinkCount, long childLinkCount)
+        throws Exception {
+        verifyLineCount(viewFileName, viewCount);
+        verifyLineCount(parentLinkFileName, parentLinkCount);
+        verifyLineCount(physicalLinkFileName, physicalLinkCount);
+        verifyLineCount(childLinkFileName, childLinkCount);
+    }
+    private void executeDeleteQuery(Connection connection, String deleteQuery, String schemaName) throws Exception {
+        connection.createStatement().execute(String.format(deleteQuery,
+                schemaName == null ? "IS NULL" : "= '" + schemaName + "'"));
+        connection.commit();
+    }
+    @Test
+    public void testDeleteBaseTableRows() throws Exception {
+        String baseTableName = generateUniqueName();
+        String baseTableFullName = SchemaUtil.getTableName(SCHEMA1, baseTableName);
+        try (Connection connection = DriverManager.getConnection(getUrl());
+             Connection viewConnection =
+                     isMultiTenant ? DriverManager.getConnection(TENANT_SPECIFIC_URL) : connection) {
+            createBaseTableAndViews(connection, baseTableFullName, viewConnection, SCHEMA2, SCHEMA2, SCHEMA2);
+            // Delete the base table row from the system catalog
+            executeDeleteQuery(connection, deleteTableRows, SCHEMA1);
+            // Verify that the views we have created are still in the system catalog table
+            ResultSet rs = connection.createStatement().executeQuery(countAllViewsQuery);
+            assertTrue(rs.next());
+            assertTrue(rs.getLong(1) == childCount + grandChildCount + grandGrandChildCount);
+            // Run the orphan view tool to identify orphan views
+            runOrphanViewTool(false, true, true, false);
+            verifyOrphanFileLineCounts(childCount + grandChildCount + grandGrandChildCount,
+                    0,
+                    childCount + grandChildCount + grandGrandChildCount,
+                    childCount);
+            // Verify that orphan views have not yet dropped as we just identified them
+            rs = connection.createStatement().executeQuery(countAllViewsQuery);
+            assertTrue(rs.next());
+            assertTrue(rs.getLong(1) == childCount + grandChildCount + grandGrandChildCount);
+            // Drop the previously identified orphan views
+            runOrphanViewTool(true, false, false, true);
+            // Verify that the orphan views and links have been removed from the system catalog table
+            verifyNoViewNoLinkInSystemCatalog(connection, SCHEMA2);
+            // Verify that there there is no link in the system child link table
+            verifyNoChildLink(connection, SCHEMA1);
+            deleteAllRows(connection, SCHEMA1, SCHEMA2, SCHEMA3, SCHEMA4);
+        }
+    }
+
+    @Test
+    public void testDeleteChildViewRows() throws Exception {
+        String baseTableName = generateUniqueName();
+        String baseTableFullName = SchemaUtil.getTableName(SCHEMA1, baseTableName);
+        try (Connection connection = DriverManager.getConnection(getUrl());
+             Connection viewConnection =
+                     isMultiTenant ? DriverManager.getConnection(TENANT_SPECIFIC_URL) : connection) {
+            createBaseTableAndViews(connection, baseTableFullName, viewConnection, null, SCHEMA3, SCHEMA3);
+            // Delete the rows of the immediate child views of the base table from the system catalog
+            executeDeleteQuery(connection, deleteViewRows, null);
+            // Verify that the other views we have created are still in the system catalog table
+            verifyCountQuery(connection, countViewsQuery, SCHEMA3, grandChildCount + grandGrandChildCount);
+            // Run the orphan view tool to clean up orphan views
+            runOrphanViewTool(true, false, true, false);
+            // Verify that the tool attempt to remove all orphan views and links
+            verifyOrphanFileLineCounts(grandChildCount + grandGrandChildCount,
+                    grandChildCount,
+                    childCount,
+                    childCount + grandChildCount);
+            // Verify that all views and links records of the views are removed from the system catalog table
+            verifyNoViewNoLinkInSystemCatalog(connection, null);
+            verifyNoViewNoLinkInSystemCatalog(connection, SCHEMA3);
+            // Verify that there there is no link in the system child link table
+            verifyNoChildLink(connection, SCHEMA1);
+            verifyNoChildLink(connection, null);
+            deleteAllRows(connection, SCHEMA1, null, SCHEMA3, SCHEMA4);
+        }
+    }
+
+    @Test
+    public void testDeleteGrandchildViewRows() throws Exception {
+        String baseTableName = generateUniqueName();
+        String baseTableFullName = SchemaUtil.getTableName(SCHEMA1, baseTableName);
+        try (Connection connection = DriverManager.getConnection(getUrl());
+             Connection viewConnection =
+                     isMultiTenant ? DriverManager.getConnection(TENANT_SPECIFIC_URL) : connection) {
+            createBaseTableAndViews(connection, baseTableFullName, viewConnection, SCHEMA2, SCHEMA3, null);
+            // Delete the grand child view rows from the system catalog
+            executeDeleteQuery(connection, deleteViewRows, SCHEMA3);
+            // Verify that grand grand child views are still in the system catalog table
+            verifyCountQuery(connection, countViewsQuery, null, grandGrandChildCount);
+            // Run the orphan view tool to clean up orphan views
+            runOrphanViewTool(true, false, true, false);
+            // Verify that the orphan views and links have been removed
+            verifyOrphanFileLineCounts(grandGrandChildCount,
+                    grandChildCount + grandGrandChildCount,
+                    grandChildCount,
+                    grandChildCount + grandGrandChildCount);
+            // Verify that all views and links records for grand and grand grand child views are removed
+            // from the system catalog table
+            verifyNoViewNoLinkInSystemCatalog(connection, SCHEMA3);
+            verifyNoViewNoLinkInSystemCatalog(connection, null);
+            // Verify the child links are also removed
+            verifyNoChildLink(connection, SCHEMA2);
+            verifyNoChildLink(connection, SCHEMA3);
+            deleteAllRows(connection, SCHEMA1, SCHEMA2, SCHEMA3, null);
+        }
+    }
+
+    @Test
+    public void testDeleteParentChildLinkRows() throws Exception {
+        String baseTableName = generateUniqueName();
+        String baseTableFullName = SchemaUtil.getTableName(SCHEMA1, baseTableName);
+        try (Connection connection = DriverManager.getConnection(getUrl());
+             Connection viewConnection =
+                     isMultiTenant ? DriverManager.getConnection(TENANT_SPECIFIC_URL) : connection) {
+            createBaseTableAndViews(connection, baseTableFullName, viewConnection, SCHEMA2, SCHEMA3, SCHEMA4);
+            // Delete the CHILD_TABLE links to grand child views
+            executeDeleteQuery(connection, deleteChildLinks, SCHEMA2);
+            // Verify that grand grand child views are still in the system catalog table
+            verifyCountQuery(connection, countViewsQuery, SCHEMA4, grandGrandChildCount);
+            // Run the orphan view tool to clean up orphan views and links
+            runOrphanViewTool(true, false, true, false);
+            // Verify that the orphan views have been removed
+            verifyOrphanFileLineCounts(grandChildCount + grandGrandChildCount,
+                    0, 0, 0);
+            // Verify that all views and links records for grand and grand grand child views are removed
+            // from the system catalog table
+            verifyNoViewNoLinkInSystemCatalog(connection, SCHEMA3);
+            verifyNoViewNoLinkInSystemCatalog(connection, SCHEMA4);
+            // Verify the child links are also removed
+            verifyNoChildLink(connection, SCHEMA2);
+            verifyNoChildLink(connection, SCHEMA3);
+            deleteAllRows(connection, SCHEMA1, SCHEMA2, SCHEMA3, SCHEMA4);
+        }
+    }
+
+    @Test
+    public void testDeleteChildParentLinkRows() throws Exception {
+        String baseTableName = generateUniqueName();
+        String baseTableFullName = SchemaUtil.getTableName(SCHEMA1, baseTableName);
+        try (Connection connection = DriverManager.getConnection(getUrl());
+             Connection viewConnection =
+                     isMultiTenant ? DriverManager.getConnection(TENANT_SPECIFIC_URL) : connection) {
+            createBaseTableAndViews(connection, baseTableFullName, viewConnection, SCHEMA2, SCHEMA3, SCHEMA4);
+            // Delete the PARENT_TABLE links from  grand grand child views
+            executeDeleteQuery(connection, deleteParentLinks, SCHEMA4);
+            // Verify that grand grand child views are still in the system catalog table
+            verifyCountQuery(connection, countViewsQuery, SCHEMA4, grandGrandChildCount);
+            // Run the orphan view tool to clean up orphan views and links
+            runOrphanViewTool(true, false, true, false);
+            // Verify that orphan views and links have been removed
+            verifyOrphanFileLineCounts(grandGrandChildCount,
+                    0, 0, 0);
+            // Verify that all views and links records for grand grand child views are removed
+            // from the system catalog table
+            verifyNoViewNoLinkInSystemCatalog(connection, SCHEMA4);
+            // Verify the child links to grand grand child views are also removed
+            verifyNoChildLink(connection, SCHEMA3);
+            deleteAllRows(connection, SCHEMA1, SCHEMA2, SCHEMA3, SCHEMA4);
+        }
+    }
+
+    @Test
+    public void testDeletePhysicalTableLinks() throws Exception {
+        String baseTableName = generateUniqueName();
+        String baseTableFullName = SchemaUtil.getTableName(SCHEMA1, baseTableName);
+        try (Connection connection = DriverManager.getConnection(getUrl());
+             Connection viewConnection =
+                     isMultiTenant ? DriverManager.getConnection(TENANT_SPECIFIC_URL) : connection) {
+            createBaseTableAndViews(connection, baseTableFullName, viewConnection, SCHEMA2, SCHEMA3, SCHEMA3);
+            // Delete the physical table link rows from the system catalog
+            executeDeleteQuery(connection, deletePhysicalLinks, SCHEMA2);
+            // Verify that the views we have created are still in the system catalog table
+            verifyCountQuery(connection, countViewsQuery, SCHEMA2, childCount);
+            verifyCountQuery(connection, countViewsQuery, SCHEMA3, grandChildCount + grandGrandChildCount);
+            // Run the orphan view tool to remove orphan views
+            runOrphanViewTool(true, false, true, false);
+            // Verify that the orphan views have been removed
+            verifyLineCount(viewFileName, childCount + grandChildCount + grandGrandChildCount);
+            verifyNoViewNoLinkInSystemCatalog(connection, SCHEMA2);
+            verifyNoViewNoLinkInSystemCatalog(connection, SCHEMA3);
+            // Verify that there there is no link in the system child link table
+            verifyNoChildLink(connection, SCHEMA1);
+            verifyNoChildLink(connection, SCHEMA2);
+            verifyNoChildLink(connection, SCHEMA3);
+            deleteAllRows(connection, SCHEMA1, SCHEMA2, SCHEMA3, SCHEMA4);
+        }
+    }
+
+    public static String[] getArgValues(boolean clean, boolean identify, boolean outputPath, boolean inputPath) {
+        final List<String> args = Lists.newArrayList();
+        if (outputPath) {
+            args.add("-op");
+            args.add(filePath);
+        }
+        if (inputPath) {
+            args.add("-ip");
+            args.add(filePath);
+        }
+        if (clean) {
+            args.add("-c");
+        }
+        if (identify) {
+            args.add("-i");
+        }
+        args.add("-a");
+        args.add("0");
+        return args.toArray(new String[0]);
+    }
+
+    public static void runOrphanViewTool(boolean clean, boolean identify, boolean outputPath, boolean inputPath)
+            throws Exception {
+        OrphanViewTool orphanViewTool = new OrphanViewTool();
+        Configuration conf = new Configuration(getUtility().getConfiguration());
+        orphanViewTool.setConf(conf);
+        final String[] cmdArgs =
+                getArgValues(clean, identify, outputPath, inputPath);
+        int status = orphanViewTool.run(cmdArgs);
+        assertEquals(0, status);
+    }
+}
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/OrphanViewTool.java b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/OrphanViewTool.java
new file mode 100644
index 0000000..a8a30b6
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/OrphanViewTool.java
@@ -0,0 +1,879 @@
+/*
+ * 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;
+
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_FAMILY;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_NAME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.LINK_TYPE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_NAME;
+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.TABLE_TYPE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TENANT_ID;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_TYPE;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileReader;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+import org.apache.commons.cli.PosixParser;
+import org.apache.commons.lang.exception.ExceptionUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.mapreduce.util.ConnectionUtil;
+import org.apache.phoenix.parse.DropTableStatement;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.query.QueryServicesOptions;
+import org.apache.phoenix.schema.MetaDataClient;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTableType;
+import org.apache.phoenix.schema.TableNotFoundException;
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A tool to identify orphan views and links, and drop them
+ *
+ */
+public class OrphanViewTool extends Configured implements Tool {
+    private static final Logger LOG = LoggerFactory.getLogger(OrphanViewTool.class);
+    // Query all the views that are not "MAPPED" views
+    private static final String viewQuery = "SELECT " +
+            TENANT_ID + ", " +
+            TABLE_SCHEM + "," +
+            TABLE_NAME +
+            " FROM " + SYSTEM_CATALOG_NAME +
+            " WHERE "+ TABLE_TYPE + " = '" + PTableType.VIEW.getSerializedValue() +"' AND NOT " +
+            VIEW_TYPE + " = " + PTable.ViewType.MAPPED.getSerializedValue();
+    // Query all physical links
+    private static final String physicalLinkQuery = "SELECT " +
+            TENANT_ID + ", " +
+            TABLE_SCHEM + ", " +
+            TABLE_NAME + ", " +
+            COLUMN_NAME + " AS PHYSICAL_TABLE_TENANT_ID, " +
+            COLUMN_FAMILY + " AS PHYSICAL_TABLE_FULL_NAME " +
+            " FROM " + SYSTEM_CATALOG_NAME +
+            " WHERE "+ LINK_TYPE + " = " +
+            PTable.LinkType.PHYSICAL_TABLE.getSerializedValue();
+    // Query all child-parent links
+    private static final String childParentLinkQuery = "SELECT " +
+            TENANT_ID + ", " +
+            TABLE_SCHEM + ", " +
+            TABLE_NAME + ", " +
+            COLUMN_NAME + " AS PARENT_VIEW_TENANT_ID, " +
+            COLUMN_FAMILY + " AS PARENT_VIEW_FULL_NAME " +
+            " FROM " + SYSTEM_CATALOG_NAME +
+            " WHERE "+ LINK_TYPE + " = " +
+            PTable.LinkType.PARENT_TABLE.getSerializedValue();
+    // Query all parent-child links
+    private static final String parentChildLinkQuery = "SELECT " +
+            TENANT_ID + ", " +
+            TABLE_SCHEM + ", " +
+            TABLE_NAME + ", " +
+            COLUMN_NAME + " AS CHILD_VIEW_TENANT_ID, " +
+            COLUMN_FAMILY + " AS CHILD_VIEW_FULL_NAME " +
+            " FROM " + SYSTEM_CHILD_LINK_NAME +
+            " WHERE "+ LINK_TYPE + " = " +
+            PTable.LinkType.CHILD_TABLE.getSerializedValue();
+
+    // Query all the tables that can be a base table
+    private static final String candidateBaseTableQuery = "SELECT " +
+            TENANT_ID + ", " +
+            TABLE_SCHEM + ", " +
+            TABLE_NAME +
+            " FROM " + SYSTEM_CATALOG_NAME +
+            " WHERE "+ " NOT " + TABLE_TYPE + " = '" + PTableType.VIEW.getSerializedValue() + "'";
+    // The path of the directory of the output files
+    private String outputPath;
+    // The path of the directory of the input files
+    private String inputPath;
+    // The flag to indicate if the orphan views and links will be deleted
+    private boolean clean = false;
+    // The maximum level found in a view tree
+    private int maxViewLevel = 0;
+    // The age of a view
+    private static final long defaultAgeMs = 24*60*60*1000; // 1 day
+    private long ageMs = 0;
+
+    // A separate file is maintained to list orphan views, and each type of orphan links
+    public static final byte VIEW = 0;
+    public static final byte PHYSICAL_TABLE_LINK = 1;
+    public static final byte PARENT_TABLE_LINK = 2;
+    public static final byte CHILD_TABLE_LINK = 3;
+    public static final byte ORPHAN_TYPE_COUNT = 4;
+
+    BufferedWriter writer[] = new BufferedWriter[ORPHAN_TYPE_COUNT];
+    BufferedReader reader[] = new BufferedReader[ORPHAN_TYPE_COUNT];
+
+    // The set of orphan views
+    HashMap<Key, View> orphanViewSet = new HashMap<>();
+    // The array list of set of views such that the views in the first set are the first level views and the views
+    // in the second set is the second level views, and so on
+    List<HashMap<Key, View>> viewSetArray = new ArrayList<HashMap<Key, View>>();
+    // The set of base tables
+    HashMap<Key, Base> baseSet = new HashMap<>();
+    // The set of orphan links. These links can be CHILD_TABLE, PARENT_TABLE, or PHYSICAL_TABLE links
+    HashSet<Link> orphanLinkSet = new HashSet<>();
+
+    public static final String fileName[] = {"OrphanView.txt", "OrphanPhysicalTableLink.txt", "OrphanParentTableLink.txt", "OrphanChildTableLink.txt"};
+    private static final Option OUTPUT_PATH_OPTION = new Option("op", "output-path", true,
+            "Output path where the files listing orphan views and links are written");
+    private static final Option INPUT_PATH_OPTION = new Option("ip", "input-path", true,
+            "Input path where the files listing orphan views and links are read");
+    private static final Option CLEAN_ORPHAN_VIEWS_OPTION = new Option("c", "clean", false,
+            "If specified, cleans orphan views and links");
+    private static final Option IDENTIFY_ORPHAN_VIEWS_OPTION = new Option("i", "identify", false,
+            "If specified, identifies orphan views and links");
+    private static final Option AGE_OPTION = new Option("a", "age", true,
+            "The minimum age (in milliseconds) for the views (default value is " + Long.toString(defaultAgeMs) + ", i.e. 1 day)");
+    private static final Option HELP_OPTION = new Option("h", "help", false, "Help");
+
+    private Options getOptions() {
+        final Options options = new Options();
+        options.addOption(OUTPUT_PATH_OPTION);
+        options.addOption(INPUT_PATH_OPTION);
+        options.addOption(CLEAN_ORPHAN_VIEWS_OPTION);
+        options.addOption(IDENTIFY_ORPHAN_VIEWS_OPTION);
+        options.addOption(AGE_OPTION);
+        options.addOption(HELP_OPTION);
+        return options;
+    }
+
+    /**
+     * Parses the commandline arguments, throws IllegalStateException if mandatory arguments are
+     * missing.
+     * @param args supplied command line arguments
+     */
+    private void parseOptions(String[] args) throws Exception {
+
+        final Options options = getOptions();
+
+        CommandLineParser parser = new PosixParser();
+        CommandLine cmdLine = null;
+        try {
+            cmdLine = parser.parse(options, args);
+        } catch (ParseException e) {
+            printHelpAndExit("Error parsing command line options: " + e.getMessage(), options);
+        }
+        if (cmdLine.hasOption(HELP_OPTION.getOpt())) {
+            printHelpAndExit(options, 0);
+        }
+        if (cmdLine.hasOption(OUTPUT_PATH_OPTION.getOpt()) && cmdLine.hasOption(INPUT_PATH_OPTION.getOpt())) {
+            throw new IllegalStateException("Specify either " + OUTPUT_PATH_OPTION.getLongOpt() + " or "
+                    + INPUT_PATH_OPTION.getOpt());
+        }
+        if (cmdLine.hasOption(INPUT_PATH_OPTION.getOpt()) && !cmdLine.hasOption(CLEAN_ORPHAN_VIEWS_OPTION.getOpt())) {
+            throw new IllegalStateException(INPUT_PATH_OPTION.getLongOpt() + " is only used with "
+                    + IDENTIFY_ORPHAN_VIEWS_OPTION.getOpt());
+        }
+        if (cmdLine.hasOption(IDENTIFY_ORPHAN_VIEWS_OPTION.getOpt()) && cmdLine.hasOption(CLEAN_ORPHAN_VIEWS_OPTION.getOpt())) {
+            throw new IllegalStateException("Specify either " + IDENTIFY_ORPHAN_VIEWS_OPTION.getLongOpt() + " or "
+                    + IDENTIFY_ORPHAN_VIEWS_OPTION.getOpt());
+        }
+        if (cmdLine.hasOption(OUTPUT_PATH_OPTION.getOpt()) && (!cmdLine.hasOption(IDENTIFY_ORPHAN_VIEWS_OPTION.getOpt()) &&
+                !cmdLine.hasOption(CLEAN_ORPHAN_VIEWS_OPTION.getOpt()))) {
+            throw new IllegalStateException(OUTPUT_PATH_OPTION.getLongOpt() + " requires either " +
+                    IDENTIFY_ORPHAN_VIEWS_OPTION.getOpt() + " or " + CLEAN_ORPHAN_VIEWS_OPTION.getOpt());
+        }
+        if (cmdLine.hasOption(CLEAN_ORPHAN_VIEWS_OPTION.getOpt())) {
+            clean = true;
+        }
+        else if (!cmdLine.hasOption(IDENTIFY_ORPHAN_VIEWS_OPTION.getOpt())) {
+            throw new IllegalStateException("Specify either " +
+                    IDENTIFY_ORPHAN_VIEWS_OPTION.getOpt() + " or " + CLEAN_ORPHAN_VIEWS_OPTION.getOpt());
+        }
+        if (cmdLine.hasOption(AGE_OPTION.getOpt())) {
+            ageMs = Long.valueOf(cmdLine.getOptionValue(AGE_OPTION.getOpt()));
+        }
+
+        outputPath = cmdLine.getOptionValue(OUTPUT_PATH_OPTION.getOpt());
+        inputPath = cmdLine.getOptionValue(INPUT_PATH_OPTION.getOpt());
+    }
+
+    private void printHelpAndExit(String errorMessage, Options options) {
+        System.err.println(errorMessage);
+        printHelpAndExit(options, 1);
+    }
+
+    private void printHelpAndExit(Options options, int exitCode) {
+        HelpFormatter formatter = new HelpFormatter();
+        formatter.printHelp("help", options);
+        System.exit(exitCode);
+    }
+
+    /**
+     * The key that uniquely identifies a table (i.e., a base table or table view)
+     */
+    private static class Key {
+        private String serializedValue;
+
+        public Key (String tenantId, String schemaName, String tableName) throws IllegalArgumentException {
+            if (tableName == null) {
+                throw new IllegalArgumentException();
+            }
+            serializedValue = (tenantId != null ? tenantId + "," : ",") +
+                    (schemaName != null ? schemaName + "," : ",") +
+                    tableName;
+        }
+
+        public Key (String tenantId, String fullTableName) {
+            String[] columns = fullTableName.split("\\.");
+            String schemaName;
+            String tableName;
+            if (columns.length == 1) {
+                schemaName = null;
+                tableName = fullTableName;
+            } else {
+                schemaName = columns[0];
+                tableName = columns[1];
+            }
+            if (tableName == null || tableName.compareTo("") == 0) {
+                throw new IllegalArgumentException();
+            }
+            serializedValue = (tenantId != null ? tenantId + "," : ",") +
+                    (schemaName != null ? schemaName + "," : ",") +
+                    tableName;
+        }
+
+        public Key (String serializedKey) {
+            serializedValue = serializedKey;
+            if (this.getTableName() == null || this.getTableName().compareTo("") == 0) {
+                throw new IllegalArgumentException();
+            }
+        }
+
+        public String getTenantId() {
+            String[] columns = serializedValue.split(",");
+            return columns[0].compareTo("") == 0 ? null : columns[0];
+        }
+
+        public String getSchemaName() {
+            String[] columns = serializedValue.split(",");
+            return columns[1].compareTo("") == 0 ? null : columns[1];
+        }
+
+        public String getTableName() {
+            String[] columns = serializedValue.split(",");
+            return columns[2];
+        }
+
+        public String getSerializedValue() {
+            return serializedValue;
+        }
+        @Override
+        public int hashCode() {
+            return Objects.hash(getSerializedValue());
+        }
+
+        @Override
+        public boolean equals(Object obj) {
+            if (this == obj)
+                return true;
+            if (getClass() != obj.getClass())
+                return false;
+            Key other = (Key) obj;
+            if (this.getSerializedValue().compareTo(other.getSerializedValue()) != 0)
+                return false;
+            return true;
+        }
+    }
+
+    /**
+     * An abstract class to represent a table that can be a base table or table view
+     */
+    private static abstract class Table {
+        protected Key key;
+        protected List<Key> childViews;
+
+        public void addChild(Key childView) {
+            if (childViews == null) {
+                childViews = new LinkedList<>();
+            }
+            childViews.add(childView);
+        }
+
+        public boolean isParent() {
+            if (childViews == null || childViews.isEmpty()) {
+                return false;
+            }
+            return true;
+        }
+    }
+
+    /**
+     * A class to represents a base table
+     */
+    private static class Base extends Table {
+        public Base (Key key) {
+            this.key = key;
+        }
+    }
+
+    /**
+     * A class to represents a table view
+     */
+    private static class View extends Table {
+        Key parent;
+        Key base;
+
+        public View (Key key) {
+            this.key = key;
+        }
+
+        public void setParent(Key parent) {
+            this.parent = parent;
+        }
+
+        public void setBase(Key base) {
+            this.base = base;
+        }
+    }
+
+    private static class Link {
+        Key src;
+        Key dst;
+        PTable.LinkType type;
+
+        public Link(Key src, Key dst, PTable.LinkType type) {
+            this.src = src;
+            this.dst = dst;
+            this.type = type;
+        }
+
+        public String serialize() {
+            return src.getSerializedValue() + "," + dst.getSerializedValue() + "," + type.toString();
+        }
+
+        @Override
+        public int hashCode() {
+            return Objects.hash(serialize());
+        }
+    }
+
+    private void gracefullyDropView(PhoenixConnection phoenixConnection, Configuration configuration,
+                          Key key) throws Exception {
+        PhoenixConnection tenantConnection;
+        if (key.getTenantId() != null) {
+            Properties tenantProps = new Properties();
+            tenantProps.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, key.getTenantId());
+            tenantConnection = ConnectionUtil.getInputConnection(configuration, tenantProps).
+                    unwrap(PhoenixConnection.class);
+        } else {
+            tenantConnection = phoenixConnection;
+        }
+
+        MetaDataClient client = new MetaDataClient(tenantConnection);
+        org.apache.phoenix.parse.TableName pTableName = org.apache.phoenix.parse.TableName
+                .create(key.getSchemaName(), key.getTableName());
+        try {
+            client.dropTable(
+                    new DropTableStatement(pTableName, PTableType.VIEW, false, true, true));
+        }
+        catch (TableNotFoundException e) {
+            LOG.info("Ignoring view " + pTableName + " as it has already been dropped");
+        }
+    }
+
+    private void removeLink(PhoenixConnection phoenixConnection, Key src, Key dst, PTable.LinkType linkType) throws Exception {
+        String deleteQuery = "DELETE FROM " +
+                ((linkType == PTable.LinkType.PHYSICAL_TABLE || linkType == PTable.LinkType.PARENT_TABLE) ? SYSTEM_CATALOG_NAME : SYSTEM_CHILD_LINK_NAME) +
+                " WHERE " + TENANT_ID + (src.getTenantId() == null ? " IS NULL" : " = '" + src.getTenantId() + "'") + " AND " +
+                TABLE_SCHEM + (src.getSchemaName() == null ? " IS NULL " : " = '" + src.getSchemaName() + "'") + " AND " +
+                TABLE_NAME + " = '" + src.getTableName() + "' AND " +
+                COLUMN_NAME + (dst.getTenantId() == null ? " IS NULL" : " = '" + dst.getTenantId() + "'") + " AND " +
+                COLUMN_FAMILY + " = '" + (dst.getSchemaName() == null ? dst.getTableName() : dst.getSchemaName() + "." +
+                dst.getTableName()) + "'";
+        phoenixConnection.createStatement().execute(deleteQuery);
+        phoenixConnection.commit();
+    }
+
+    private byte getLinkType(PTable.LinkType linkType) {
+        byte type;
+        if (linkType == PTable.LinkType.PHYSICAL_TABLE) {
+            type = PHYSICAL_TABLE_LINK;
+        }
+        else if (linkType == PTable.LinkType.PARENT_TABLE) {
+            type = PARENT_TABLE_LINK;
+        } else if (linkType == PTable.LinkType.CHILD_TABLE) {
+            type = CHILD_TABLE_LINK;
+        }
+        else {
+            throw new AssertionError("Unknown Link Type");
+        }
+        return type;
+    }
+
+    private PTable.LinkType getLinkType(byte linkType) {
+        PTable.LinkType type;
+        if (linkType == PHYSICAL_TABLE_LINK) {
+            type = PTable.LinkType.PHYSICAL_TABLE;
+        }
+        else if (linkType == PARENT_TABLE_LINK) {
+            type = PTable.LinkType.PARENT_TABLE;
+        } else if (linkType == CHILD_TABLE_LINK) {
+            type = PTable.LinkType.CHILD_TABLE;
+        }
+        else {
+            throw new AssertionError("Unknown Link Type");
+        }
+        return type;
+    }
+
+    private void removeOrLogOrphanLinks(PhoenixConnection phoenixConnection) {
+        for (Link link : orphanLinkSet) {
+            try {
+                byte linkType = getLinkType(link.type);
+                if (outputPath != null) {
+                    writer[linkType].write(link.src.getSerializedValue() + "-->" + link.dst.getSerializedValue());
+                    writer[linkType].newLine();
+                }
+                else if (!clean){
+                    System.out.println(link.src.getSerializedValue() + "-(" + link.type + ")->" + link.dst.getSerializedValue());
+                }
+                if (clean) {
+                    removeLink(phoenixConnection, link.src, link.dst, link.type);
+                }
+            } catch (Exception e) {
+                // ignore
+            }
+        }
+    }
+    private void forcefullyDropView(PhoenixConnection phoenixConnection,
+                                    Key key) throws Exception {
+        String deleteRowsFromCatalog = "DELETE FROM " + SYSTEM_CATALOG_NAME +
+                " WHERE " + TENANT_ID + (key.getTenantId() == null ? " IS NULL" : " = '" + key.getTenantId() + "'") + " AND " +
+                TABLE_SCHEM + (key.getSchemaName() == null ? " IS NULL " : " = '" + key.getSchemaName() + "'") + " AND " +
+                TABLE_NAME + " = '" + key.getTableName() + "'";
+        String deleteRowsFromChildLink = "DELETE FROM " + SYSTEM_CHILD_LINK_NAME +
+                " WHERE " + COLUMN_NAME + (key.getTenantId() == null ? " IS NULL" : " = '" + key.getTenantId() + "'") + " AND " +
+                COLUMN_FAMILY + " = '" + (key.getSchemaName() == null ? key.getTableName() : key.getSchemaName() + "." + key.getTableName()) + "'";
+        try {
+            phoenixConnection.createStatement().execute(deleteRowsFromCatalog);
+            phoenixConnection.createStatement().execute(deleteRowsFromChildLink);
+            phoenixConnection.commit();
+        } catch (SQLException e) {
+            throw new IOException(e);
+        }
+    }
+
+    private void dropOrLogOrphanViews(PhoenixConnection phoenixConnection, Configuration configuration,
+                          Key key) throws Exception {
+        if (outputPath != null) {
+            writer[VIEW].write(key.getSerializedValue());
+            writer[VIEW].newLine();
+        }
+        else if (!clean) {
+            System.out.println(key.getSerializedValue());
+            return;
+        }
+        if (!clean) {
+            return;
+        }
+        gracefullyDropView(phoenixConnection, configuration, key);
+    }
+
+    /**
+     * Go through all the views in the system catalog table and add them to orphanViewSet
+     * @param phoenixConnection
+     * @throws Exception
+     */
+    private void populateOrphanViewSet(PhoenixConnection phoenixConnection)
+            throws Exception {
+        ResultSet viewRS = phoenixConnection.createStatement().executeQuery(viewQuery);
+        while (viewRS.next()) {
+            String tenantId = viewRS.getString(1);
+            String schemaName = viewRS.getString(2);
+            String tableName = viewRS.getString(3);
+            Key key = new Key(tenantId, schemaName, tableName);
+            View view = new View(key);
+            orphanViewSet.put(key, view);
+        }
+    }
+
+    /**
+     * Go through all the tables in the system catalog table and update baseSet
+     * @param phoenixConnection
+     * @throws Exception
+     */
+    private void populateBaseSet(PhoenixConnection phoenixConnection)
+            throws Exception {
+        ResultSet baseTableRS = phoenixConnection.createStatement().executeQuery(candidateBaseTableQuery);
+        while (baseTableRS.next()) {
+            String tenantId = baseTableRS.getString(1);
+            String schemaName = baseTableRS.getString(2);
+            String tableName = baseTableRS.getString(3);
+            Key key = new Key(tenantId, schemaName, tableName);
+            Base base = new Base(key);
+            baseSet.put(key, base);
+        }
+    }
+
+    /**
+     * Go through all the physical links in the system catalog table and update the base table info of the
+     * view objects in orphanViewSet. If the base or view object does not exist for a given link, then add the link
+     * to orphanLinkSet
+     * @param phoenixConnection
+     * @throws Exception
+     */
+    private void processPhysicalLinks(PhoenixConnection phoenixConnection)
+            throws Exception {
+        ResultSet physicalLinkRS = phoenixConnection.createStatement().executeQuery(physicalLinkQuery);
+        while (physicalLinkRS.next()) {
+            String tenantId = physicalLinkRS.getString(1);
+            String schemaName = physicalLinkRS.getString(2);
+            String tableName = physicalLinkRS.getString(3);
+            Key viewKey = new Key(tenantId, schemaName, tableName);
+            View view = orphanViewSet.get(viewKey);
+
+            String baseTenantId = physicalLinkRS.getString(4);
+            String baseFullTableName = physicalLinkRS.getString(5);
+            Key baseKey = new Key(baseTenantId, baseFullTableName);
+            Base base = baseSet.get(baseKey);
+
+            if (view == null || base == null) {
+                orphanLinkSet.add(new Link(viewKey, baseKey, PTable.LinkType.PHYSICAL_TABLE));
+            }
+            else {
+                view.setBase(baseKey);
+            }
+        }
+    }
+
+    /**
+     * Go through all the child-parent links and update the parent field of the view objects of orphanViewSet.
+     * Check if the child does not exist add the link to orphanLinkSet.
+     * @param phoenixConnection
+     * @throws Exception
+     */
+    private void processChildParentLinks(PhoenixConnection phoenixConnection)
+            throws Exception {
+        ResultSet childParentLinkRS = phoenixConnection.createStatement().executeQuery(childParentLinkQuery);
+        while (childParentLinkRS.next()) {
+            String childTenantId = childParentLinkRS.getString(1);
+            String childSchemaName = childParentLinkRS.getString(2);
+            String childTableName = childParentLinkRS.getString(3);
+            Key childKey = new Key(childTenantId, childSchemaName, childTableName);
+            View childView = orphanViewSet.get(childKey);
+
+            String parentTenantId = childParentLinkRS.getString(4);
+            String parentFullTableName = childParentLinkRS.getString(5);
+            Key parentKey = new Key(parentTenantId, parentFullTableName);
+            View parentView = orphanViewSet.get(parentKey);
+
+            // Check if parentTenantId is not set but it should have been the same as the childTenantId. Is this a bug?
+            if (childView != null && parentView == null && parentTenantId == null && childTenantId != null) {
+                Key anotherParentKey = new Key(childTenantId, parentFullTableName);
+                parentView = orphanViewSet.get(anotherParentKey);
+                if (parentView != null) {
+                    parentKey = anotherParentKey;
+                }
+            }
+
+            if (childView == null || parentView == null) {
+                orphanLinkSet.add(new Link(childKey, parentKey, PTable.LinkType.PARENT_TABLE));
+            }
+            else {
+                childView.setParent(parentKey);
+            }
+        }
+    }
+
+    /**
+     * Go through all the parent-child links and update the parent field of the
+     * child view objects of orphanViewSet and the child links of the parent objects (which can be a view from
+     * orphanViewSet or a base table from baseSet. Check if the child or parent does not exist, and if so, add the link
+     * to orphanLinkSet.
+     * @param phoenixConnection
+     * @throws Exception
+     */
+    private void processParentChildLinks(PhoenixConnection phoenixConnection)
+            throws Exception {
+        ResultSet parentChildLinkRS = phoenixConnection.createStatement().executeQuery(parentChildLinkQuery);
+        while (parentChildLinkRS.next()) {
+            String tenantId = parentChildLinkRS.getString(1);
+            String schemaName = parentChildLinkRS.getString(2);
+            String tableName = parentChildLinkRS.getString(3);
+            Key parentKey = new Key(tenantId, schemaName, tableName);
+            Base base = baseSet.get(parentKey);
+            View parentView = orphanViewSet.get(parentKey);
+
+            String childTenantId = parentChildLinkRS.getString(4);
+            String childFullTableName = parentChildLinkRS.getString(5);
+            Key childKey = new Key(childTenantId, childFullTableName);
+            View childView = orphanViewSet.get(childKey);
+
+            if (childView == null) {
+                // No child for this link
+                orphanLinkSet.add(new Link(parentKey, childKey, PTable.LinkType.CHILD_TABLE));
+            }
+            else if (base != null) {
+                base.addChild(childKey);
+            }
+            else if (parentView != null) {
+                parentView.addChild(childKey);
+            }
+            else {
+                // No parent for this link
+                orphanLinkSet.add(new Link(parentKey, childKey, PTable.LinkType.CHILD_TABLE));
+            }
+        }
+    }
+
+    private void removeBaseTablesWithNoChildViewFromBaseSet() {
+        Iterator<Map.Entry<Key, Base>> iterator = baseSet.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Key, Base> entry = iterator.next();
+            if (entry.getValue().childViews == null || entry.getValue().childViews.isEmpty()) {
+                iterator.remove();
+            }
+        }
+    }
+
+    /**
+     * Starting from the child views of the base tables from baseSet, visit views level by level and identify
+     * missing or broken links, and thereby identify orphan vies
+     */
+    private void visitViewsLevelByLevelAndIdentifyOrphanViews() {
+        if (baseSet.isEmpty())
+            return;
+        HashMap<Key, View> viewSet = new HashMap<>();
+        viewSetArray.add(0, viewSet);
+        // Remove the child views of the tables of baseSet from orphanViewSet and add them to viewSetArray[0]
+        // if these views have the correct physical link
+        for (Map.Entry<Key, Base> baseEntry : baseSet.entrySet()) {
+            for (Key child : baseEntry.getValue().childViews) {
+                View childView = orphanViewSet.get(child);
+                if (childView != null &&
+                        childView.base != null && childView.base.equals(baseEntry.getKey())) {
+                    orphanViewSet.remove(child);
+                    viewSet.put(child, childView);
+                }
+            }
+        }
+        HashMap<Key, View> parentViewSet = viewSet;
+        // Remove the child views of  viewSetArray[N] from orphanViewSet and add them to viewSetArray[N+1]
+        // if these view have the correct physical link and parent link
+        maxViewLevel = 1;
+        for (int i = 1; !parentViewSet.isEmpty(); i++) {
+            HashMap<Key, View> childViewSet = new HashMap<>();
+            viewSetArray.add(i, childViewSet);
+            for (Map.Entry<Key, View> viewEntry : parentViewSet.entrySet()) {
+                View parentView = viewEntry.getValue();
+                Key parentKey = viewEntry.getKey();
+                if (parentView.isParent()) {
+                    for (Key child : parentView.childViews) {
+                        View childView = orphanViewSet.get(child);
+                        if (childView != null &&
+                                childView.parent != null && childView.parent.equals(parentKey) &&
+                                childView.base != null && childView.base.equals(parentView.base)) {
+                            orphanViewSet.remove(child);
+                            childViewSet.put(child, childView);
+                        }
+                    }
+                }
+            }
+            parentViewSet = childViewSet;
+            maxViewLevel += 1;
+        }
+    }
+
+    private void identifyOrphanViews(PhoenixConnection phoenixConnection)
+            throws Exception {
+        if (inputPath != null) {
+            readOrphanViews();
+            return;
+        }
+        // Go through the views and add them to orphanViewSet
+        populateOrphanViewSet(phoenixConnection);
+        // Go through the candidate base tables and add them to baseSet
+        populateBaseSet(phoenixConnection);
+        // Go through physical links and update the views of orphanLinkSet
+        processPhysicalLinks(phoenixConnection);
+        // Go through the parent-child links and update the views of orphanViewSet and the tables of baseSet
+        processParentChildLinks(phoenixConnection);
+        // Go through index-view links and update the views of orphanLinkSet
+        processChildParentLinks(phoenixConnection);
+
+        if (baseSet == null)
+            return;
+        // Remove the base tables with no child from baseSet
+        removeBaseTablesWithNoChildViewFromBaseSet();
+        // Starting from the child views of the base tables, visit views level by level and identify
+        // missing or broken links and thereby identify orphan vies
+        visitViewsLevelByLevelAndIdentifyOrphanViews();
+    }
+
+    private void createSnapshot(PhoenixConnection phoenixConnection, long scn)
+        throws Exception {
+        phoenixConnection.getQueryServices().getAdmin().snapshot("OrphanViewTool." + Long.toString(scn),
+                TableName.valueOf(SYSTEM_CATALOG_NAME));
+        phoenixConnection.getQueryServices().getAdmin().snapshot("OrphanViewTool." + Long.toString(scn+1),
+                TableName.valueOf(SYSTEM_CHILD_LINK_NAME));
+    }
+
+    private void readOrphanViews() throws Exception {
+        String aLine;
+        reader[VIEW] = new BufferedReader(new FileReader(inputPath + fileName[VIEW]));
+        while ((aLine = reader[VIEW].readLine()) != null) {
+            Key key = new Key(aLine);
+            orphanViewSet.put(key, new View(key));
+        }
+    }
+
+    private void readAndRemoveOrphanLinks(PhoenixConnection phoenixConnection) throws Exception{
+        String aLine;
+        for (byte i = VIEW+1; i < ORPHAN_TYPE_COUNT; i++) {
+            reader[i] = new BufferedReader(new FileReader(inputPath + fileName[i]));
+            while ((aLine = reader[i].readLine()) != null) {
+                String ends[] = aLine.split("-->");
+                removeLink(phoenixConnection, new Key(ends[0]), new Key(ends[1]), getLinkType(i));
+            }
+        }
+    }
+
+    private void closeConnectionAndFiles(Connection connection) throws IOException {
+        try {
+            if (connection != null) {
+                connection.close();
+            }
+        } catch (SQLException sqle) {
+            LOG.error("Failed to close connection ", sqle.getMessage());
+            throw new RuntimeException("Failed to close connection");
+        }
+        for (byte i = VIEW; i < ORPHAN_TYPE_COUNT; i++) {
+            if (writer[i] != null) {
+                writer[i].close();
+            }
+            if (reader[i] != null) {
+                reader[i].close();
+            }
+        }
+    }
+
+    /**
+     * Examples for input arguments:
+     * -c : cleans orphan views
+     * -c -op /tmp/ : cleans orphan views and links, and logs their names to the files named Orphan*.txt in /tmp/
+     * -i : identifies orphan views and links, and prints their names on the console
+     * -i -op /tmp/ : identifies orphan views and links, and logs the name of their names to files named Orphan*.txt in /tmp/
+     * -c -ip /tmp/ : cleans the views listed in files at /tmp/
+     */
+    @Override
+    public int run(String[] args) throws Exception {
+        Connection connection = null;
+        try {
+            final Configuration configuration = HBaseConfiguration.addHbaseResources(getConf());
+
+            try {
+                parseOptions(args);
+            } catch (IllegalStateException e) {
+                printHelpAndExit(e.getMessage(), getOptions());
+            }
+
+            Properties props = new Properties();
+            long scn = System.currentTimeMillis() - ageMs;
+            props.setProperty("CurrentSCN", Long.toString(scn));
+            connection = ConnectionUtil.getInputConnection(configuration);
+            PhoenixConnection phoenixConnection = connection.unwrap(PhoenixConnection.class);
+
+            if (clean) {
+                // Take a snapshot of system tables to be modified
+                createSnapshot(phoenixConnection, scn);
+            }
+            if (outputPath != null) {
+                // Create files to log orphan views and links
+                for (int i = VIEW; i < ORPHAN_TYPE_COUNT; i++) {
+                    File file = new File(outputPath + fileName[i]);
+                    if (file.exists()) {
+                        file.delete();
+                    }
+                    file.createNewFile();
+                    writer[i] = new BufferedWriter(new FileWriter(file));
+                }
+            }
+            identifyOrphanViews(phoenixConnection);
+            for (Map.Entry<Key, View> entry : orphanViewSet.entrySet()) {
+                try {
+                    dropOrLogOrphanViews(phoenixConnection, configuration, entry.getKey());
+                } catch (Exception e) {
+                    // Ignore
+                }
+            };
+            if (clean) {
+                // Wait for the view drop tasks in the SYSTEM.TASK table to be processed
+                long timeInterval = configuration.getLong(QueryServices.TASK_HANDLING_INTERVAL_MS_ATTRIB,
+                        QueryServicesOptions.DEFAULT_TASK_HANDLING_INTERVAL_MS);
+                Thread.sleep(maxViewLevel * timeInterval);
+                // Clean up any remaining orphan view records from system tables
+                for (Map.Entry<Key, View> entry : orphanViewSet.entrySet()) {
+                    try {
+                        forcefullyDropView(phoenixConnection, entry.getKey());
+                    } catch (Exception e) {
+                        // Ignore
+                    }
+                };
+            }
+            if (inputPath == null) {
+                removeOrLogOrphanLinks(phoenixConnection);
+            }
+            else {
+                readAndRemoveOrphanLinks(phoenixConnection);
+            }
+            return 0;
+        } catch (Exception ex) {
+            LOG.error("Orphan View Tool : An exception occurred " + ExceptionUtils.getMessage(ex) + " at:\n" +
+                    ExceptionUtils.getStackTrace(ex));
+            return -1;
+        } finally {
+            closeConnectionAndFiles(connection);
+        }
+    }
+
+    public static void main(final String[] args) throws Exception {
+        int result = ToolRunner.run(new OrphanViewTool(), args);
+        System.exit(result);
+    }
+}


[phoenix] 08/18: PHOENIX-4983: Added missing apache license header.

Posted by pb...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

pboado pushed a commit to branch 4.x-cdh5.16
in repository https://gitbox.apache.org/repos/asf/phoenix.git

commit f530f94659bcb337c8adce997ac7696431c719e2
Author: s.kadam <s....@salesforce.com>
AuthorDate: Fri Dec 14 16:04:29 2018 +0000

    PHOENIX-4983: Added missing apache license header.
---
 .../org/apache/phoenix/end2end/UpsertWithSCNIT.java     | 17 +++++++++++++++++
 1 file changed, 17 insertions(+)

diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertWithSCNIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertWithSCNIT.java
index 6f231ff..40bb883 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertWithSCNIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertWithSCNIT.java
@@ -1,3 +1,20 @@
+/*
+ * 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 org.apache.phoenix.exception.SQLExceptionCode;


[phoenix] 13/18: PHOENIX-5074; fix compilation failure.

Posted by pb...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

pboado pushed a commit to branch 4.x-cdh5.16
in repository https://gitbox.apache.org/repos/asf/phoenix.git

commit 82172a167de8adb709374b03c9a43bc1dc494e74
Author: Lars Hofhansl <la...@apache.org>
AuthorDate: Tue Dec 25 10:21:35 2018 +0000

    PHOENIX-5074; fix compilation failure.
---
 .../src/it/java/org/apache/phoenix/end2end/DropTableWithViewsIT.java  | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DropTableWithViewsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DropTableWithViewsIT.java
index a4cd354..6aaf703 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DropTableWithViewsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DropTableWithViewsIT.java
@@ -57,10 +57,10 @@ public class DropTableWithViewsIT extends SplitSystemCatalogIT {
     public static void doSetup() throws Exception {
         SplitSystemCatalogIT.doSetup();
         TaskRegionEnvironment =
-                getUtility()
+                (RegionCoprocessorEnvironment)getUtility()
                         .getRSForFirstRegionInTable(
                                 PhoenixDatabaseMetaData.SYSTEM_TASK_HBASE_TABLE_NAME)
-                        .getRegions(PhoenixDatabaseMetaData.SYSTEM_TASK_HBASE_TABLE_NAME)
+                        .getOnlineRegions(PhoenixDatabaseMetaData.SYSTEM_TASK_HBASE_TABLE_NAME)
                         .get(0).getCoprocessorHost()
                         .findCoprocessorEnvironment(TaskRegionObserver.class.getName());
     }


[phoenix] 02/18: ScanningResultIterator metric RowsScanned not set. PHOENIX-5051

Posted by pb...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

pboado pushed a commit to branch 4.x-cdh5.16
in repository https://gitbox.apache.org/repos/asf/phoenix.git

commit c360f87cf14c2540137798480cd3d70a933ebbbf
Author: chfeng <ch...@tencent.com>
AuthorDate: Wed Dec 5 02:40:29 2018 +0000

    ScanningResultIterator metric RowsScanned not set. PHOENIX-5051
---
 .../main/java/org/apache/phoenix/iterate/ScanningResultIterator.java    | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/ScanningResultIterator.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/ScanningResultIterator.java
index f02e9d3..893eaa2 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/iterate/ScanningResultIterator.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/ScanningResultIterator.java
@@ -117,7 +117,7 @@ public class ScanningResultIterator implements ResultIterator {
                     scanMetricsMap.get(RPC_RETRIES_METRIC_NAME));
             changeMetric(scanMetricsHolder.getCountOfRemoteRPCRetries(),
                     scanMetricsMap.get(REMOTE_RPC_RETRIES_METRIC_NAME));
-            changeMetric(scanMetricsHolder.getCountOfRowsFiltered(),
+            changeMetric(scanMetricsHolder.getCountOfRowsScanned(),
                     scanMetricsMap.get(COUNT_OF_ROWS_SCANNED_KEY_METRIC_NAME));
             changeMetric(scanMetricsHolder.getCountOfRowsFiltered(),
                     scanMetricsMap.get(COUNT_OF_ROWS_FILTERED_KEY_METRIC_NAME));


[phoenix] 06/18: PHOENIX-4983: Allow using a connection with a SCN set to write data to tables EXCEPT transactional tables or mutable tables with indexes or tables with ROW_TIMESTAMP column.

Posted by pb...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

pboado pushed a commit to branch 4.x-cdh5.16
in repository https://gitbox.apache.org/repos/asf/phoenix.git

commit 4de622ab30d3f5aeb755ffaf786ec9ec4cdd3ba1
Author: s.kadam <s....@salesforce.com>
AuthorDate: Mon Dec 10 22:40:17 2018 +0000

    PHOENIX-4983: Allow using a connection with a SCN set to write data to tables EXCEPT transactional tables or mutable tables with indexes or tables with ROW_TIMESTAMP column.
---
 .../apache/phoenix/end2end/UpsertWithSCNIT.java    | 139 +++++++++++++++++++++
 .../org/apache/phoenix/compile/UpsertCompiler.java |  23 +++-
 .../apache/phoenix/exception/SQLExceptionCode.java |  13 +-
 .../org/apache/phoenix/jdbc/PhoenixConnection.java |   2 +-
 4 files changed, 172 insertions(+), 5 deletions(-)

diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertWithSCNIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertWithSCNIT.java
new file mode 100644
index 0000000..6f231ff
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertWithSCNIT.java
@@ -0,0 +1,139 @@
+package org.apache.phoenix.end2end;
+
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.sql.Connection;
+import java.sql.Date;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.Properties;
+
+import static org.hamcrest.CoreMatchers.containsString;
+import static org.hamcrest.core.Is.is;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+
+public class UpsertWithSCNIT extends ParallelStatsDisabledIT {
+
+    @Rule
+    public final ExpectedException exception = ExpectedException.none();
+    Properties props = null;
+    PreparedStatement prep = null;
+    String tableName =null;
+
+    private void helpTestUpserWithSCNIT(boolean rowColumn, boolean txTable,
+                                        boolean mutable, boolean local, boolean global)
+            throws SQLException {
+
+        tableName = generateUniqueName();
+        String indx;
+        String createTable = "CREATE TABLE "+tableName+" ("
+                + (rowColumn ? "CREATED_DATE DATE NOT NULL, ":"")
+                + "METRIC_ID CHAR(15) NOT NULL,METRIC_VALUE VARCHAR(50) CONSTRAINT PK PRIMARY KEY("
+                + (rowColumn? "CREATED_DATE ROW_TIMESTAMP, ":"") + "METRIC_ID)) "
+                + (mutable? "IMMUTABLE_ROWS=false":"" )
+                + (txTable ? "TRANSACTION_PROVIDER='TEPHRA',TRANSACTIONAL=true":"");
+        props = new Properties();
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        conn.createStatement().execute(createTable);
+
+        if(local || global ){
+            indx = "CREATE "+ (local? "LOCAL " : "") + "INDEX "+tableName+"_idx ON " +
+                    ""+tableName+" (METRIC_VALUE)";
+            conn.createStatement().execute(indx);
+        }
+
+        props.setProperty("CurrentSCN", Long.toString(System.currentTimeMillis()));
+        conn = DriverManager.getConnection(getUrl(), props);
+        conn.setAutoCommit(true);
+        String upsert = "UPSERT INTO "+tableName+" (METRIC_ID, METRIC_VALUE) VALUES (?,?)";
+        prep = conn.prepareStatement(upsert);
+        prep.setString(1,"abc");
+        prep.setString(2,"This is the first comment!");
+    }
+
+    @Test // See https://issues.apache.org/jira/browse/PHOENIX-4983
+    public void testUpsertOnSCNSetTxnTable() throws SQLException {
+
+        helpTestUpserWithSCNIT(false, true, false, false, false);
+        exception.expect(SQLException.class);
+        exception.expectMessage(containsString(String.valueOf(
+                SQLExceptionCode
+                .CANNOT_SPECIFY_SCN_FOR_TXN_TABLE
+                .getErrorCode())));
+        prep.executeUpdate();
+    }
+
+    @Test
+    public void testUpsertOnSCNSetMutTableWithoutIdx() throws Exception {
+
+        helpTestUpserWithSCNIT(false, false, true, false, false);
+        prep.executeUpdate();
+        props = new Properties();
+        Connection conn = DriverManager.getConnection(getUrl(),props);
+        ResultSet rs = conn.createStatement().executeQuery("SELECT * FROM "+tableName);
+        assertTrue(rs.next());
+        assertEquals("abc", rs.getString(1));
+        assertEquals("This is the first comment!", rs.getString(2));
+        assertFalse(rs.next());
+    }
+
+    @Test
+    public void testUpsertOnSCNSetTable() throws Exception {
+
+        helpTestUpserWithSCNIT(false, false, false, false, false);
+        prep.executeUpdate();
+        props = new Properties();
+        Connection conn = DriverManager.getConnection(getUrl(),props);
+        ResultSet rs = conn.createStatement().executeQuery("SELECT * FROM "+tableName);
+        assertTrue(rs.next());
+        assertEquals("abc", rs.getString(1));
+        assertEquals("This is the first comment!", rs.getString(2));
+        assertFalse(rs.next());
+    }
+
+    @Test
+    public void testUpsertOnSCNSetMutTableWithLocalIdx() throws Exception {
+
+        helpTestUpserWithSCNIT(false, false, true, true, false);
+        exception.expect(SQLException.class);
+        exception.expectMessage(containsString(String.valueOf(
+                SQLExceptionCode
+                .CANNOT_UPSERT_WITH_SCN_FOR_MUTABLE_TABLE_WITH_INDEXES
+                .getErrorCode())));
+        prep.executeUpdate();
+    }
+    @Test
+    public void testUpsertOnSCNSetMutTableWithGlobalIdx() throws Exception {
+
+        helpTestUpserWithSCNIT(false, false, true, false, true);
+        exception.expect(SQLException.class);
+        exception.expectMessage(containsString(String.valueOf(
+                SQLExceptionCode
+                        .CANNOT_UPSERT_WITH_SCN_FOR_MUTABLE_TABLE_WITH_INDEXES
+                        .getErrorCode())));
+        prep.executeUpdate();
+
+    }
+    @Test
+    public void testUpsertOnSCNSetWithRowTSColumn() throws Exception {
+
+        helpTestUpserWithSCNIT(true, false, false, false, false);
+        exception.expect(SQLException.class);
+        exception.expectMessage(containsString(String.valueOf(
+                SQLExceptionCode
+                        .CANNOT_UPSERT_WITH_SCN_FOR_ROW_TIMSTAMP_COLUMN
+                        .getErrorCode())));
+        prep.executeUpdate();
+    }
+}
+
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 a770339..ec0c67c 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
@@ -366,6 +366,8 @@ public class UpsertCompiler {
         // Cannot update:
         // - read-only VIEW
         // - transactional table with a connection having an SCN
+        // - mutable table with indexes and SCN set
+        // - tables with ROW_TIMESTAMP columns
         if (table.getType() == PTableType.VIEW && table.getViewType().isReadOnly()) {
             throw new ReadOnlyTableException(schemaName,tableName);
         } else if (connection.isBuildingIndex() && table.getType() != PTableType.INDEX) {
@@ -374,8 +376,25 @@ public class UpsertCompiler {
             .setTableName(tableName)
             .build().buildException();
         } else if (table.isTransactional() && connection.getSCN() != null) {
-            throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_SPECIFY_SCN_FOR_TXN_TABLE).setSchemaName(schemaName)
-            .setTableName(tableName).build().buildException();
+            throw new SQLExceptionInfo.Builder(SQLExceptionCode
+                    .CANNOT_SPECIFY_SCN_FOR_TXN_TABLE)
+                    .setSchemaName(schemaName)
+                    .setTableName(tableName).build().buildException();
+        } else if (!table.isImmutableRows() && connection.getSCN() != null
+                && !table.getIndexes().isEmpty() && !connection.isRunningUpgrade()
+                && !connection.isBuildingIndex()) {
+            throw new SQLExceptionInfo
+                    .Builder(SQLExceptionCode
+                    .CANNOT_UPSERT_WITH_SCN_FOR_MUTABLE_TABLE_WITH_INDEXES)
+                    .setSchemaName(schemaName)
+                    .setTableName(tableName).build().buildException();
+        } else if(connection.getSCN() != null && !connection.isRunningUpgrade()
+                && !connection.isBuildingIndex() && table.getRowTimestampColPos() >= 0) {
+            throw new SQLExceptionInfo
+                    .Builder(SQLExceptionCode
+                    .CANNOT_UPSERT_WITH_SCN_FOR_ROW_TIMSTAMP_COLUMN)
+                    .setSchemaName(schemaName)
+                    .setTableName(tableName).build().buildException();
         }
         boolean isSalted = table.getBucketNum() != null;
         isTenantSpecific = table.isMultiTenant() && connection.getTenantId() != null;
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 5bffed5..cf0b9ae 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
@@ -289,7 +289,8 @@ public enum SQLExceptionCode {
     TRANSACTION_FAILED(1077, "44A08", "Transaction Failure "),
     CANNOT_CREATE_TXN_TABLE_IF_TXNS_DISABLED(1078, "44A09", "Cannot create a transactional table if transactions are disabled."),
     CANNOT_ALTER_TO_BE_TXN_IF_TXNS_DISABLED(1079, "44A10", "Cannot alter table to be transactional table if transactions are disabled."),
-    CANNOT_CREATE_TXN_TABLE_WITH_ROW_TIMESTAMP(1080, "44A11", "Cannot create a transactional table if transactions are disabled."),
+    CANNOT_CREATE_TXN_TABLE_WITH_ROW_TIMESTAMP(1080, "44A11", "Cannot create a transactional" +
+            " table with ROW_TIMESTAMP column."),
     CANNOT_ALTER_TO_BE_TXN_WITH_ROW_TIMESTAMP(1081, "44A12", "Cannot alter table to be transactional table if transactions are disabled."),
     TX_MUST_BE_ENABLED_TO_SET_TX_CONTEXT(1082, "44A13", "Cannot set transaction context if transactions are disabled."),
     TX_MUST_BE_ENABLED_TO_SET_AUTO_FLUSH(1083, "44A14", "Cannot set auto flush if transactions are disabled."),
@@ -461,7 +462,15 @@ public enum SQLExceptionCode {
     MAX_MUTATION_SIZE_EXCEEDED(729, "LIM01", "MutationState size is bigger than maximum allowed number of rows"),
     MAX_MUTATION_SIZE_BYTES_EXCEEDED(730, "LIM02", "MutationState size is bigger than maximum allowed number of bytes"), 
     INSUFFICIENT_MEMORY(999, "50M01", "Unable to allocate enough memory."),
-    HASH_JOIN_CACHE_NOT_FOUND(900, "HJ01", "Hash Join cache not found");
+    HASH_JOIN_CACHE_NOT_FOUND(900, "HJ01", "Hash Join cache not found"),
+
+    CANNOT_UPSERT_WITH_SCN_FOR_ROW_TIMSTAMP_COLUMN(901,"43M12",
+            "Cannot use a connection with SCN set to upsert data for " +
+                    "table with ROW_TIMESTAMP column."),
+    CANNOT_UPSERT_WITH_SCN_FOR_MUTABLE_TABLE_WITH_INDEXES(903,"43M14",
+            "Cannot use a connection with SCN set to " +
+                    "upsert data for a mutable table with indexes.");
+
 
     private final int errorCode;
     private final String sqlState;
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 6da579f..596e27c 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
@@ -858,7 +858,7 @@ public class PhoenixConnection implements Connection, MetaDataMutated, SQLClosea
 
     @Override
     public boolean isReadOnly() throws SQLException {
-        return readOnly || (scn != null && !buildingIndex && !isRunningUpgrade);
+        return readOnly;
     }
 
     @Override


[phoenix] 07/18: PHOENIX-5048 Index Rebuilder does not handle INDEX_STATE timestamp check for all index

Posted by pb...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

pboado pushed a commit to branch 4.x-cdh5.16
in repository https://gitbox.apache.org/repos/asf/phoenix.git

commit 4db9a6fb614a9d39130fe764adf52d92fb1ec8f7
Author: Monani Mihir <mo...@gmail.com>
AuthorDate: Fri Dec 14 12:45:55 2018 +0000

    PHOENIX-5048 Index Rebuilder does not handle INDEX_STATE timestamp check for all index
    
    Signed-off-by: Geoffrey Jacoby <gj...@apache.org>
---
 .../coprocessor/MetaDataRegionObserver.java        | 35 +++++++++++++---------
 1 file changed, 21 insertions(+), 14 deletions(-)

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 4968525..4045d47 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
@@ -512,20 +512,27 @@ public class MetaDataRegionObserver extends BaseRegionObserver {
 								String indexTableFullName = SchemaUtil.getTableName(
 										indexPTable.getSchemaName().getString(),
 										indexPTable.getTableName().getString());
-								if (scanEndTime == latestUpperBoundTimestamp) {
-									IndexUtil.updateIndexState(conn, indexTableFullName, PIndexState.ACTIVE, 0L, latestUpperBoundTimestamp);
-									batchExecutedPerTableMap.remove(dataPTable.getName());
-                                    LOG.info("Making Index:" + indexPTable.getTableName() + " active after rebuilding");
-								} else {
-								    // Increment timestamp so that client sees updated disable timestamp
-                                    IndexUtil.updateIndexState(conn, indexTableFullName, indexPTable.getIndexState(), scanEndTime * signOfDisableTimeStamp, latestUpperBoundTimestamp);
-									Long noOfBatches = batchExecutedPerTableMap.get(dataPTable.getName());
-									if (noOfBatches == null) {
-										noOfBatches = 0l;
-									}
-									batchExecutedPerTableMap.put(dataPTable.getName(), ++noOfBatches);
-									LOG.info("During Round-robin build: Successfully updated index disabled timestamp  for "
-													+ indexTableFullName + " to " + scanEndTime);
+								try {
+								    if (scanEndTime == latestUpperBoundTimestamp) {
+								        IndexUtil.updateIndexState(conn, indexTableFullName, PIndexState.ACTIVE, 0L,
+								            latestUpperBoundTimestamp);
+								        batchExecutedPerTableMap.remove(dataPTable.getName());
+								        LOG.info("Making Index:" + indexPTable.getTableName() + " active after rebuilding");
+								    } else {
+								        // Increment timestamp so that client sees updated disable timestamp
+								        IndexUtil.updateIndexState(conn, indexTableFullName, indexPTable.getIndexState(),
+								            scanEndTime * signOfDisableTimeStamp, latestUpperBoundTimestamp);
+								        Long noOfBatches = batchExecutedPerTableMap.get(dataPTable.getName());
+								        if (noOfBatches == null) {
+								            noOfBatches = 0l;
+								        }
+								        batchExecutedPerTableMap.put(dataPTable.getName(), ++noOfBatches);
+								        LOG.info(
+								            "During Round-robin build: Successfully updated index disabled timestamp  for "
+								                + indexTableFullName + " to " + scanEndTime);
+								    }
+								} catch (SQLException e) {
+								    LOG.error("Unable to rebuild " + dataPTable + " index " + indexTableFullName, e);
 								}
 							}
 						} catch (Exception e) {


[phoenix] 16/18: PHOENIX-5055 Split mutations batches probably affects correctness of index data

Posted by pb...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

pboado pushed a commit to branch 4.x-cdh5.16
in repository https://gitbox.apache.org/repos/asf/phoenix.git

commit 0e5a2635ea023d72459e63bd6443f3733642482b
Author: jaanai <ja...@apache.org>
AuthorDate: Sat Jan 5 13:17:42 2019 +0000

    PHOENIX-5055 Split mutations batches probably affects correctness of index data
---
 .../apache/phoenix/end2end/MutationStateIT.java    | 47 +++++++++++++++++++++-
 .../org/apache/phoenix/end2end/QueryMoreIT.java    |  6 +--
 .../org/apache/phoenix/execute/MutationState.java  | 41 ++++++++++++++-----
 .../apache/phoenix/execute/MutationStateTest.java  | 41 +++++++++++++++++++
 4 files changed, 122 insertions(+), 13 deletions(-)

diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MutationStateIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MutationStateIT.java
index 36782c1..5a5fb56 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MutationStateIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MutationStateIT.java
@@ -25,8 +25,14 @@ import java.sql.DriverManager;
 import java.sql.PreparedStatement;
 import java.sql.SQLException;
 import java.sql.Statement;
+import java.util.Iterator;
 import java.util.Properties;
 
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.execute.MutationState;
 import org.apache.phoenix.jdbc.PhoenixConnection;
@@ -157,5 +163,44 @@ public class MutationStateIT extends ParallelStatsDisabledIT {
         stmt.execute();
         assertTrue("Mutation state size should decrease", prevEstimatedSize+4 > state.getEstimatedSize());
     }
-    
+
+    @Test
+    public void testSplitMutationsIntoSameGroupForSingleRow() throws Exception {
+        String tableName = "TBL_" + generateUniqueName();
+        String indexName = "IDX_" + generateUniqueName();
+        Properties props = new Properties();
+        props.put("phoenix.mutate.batchSize", "2");
+        try (PhoenixConnection conn = DriverManager.getConnection(getUrl(), props).unwrap(PhoenixConnection.class)) {
+            conn.setAutoCommit(false);
+            conn.createStatement().executeUpdate(
+                    "CREATE TABLE "  + tableName + " ("
+                            + "A VARCHAR NOT NULL PRIMARY KEY,"
+                            + "B VARCHAR,"
+                            + "C VARCHAR,"
+                            + "D VARCHAR) COLUMN_ENCODED_BYTES = 0");
+            conn.createStatement().executeUpdate("CREATE INDEX " + indexName + " on "  + tableName + " (C) INCLUDE(D)");
+
+            conn.createStatement().executeUpdate("UPSERT INTO "  + tableName + "(A,B,C,D) VALUES ('A2','B2','C2','D2')");
+            conn.createStatement().executeUpdate("UPSERT INTO "  + tableName + "(A,B,C,D) VALUES ('A3','B3', 'C3', null)");
+            conn.commit();
+
+            Table htable = conn.getQueryServices().getTable(Bytes.toBytes(tableName));
+            Scan scan = new Scan();
+            scan.setRaw(true);
+            Iterator<Result> scannerIter = htable.getScanner(scan).iterator();
+            while (scannerIter.hasNext()) {
+                long ts = -1;
+                Result r = scannerIter.next();
+                for (Cell cell : r.listCells()) {
+                    if (ts == -1) {
+                        ts = cell.getTimestamp();
+                    } else {
+                        assertEquals("(" + cell.toString() + ") has different ts", ts, cell.getTimestamp());
+                    }
+                }
+            }
+            htable.close();
+        }
+    }
+
 }
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 2b1d31e..7c45f1a 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
@@ -493,14 +493,14 @@ public class QueryMoreIT extends ParallelStatsDisabledIT {
         connection.commit();
         assertEquals(2L, connection.getMutationState().getBatchCount());
         
-        // set the batch size (rows) to 1 
-        connectionProperties.setProperty(QueryServices.MUTATE_BATCH_SIZE_ATTRIB, "1");
+        // set the batch size (rows) to 2 since three are at least 2 mutations when updates a single row
+        connectionProperties.setProperty(QueryServices.MUTATE_BATCH_SIZE_ATTRIB, "2");
         connectionProperties.setProperty(QueryServices.MUTATE_BATCH_SIZE_BYTES_ATTRIB, "128");
         connection = (PhoenixConnection) DriverManager.getConnection(getUrl(), connectionProperties);
         upsertRows(connection, fullTableName);
         connection.commit();
         // each row should be in its own batch
-        assertEquals(4L, connection.getMutationState().getBatchCount());
+        assertEquals(2L, connection.getMutationState().getBatchCount());
     }
     
     private void upsertRows(PhoenixConnection conn, String fullTableName) throws SQLException {
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 14f13b3..e33a005 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
@@ -100,6 +100,7 @@ import org.apache.phoenix.util.TransactionUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.base.Preconditions;
 import com.google.common.base.Predicate;
 import com.google.common.collect.Iterators;
 import com.google.common.collect.Lists;
@@ -1092,34 +1093,56 @@ public class MutationState implements SQLCloseable {
     }
 
     /**
-     * Split the list of mutations into multiple lists that don't exceed row and byte thresholds
+     *
+     * Split the list of mutations into multiple lists. since a single row update can contain multiple mutations,
+     * we only check if the current batch has exceeded the row or size limit for different rows,
+     * so that mutations for a single row don't end up in different batches.
      * 
      * @param allMutationList
      *            List of HBase mutations
      * @return List of lists of mutations
      */
-    public static List<List<Mutation>> getMutationBatchList(long batchSize, long batchSizeBytes,
-            List<Mutation> allMutationList) {
+    public static List<List<Mutation>> getMutationBatchList(long batchSize, long batchSizeBytes, List<Mutation> allMutationList) {
+        Preconditions.checkArgument(batchSize> 1,
+                "Mutation types are put or delete, for one row all mutations must be in one batch.");
+        Preconditions.checkArgument(batchSizeBytes > 0, "Batch size must be larger than 0");
         List<List<Mutation>> mutationBatchList = Lists.newArrayList();
         List<Mutation> currentList = Lists.newArrayList();
+        List<Mutation> sameRowList = Lists.newArrayList();
         long currentBatchSizeBytes = 0L;
-        for (Mutation mutation : allMutationList) {
-            long mutationSizeBytes = KeyValueUtil.calculateMutationDiskSize(mutation);
-            if (currentList.size() == batchSize || currentBatchSizeBytes + mutationSizeBytes > batchSizeBytes) {
+        for (int i = 0; i < allMutationList.size(); ) {
+            long sameRowBatchSize = 1L;
+            Mutation mutation = allMutationList.get(i);
+            long sameRowMutationSizeBytes = KeyValueUtil.calculateMutationDiskSize(mutation);
+            sameRowList.add(mutation);
+            while (i + 1 < allMutationList.size() &&
+                    Bytes.compareTo(allMutationList.get(i + 1).getRow(), mutation.getRow()) == 0) {
+                Mutation sameRowMutation = allMutationList.get(i + 1);
+                sameRowList.add(sameRowMutation);
+                sameRowMutationSizeBytes += KeyValueUtil.calculateMutationDiskSize(sameRowMutation);
+                sameRowBatchSize++;
+                i++;
+            }
+
+            if (currentList.size() + sameRowBatchSize > batchSize ||
+                    currentBatchSizeBytes + sameRowMutationSizeBytes > batchSizeBytes) {
                 if (currentList.size() > 0) {
                     mutationBatchList.add(currentList);
                     currentList = Lists.newArrayList();
                     currentBatchSizeBytes = 0L;
                 }
             }
-            currentList.add(mutation);
-            currentBatchSizeBytes += mutationSizeBytes;
+
+            currentList.addAll(sameRowList);
+            currentBatchSizeBytes += sameRowMutationSizeBytes;
+            sameRowList.clear();
+            i++;
         }
+
         if (currentList.size() > 0) {
             mutationBatchList.add(currentList);
         }
         return mutationBatchList;
-
     }
 
     public byte[] encodeTransaction() throws SQLException {
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/execute/MutationStateTest.java b/phoenix-core/src/test/java/org/apache/phoenix/execute/MutationStateTest.java
index 8553b73..22662b2 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/execute/MutationStateTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/execute/MutationStateTest.java
@@ -29,6 +29,9 @@ import java.util.List;
 
 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.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.phoenix.schema.types.PUnsignedInt;
@@ -36,6 +39,8 @@ import org.apache.phoenix.schema.types.PVarchar;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.junit.Test;
 
+import com.google.common.collect.ImmutableList;
+
 public class MutationStateTest {
 
     @Test
@@ -134,4 +139,40 @@ public class MutationStateTest {
         assertTrue("app2".equals(PVarchar.INSTANCE.toObject(CellUtil.cloneValue(keyValues2.get(1)))));
 
     }
+
+    @Test
+    public void testGetMutationBatchList() {
+        byte[] r1 = Bytes.toBytes(1);
+        byte[] r2 = Bytes.toBytes(2);
+        byte[] r3 = Bytes.toBytes(3);
+        byte[] r4 = Bytes.toBytes(4);
+        // one put and one delete as a group
+        {
+            List<Mutation> list = ImmutableList.of(new Put(r1), new Put(r2), new Delete(r2));
+            List<List<Mutation>> batchLists = MutationState.getMutationBatchList(2, 10, list);
+            assertTrue(batchLists.size() == 2);
+            assertEquals(batchLists.get(0).size(), 1);
+            assertEquals(batchLists.get(1).size(), 2);
+        }
+
+        {
+            List<Mutation> list = ImmutableList.of(new Put(r1), new Delete(r1), new Put(r2));
+            List<List<Mutation>> batchLists = MutationState.getMutationBatchList(2, 10, list);
+            assertTrue(batchLists.size() == 2);
+            assertEquals(batchLists.get(0).size(), 2);
+            assertEquals(batchLists.get(1).size(), 1);
+        }
+
+        {
+            List<Mutation> list = ImmutableList.of(new Put(r3), new Put(r1), new Delete(r1), new Put(r2), new Put(r4), new Delete(r4));
+            List<List<Mutation>> batchLists = MutationState.getMutationBatchList(2, 10, list);
+            assertTrue(batchLists.size() == 4);
+            assertEquals(batchLists.get(0).size(), 1);
+            assertEquals(batchLists.get(1).size(), 2);
+            assertEquals(batchLists.get(2).size(), 1);
+            assertEquals(batchLists.get(3).size(), 2);
+        }
+
+    }
+
 }


[phoenix] 04/18: PHOENIX-4763: Changing a base table property value should be reflected in child views (if the property wasn't changed)

Posted by pb...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

pboado pushed a commit to branch 4.x-cdh5.16
in repository https://gitbox.apache.org/repos/asf/phoenix.git

commit 622fcf4802d83316111fd19203723e74f843f67d
Author: Chinmay Kulkarni <ch...@gmail.com>
AuthorDate: Mon Dec 10 05:07:41 2018 +0000

    PHOENIX-4763: Changing a base table property value should be reflected in child views (if the property wasn't changed)
---
 .../phoenix/end2end/AlterTableWithViewsIT.java     | 117 +++++--
 .../end2end/ExplainPlanWithStatsEnabledIT.java     |   8 +-
 .../apache/phoenix/end2end/PropertiesInSyncIT.java |   6 +-
 .../IndexHalfStoreFileReaderGenerator.java         |   3 +-
 .../org/apache/phoenix/compile/DeleteCompiler.java |   2 +-
 .../org/apache/phoenix/compile/JoinCompiler.java   |   2 +-
 .../phoenix/compile/TupleProjectionCompiler.java   |   3 +-
 .../org/apache/phoenix/compile/UpsertCompiler.java |   2 +-
 .../org/apache/phoenix/compile/WhereOptimizer.java |   3 +-
 .../phoenix/coprocessor/MetaDataEndpointImpl.java  | 134 ++++++--
 .../phoenix/coprocessor/MetaDataProtocol.java      |  32 +-
 .../coprocessor/generated/MetaDataProtos.java      | 356 ++++++++++-----------
 .../coprocessor/generated/PTableProtos.java        |  99 +++---
 .../coprocessor/generated/ServerCachingProtos.java | 122 +++----
 .../org/apache/phoenix/index/IndexMaintainer.java  |  16 +-
 .../phoenix/index/PhoenixIndexFailurePolicy.java   |   2 +-
 .../org/apache/phoenix/schema/DelegateTable.java   |  12 +-
 .../org/apache/phoenix/schema/MetaDataClient.java  |  25 +-
 .../java/org/apache/phoenix/schema/PTable.java     |   6 +-
 .../java/org/apache/phoenix/schema/PTableImpl.java |  56 +++-
 .../java/org/apache/phoenix/util/MetaDataUtil.java |  46 ++-
 .../org/apache/phoenix/util/MetaDataUtilTest.java  | 115 +++++--
 phoenix-protocol/src/main/MetaDataService.proto    |   4 +-
 phoenix-protocol/src/main/PTable.proto             |   2 +-
 .../src/main/ServerCachingService.proto            |   2 +-
 25 files changed, 739 insertions(+), 436 deletions(-)

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 9e7aaa2..82a119f 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
@@ -73,6 +73,8 @@ import org.junit.runners.Parameterized.Parameters;
 import com.google.common.base.Function;
 import com.google.common.collect.Lists;
 
+import static org.apache.phoenix.query.QueryServicesOptions.DEFAULT_USE_STATS_FOR_PARALLELIZATION;
+
 @RunWith(Parameterized.class)
 public class AlterTableWithViewsIT extends SplitSystemCatalogIT {
 
@@ -174,41 +176,53 @@ public class AlterTableWithViewsIT extends SplitSystemCatalogIT {
             conn.createStatement().execute(generateDDL("UPDATE_CACHE_FREQUENCY=2", ddlFormat));
             viewConn.createStatement().execute("CREATE VIEW " + viewOfTable1 + " ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 VARCHAR ) AS SELECT * FROM " + tableName);
             viewConn.createStatement().execute("CREATE VIEW " + viewOfTable2 + " ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 VARCHAR ) AS SELECT * FROM " + tableName);
-            
-            viewConn.createStatement().execute("ALTER VIEW " + viewOfTable2 + " SET UPDATE_CACHE_FREQUENCY = 1");
-            
-            PhoenixConnection phoenixConn = conn.unwrap(PhoenixConnection.class);
-            PTable table = phoenixConn.getTable(new PTableKey(null, tableName));
             PName tenantId = isMultiTenant ? PNameFactory.newName(TENANT1) : null;
-            assertFalse(table.isImmutableRows());
-            assertEquals(2, table.getUpdateCacheFrequency());
+
+            // Initially all property values should be the same for the base table and its views
+            PTable table = conn.unwrap(PhoenixConnection.class).getTable(new PTableKey(null, tableName));
             PTable viewTable1 = viewConn.unwrap(PhoenixConnection.class).getTable(new PTableKey(tenantId, viewOfTable1));
+            PTable viewTable2 = viewConn.unwrap(PhoenixConnection.class).getTable(new PTableKey(tenantId, viewOfTable2));
+            assertFalse(table.isImmutableRows());
             assertFalse(viewTable1.isImmutableRows());
+            assertFalse(viewTable2.isImmutableRows());
+            assertEquals(2, table.getUpdateCacheFrequency());
             assertEquals(2, viewTable1.getUpdateCacheFrequency());
+            assertEquals(2, viewTable2.getUpdateCacheFrequency());
+            assertNull(table.useStatsForParallelization());
+            assertNull(viewTable1.useStatsForParallelization());
+            assertNull(viewTable2.useStatsForParallelization());
+
+            // Alter a property value for one of the views
+            viewConn.createStatement().execute("ALTER VIEW " + viewOfTable2
+                    + " SET UPDATE_CACHE_FREQUENCY=1, USE_STATS_FOR_PARALLELIZATION=false");
             // query the view to force the table cache to be updated
             viewConn.createStatement().execute("SELECT * FROM "+viewOfTable2);
-            PTable viewTable2 = viewConn.unwrap(PhoenixConnection.class).getTable(new PTableKey(tenantId, viewOfTable2));
-            assertFalse(viewTable2.isImmutableRows());
+            viewTable2 = viewConn.unwrap(PhoenixConnection.class).getTable(new PTableKey(tenantId, viewOfTable2));
             assertEquals(1, viewTable2.getUpdateCacheFrequency());
-            
-            conn.createStatement().execute("ALTER TABLE " + tableName + " SET IMMUTABLE_ROWS=true, UPDATE_CACHE_FREQUENCY=3");
+            assertFalse(viewTable2.useStatsForParallelization());
+
+            // Alter a property value for the base table. So the view for which this property was
+            // not modified earlier should get the base table's altered property value
+            conn.createStatement().execute("ALTER TABLE " + tableName
+                    + " SET IMMUTABLE_ROWS=true, UPDATE_CACHE_FREQUENCY=3, "
+                    + "USE_STATS_FOR_PARALLELIZATION=true");
             // query the views to force the table cache to be updated
             viewConn.createStatement().execute("SELECT * FROM "+viewOfTable1);
             viewConn.createStatement().execute("SELECT * FROM "+viewOfTable2);
-            
-            phoenixConn = conn.unwrap(PhoenixConnection.class);
-            table = phoenixConn.getTable(new PTableKey(null, tableName));
-            assertTrue(table.isImmutableRows());
-            assertEquals(3, table.getUpdateCacheFrequency());
-            
+            table = conn.unwrap(PhoenixConnection.class).getTable(new PTableKey(null, tableName));
             viewTable1 = viewConn.unwrap(PhoenixConnection.class).getTable(new PTableKey(tenantId, viewOfTable1));
-            assertTrue(viewTable1.isImmutableRows());
-            assertEquals(2, viewTable1.getUpdateCacheFrequency());
-            
             viewTable2 = viewConn.unwrap(PhoenixConnection.class).getTable(new PTableKey(tenantId, viewOfTable2));
+            assertTrue(table.isImmutableRows());
+            assertTrue(viewTable1.isImmutableRows());
             assertTrue(viewTable2.isImmutableRows());
-            // update cache frequency is not propagated to the view since it was altered on the view
+            assertEquals(3, table.getUpdateCacheFrequency());
+            // The updated property value in the base table is reflected in this view
+            assertEquals(3, viewTable1.getUpdateCacheFrequency());
+            // The update property is not propagated to this view since it was altered on the view
             assertEquals(1, viewTable2.getUpdateCacheFrequency());
+            assertTrue(table.useStatsForParallelization());
+            assertTrue(viewTable1.useStatsForParallelization());
+            assertFalse(viewTable2.useStatsForParallelization());
 
             long gpw = 1000000;
             conn.createStatement().execute("ALTER TABLE " + tableName + " SET GUIDE_POSTS_WIDTH=" + gpw);
@@ -234,6 +248,67 @@ public class AlterTableWithViewsIT extends SplitSystemCatalogIT {
             assertTrue(rs.wasNull());
         } 
     }
+
+    @Test
+    public void testCreateViewWithPropsMaintainsOwnProps() throws Exception {
+        try (Connection conn = DriverManager.getConnection(getUrl());
+                Connection viewConn = isMultiTenant ?
+                        DriverManager.getConnection(TENANT_SPECIFIC_URL1) : conn) {
+            String tableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
+            String viewOfTable1 = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
+            String viewOfTable2 = SchemaUtil.getTableName(SCHEMA3, generateUniqueName());
+
+            String ddlFormat = "CREATE TABLE IF NOT EXISTS " + tableName + " ("
+                    + " %s ID char(1) NOT NULL," + " COL1 integer NOT NULL, COL2 bigint NOT NULL,"
+                    + " CONSTRAINT NAME_PK PRIMARY KEY (%s ID, COL1, COL2)) %s ";
+            conn.createStatement().execute(generateDDL("UPDATE_CACHE_FREQUENCY=2", ddlFormat));
+
+            viewConn.createStatement().execute("CREATE VIEW " + viewOfTable1
+                    + " ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 VARCHAR ) AS SELECT * FROM "
+                    + tableName + " UPDATE_CACHE_FREQUENCY=7");
+            viewConn.createStatement().execute("CREATE VIEW " + viewOfTable2
+                    + " ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 VARCHAR ) AS SELECT * FROM "
+                    + tableName + " USE_STATS_FOR_PARALLELIZATION=true");
+            PName tenantId = isMultiTenant ? PNameFactory.newName(TENANT1) : null;
+
+            // Initially, property values not specified during view creation should be the same for
+            // the base table and its views those specified should have corresponding values
+            PTable table = conn.unwrap(PhoenixConnection.class)
+                    .getTable(new PTableKey(null, tableName));
+            PTable viewTable1 = viewConn.unwrap(PhoenixConnection.class)
+                    .getTable(new PTableKey(tenantId, viewOfTable1));
+            PTable viewTable2 = viewConn.unwrap(PhoenixConnection.class)
+                    .getTable(new PTableKey(tenantId, viewOfTable2));
+            assertEquals(2, table.getUpdateCacheFrequency());
+            assertEquals(7, viewTable1.getUpdateCacheFrequency());
+            assertEquals(2, viewTable2.getUpdateCacheFrequency());
+            assertNull(table.useStatsForParallelization());
+            assertNull(viewTable1.useStatsForParallelization());
+            assertTrue(viewTable2.useStatsForParallelization());
+
+            // Alter a property value for the base table. So the view for which this property was
+            // not explicitly set or modified earlier should get the base table's new property value
+            conn.createStatement().execute("ALTER TABLE " + tableName
+                    + " SET UPDATE_CACHE_FREQUENCY=3, USE_STATS_FOR_PARALLELIZATION=false");
+            // query the views to force the table cache to be updated
+            viewConn.createStatement().execute("SELECT * FROM " + viewOfTable1);
+            viewConn.createStatement().execute("SELECT * FROM " + viewOfTable2);
+            table = conn.unwrap(PhoenixConnection.class)
+                    .getTable(new PTableKey(null, tableName));
+            viewTable1 = viewConn.unwrap(PhoenixConnection.class)
+                    .getTable(new PTableKey(tenantId, viewOfTable1));
+            viewTable2 = viewConn.unwrap(PhoenixConnection.class)
+                    .getTable(new PTableKey(tenantId, viewOfTable2));
+            assertEquals(3, table.getUpdateCacheFrequency());
+            // The updated property value is only propagated to the view in which we did not specify
+            // a value for the property during view creation or alter its value later on
+            assertEquals(7, viewTable1.getUpdateCacheFrequency());
+            assertEquals(3, viewTable2.getUpdateCacheFrequency());
+            assertFalse(table.useStatsForParallelization());
+            assertFalse(viewTable1.useStatsForParallelization());
+            assertTrue(viewTable2.useStatsForParallelization());
+        }
+    }
     
     @Test
     public void testDropColumnsFromBaseTableWithView() throws Exception {
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
index 0dc4410..bd95dc1 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
@@ -1220,12 +1220,8 @@ public class ExplainPlanWithStatsEnabledIT extends ParallelStatsEnabledIT {
                 tenantConn.createStatement().execute("CREATE VIEW " + tenantViewName + " AS SELECT * FROM " + viewName);
                 conn.createStatement()
                         .execute("ALTER TABLE " + tableName + " set USE_STATS_FOR_PARALLELIZATION=" + useStats);
-                // changing a property on a base table does not change the property on a view
-                validatePropertyOnViewIndex(viewName, viewIndexName, !useStats, conn, tenantConn);
-
-                // need to explicitly change the property on the view
-                conn.createStatement()
-                        .execute("ALTER VIEW " + viewName + " set USE_STATS_FOR_PARALLELIZATION=" + useStats);
+                // changing a property on a base table is propagated to its view
+                // if the view has not previously modified the property
                 validatePropertyOnViewIndex(viewName, viewIndexName, useStats, conn, tenantConn);
             }
         }
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/PropertiesInSyncIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PropertiesInSyncIT.java
index 66379d4..cf3c27e 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/PropertiesInSyncIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PropertiesInSyncIT.java
@@ -398,8 +398,10 @@ public class PropertiesInSyncIT extends ParallelStatsDisabledIT {
             }
         }
         // Now synchronize required properties and verify HBase metadata property values
-        syncTableAndIndexProperties(conn.unwrap(PhoenixConnection.class),
-                conn.unwrap(PhoenixConnection.class).getQueryServices().getAdmin());
+        PhoenixConnection upgradeConn = conn.unwrap(PhoenixConnection.class);
+        // Simulate an upgrade by setting the upgrade flag
+        upgradeConn.setRunningUpgrade(true);
+        syncTableAndIndexProperties(upgradeConn, upgradeConn.getQueryServices().getAdmin());
         for (String t: createdTables) {
             verifyHBaseColumnFamilyProperties(t, conn, false, false);
         }
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexHalfStoreFileReaderGenerator.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexHalfStoreFileReaderGenerator.java
index e56dd9d..fcf8547 100644
--- a/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexHalfStoreFileReaderGenerator.java
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/regionserver/IndexHalfStoreFileReaderGenerator.java
@@ -59,7 +59,6 @@ import org.apache.phoenix.schema.PTable.IndexType;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.IndexUtil;
-import org.apache.phoenix.util.MetaDataUtil;
 import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.RepairUtil;
 
@@ -157,7 +156,7 @@ public class IndexHalfStoreFileReaderGenerator extends BaseRegionObserver {
                 for (PTable index : indexes) {
                     if (index.getIndexType() == IndexType.LOCAL) {
                         IndexMaintainer indexMaintainer = index.getIndexMaintainer(dataTable, conn);
-                        indexMaintainers.put(new ImmutableBytesWritable(index.getViewIndexType().toBytes(index.getViewIndexId())),
+                        indexMaintainers.put(new ImmutableBytesWritable(index.getviewIndexIdType().toBytes(index.getViewIndexId())),
                             indexMaintainer);
                     }
                 }
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
index ed21374..d414205 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
@@ -156,7 +156,7 @@ public class DeleteCompiler {
         int offset = (table.getBucketNum() == null ? 0 : 1);
         byte[][] values = new byte[pkColumns.size()][];
         if (isSharedViewIndex) {
-            values[offset++] = table.getViewIndexType().toBytes(table.getViewIndexId());
+            values[offset++] = table.getviewIndexIdType().toBytes(table.getViewIndexId());
         }
         if (isMultiTenant) {
             values[offset++] = tenantIdBytes;
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/JoinCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/JoinCompiler.java
index 807e54b..b2b660e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/JoinCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/JoinCompiler.java
@@ -1283,7 +1283,7 @@ public class JoinCompiler {
                 .setMultiTenant(left.isMultiTenant())
                 .setStoreNulls(left.getStoreNulls())
                 .setViewType(left.getViewType())
-                .setViewIndexType(left.getViewIndexType())
+                .setViewIndexIdType(left.getviewIndexIdType())
                 .setViewIndexId(left.getViewIndexId())
                 .setIndexType(left.getIndexType())
                 .setTransactionProvider(left.getTransactionProvider())
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/TupleProjectionCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/TupleProjectionCompiler.java
index 5b92a5d..40a0ee4 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/TupleProjectionCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/TupleProjectionCompiler.java
@@ -18,7 +18,6 @@
 package org.apache.phoenix.compile;
 import static org.apache.phoenix.query.QueryConstants.VALUE_COLUMN_FAMILY;
 import static org.apache.phoenix.query.QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT;
-import static org.apache.phoenix.schema.PTable.ImmutableStorageScheme;
 
 import java.sql.SQLException;
 import java.util.ArrayList;
@@ -203,7 +202,7 @@ public class TupleProjectionCompiler {
                 .setMultiTenant(table.isMultiTenant())
                 .setStoreNulls(table.getStoreNulls())
                 .setViewType(table.getViewType())
-                .setViewIndexType(table.getViewIndexType())
+                .setViewIndexIdType(table.getviewIndexIdType())
                 .setViewIndexId(table.getViewIndexId())
                 .setTransactionProvider(table.getTransactionProvider())
                 .setUpdateCacheFrequency(table.getUpdateCacheFrequency())
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 410ac22..a770339 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
@@ -753,7 +753,7 @@ public class UpsertCompiler {
         final byte[][] values = new byte[nValuesToSet][];
         int nodeIndex = 0;
         if (isSharedViewIndex) {
-            values[nodeIndex++] = table.getViewIndexType().toBytes(table.getViewIndexId());
+            values[nodeIndex++] = table.getviewIndexIdType().toBytes(table.getViewIndexId());
         }
         if (isTenantSpecific) {
             PName tenantId = connection.getTenantId();
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 b2e4c41..b845a09 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
@@ -69,7 +69,6 @@ import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.types.PVarbinary;
 import org.apache.phoenix.schema.types.PVarchar;
 import org.apache.phoenix.util.ByteUtil;
-import org.apache.phoenix.util.MetaDataUtil;
 import org.apache.phoenix.util.ScanUtil;
 import org.apache.phoenix.util.SchemaUtil;
 
@@ -179,7 +178,7 @@ public class WhereOptimizer {
         // Add unique index ID for shared indexes on views. This ensures
         // that different indexes don't interleave.
         if (hasViewIndex) {
-            byte[] viewIndexBytes = table.getViewIndexType().toBytes(table.getViewIndexId());
+            byte[] viewIndexBytes = table.getviewIndexIdType().toBytes(table.getViewIndexId());
             KeyRange indexIdKeyRange = KeyRange.getKeyRange(viewIndexBytes);
             cnf.add(Collections.singletonList(indexIdKeyRange));
             pkPos++;
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 d138132..cd9efee 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
@@ -87,7 +87,6 @@ import java.security.PrivilegedExceptionAction;
 import java.sql.ResultSetMetaData;
 import java.sql.SQLException;
 import java.sql.Statement;
-import java.sql.Timestamp;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
@@ -464,6 +463,9 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
 
     // index for link type key value that is used to store linking rows
     private static final int LINK_TYPE_INDEX = 0;
+    // Used to add a tag to a cell when a view modifies a table property to indicate that this
+    // property should not be derived from the base table
+    private static final byte[] VIEW_MODIFIED_PROPERTY_BYTES = Bytes.toBytes(1);
 
     private static final KeyValue CLASS_NAME_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, CLASS_NAME_BYTES);
     private static final KeyValue JAR_PATH_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, JAR_PATH_BYTES);
@@ -786,11 +788,12 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
 
         // now go up from child to parent all the way to the base table:
         PTable baseTable = null;
+        PTable immediateParent = null;
         long maxTableTimestamp = -1;
         int numPKCols = table.getPKColumns().size();
         for (int i = 0; i < ancestorList.size(); i++) {
             TableInfo parentTableInfo = ancestorList.get(i);
-            PTable pTable = null;
+            PTable pTable;
             String fullParentTableName = SchemaUtil.getTableName(parentTableInfo.getSchemaName(),
                 parentTableInfo.getTableName());
             PName parentTenantId =
@@ -815,6 +818,9 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             if (pTable == null) {
                 throw new ParentTableNotFoundException(parentTableInfo, fullTableName);
             } else {
+                if (immediateParent == null) {
+                    immediateParent = pTable;
+                }
                 // only combine columns for view indexes (and not local indexes on regular tables
                 // which also have a viewIndexId)
                 if (i == 0 && hasIndexId && pTable.getType() != PTableType.VIEW) {
@@ -951,13 +957,21 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 isDiverged ? QueryConstants.DIVERGED_VIEW_BASE_COLUMN_COUNT
                         : columnsToAdd.size() - myColumns.size() + (isSalted ? 1 : 0);
 
+        // Inherit view-modifiable properties from the parent table/view if the current view has
+        // not previously modified this property
+        Long updateCacheFreq = (table.getType() != PTableType.VIEW ||
+                table.hasViewModifiedUpdateCacheFrequency()) ?
+                table.getUpdateCacheFrequency() : immediateParent.getUpdateCacheFrequency();
+        Boolean useStatsForParallelization = (table.getType() != PTableType.VIEW ||
+                table.hasViewModifiedUseStatsForParallelization()) ?
+                table.useStatsForParallelization() : immediateParent.useStatsForParallelization();
         // When creating a PTable for views or view indexes, use the baseTable PTable for attributes
         // inherited from the physical base table.
         // if a TableProperty is not valid on a view we set it to the base table value
         // if a TableProperty is valid on a view and is not mutable on a view we set it to the base table value
-        // if a TableProperty is valid on a view and is mutable on a view we use the value set on the view
-        // TODO Implement PHOENIX-4763 to set the view properties correctly instead of just
-        // setting them same as the base table
+        // if a TableProperty is valid on a view and is mutable on a view, we use the value set
+        // on the view if the view had previously modified the property, otherwise we propagate the
+        // value from the base table (see PHOENIX-4763)
         PTableImpl pTable = PTableImpl.builderWithColumns(table, columnsToAdd)
                 .setImmutableRows(baseTable.isImmutableRows())
                 .setDisableWAL(baseTable.isWALDisabled())
@@ -974,6 +988,8 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 .setTimeStamp(maxTableTimestamp)
                 .setExcludedColumns(excludedColumns == null ?
                         ImmutableList.<PColumn>of() : ImmutableList.copyOf(excludedColumns))
+                .setUpdateCacheFrequency(updateCacheFreq)
+                .setUseStatsForParallelization(useStatsForParallelization)
                 .build();
         return WhereConstantParser.addViewInfoToPColumnsIfNeeded(pTable);
     }
@@ -1436,8 +1452,8 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         }
         Cell viewTypeKv = tableKeyValues[VIEW_TYPE_INDEX];
         ViewType viewType = viewTypeKv == null ? null : ViewType.fromSerializedValue(viewTypeKv.getValueArray()[viewTypeKv.getValueOffset()]);
-        PDataType viewIndexType = getViewIndexType(tableKeyValues);
-        Long viewIndexId = getViewIndexId(tableKeyValues, viewIndexType);
+        PDataType viewIndexIdType = getViewIndexIdType(tableKeyValues);
+        Long viewIndexId = getViewIndexId(tableKeyValues, viewIndexIdType);
         Cell indexTypeKv = tableKeyValues[INDEX_TYPE_INDEX];
         IndexType indexType = indexTypeKv == null ? null : IndexType.fromSerializedValue(indexTypeKv.getValueArray()[indexTypeKv.getValueOffset()]);
         Cell baseColumnCountKv = tableKeyValues[BASE_COLUMN_COUNT_INDEX];
@@ -1449,6 +1465,12 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         long updateCacheFrequency = updateCacheFrequencyKv == null ? 0 :
             PLong.INSTANCE.getCodec().decodeLong(updateCacheFrequencyKv.getValueArray(),
                     updateCacheFrequencyKv.getValueOffset(), SortOrder.getDefault());
+
+        // Check the cell tag to see whether the view has modified this property
+        final byte[] tagUpdateCacheFreq = (updateCacheFrequencyKv == null) ?
+                HConstants.EMPTY_BYTE_ARRAY : CellUtil.getTagArray(updateCacheFrequencyKv);
+        boolean viewModifiedUpdateCacheFrequency = (PTableType.VIEW.equals(tableType)) &&
+                Bytes.contains(tagUpdateCacheFreq, VIEW_MODIFIED_PROPERTY_BYTES);
         Cell indexDisableTimestampKv = tableKeyValues[INDEX_DISABLE_TIMESTAMP];
         long indexDisableTimestamp = indexDisableTimestampKv == null ? 0L : PLong.INSTANCE.getCodec().decodeLong(indexDisableTimestampKv.getValueArray(),
                 indexDisableTimestampKv.getValueOffset(), SortOrder.getDefault());
@@ -1474,6 +1496,12 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     encodingSchemeKv.getValueOffset(), encodingSchemeKv.getValueLength()));
         Cell useStatsForParallelizationKv = tableKeyValues[USE_STATS_FOR_PARALLELIZATION_INDEX];
         Boolean useStatsForParallelization = useStatsForParallelizationKv == null ? null : Boolean.TRUE.equals(PBoolean.INSTANCE.toObject(useStatsForParallelizationKv.getValueArray(), useStatsForParallelizationKv.getValueOffset(), useStatsForParallelizationKv.getValueLength()));
+
+        // Check the cell tag to see whether the view has modified this property
+        final byte[] tagUseStatsForParallelization = (useStatsForParallelizationKv == null) ?
+                HConstants.EMPTY_BYTE_ARRAY : CellUtil.getTagArray(useStatsForParallelizationKv);
+        boolean viewModifiedUseStatsForParallelization = (PTableType.VIEW.equals(tableType)) &&
+                Bytes.contains(tagUseStatsForParallelization, VIEW_MODIFIED_PROPERTY_BYTES);
         
         List<PColumn> columns = Lists.newArrayListWithExpectedSize(columnCount);
         List<PTable> indexes = Lists.newArrayList();
@@ -1529,7 +1557,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 .setMultiTenant(multiTenant)
                 .setStoreNulls(storeNulls)
                 .setViewType(viewType)
-                .setViewIndexType(viewIndexType)
+                .setViewIndexIdType(viewIndexIdType)
                 .setViewIndexId(viewIndexId)
                 .setIndexType(indexType)
                 .setTransactionProvider(transactionProvider)
@@ -1557,28 +1585,30 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 .setParentTableName(parentTableName)
                 .setPhysicalNames(physicalTables == null ?
                         ImmutableList.<PName>of() : ImmutableList.copyOf(physicalTables))
+                .setViewModifiedUpdateCacheFrequency(viewModifiedUpdateCacheFrequency)
+                .setViewModifiedUseStatsForParallelization(viewModifiedUseStatsForParallelization)
                 .setColumns(columns)
                 .build();
     }
-    private Long getViewIndexId(Cell[] tableKeyValues, PDataType viewIndexType) {
+    private Long getViewIndexId(Cell[] tableKeyValues, PDataType viewIndexIdType) {
         Cell viewIndexIdKv = tableKeyValues[VIEW_INDEX_ID_INDEX];
         return viewIndexIdKv == null ? null :
-                decodeViewIndexId(viewIndexIdKv, viewIndexType);
+                decodeViewIndexId(viewIndexIdKv, viewIndexIdType);
     }
 
     /**
      * Returns viewIndexId based on its underlying data type
      *
-     * @param tableKeyValues
-     * @param viewIndexType
+     * @param viewIndexIdKv
+     * @param viewIndexIdType
      * @return
      */
-    private Long decodeViewIndexId(Cell viewIndexIdKv, PDataType viewIndexType) {
-        return viewIndexType.getCodec().decodeLong(viewIndexIdKv.getValueArray(),
+    private Long decodeViewIndexId(Cell viewIndexIdKv, PDataType viewIndexIdType) {
+        return viewIndexIdType.getCodec().decodeLong(viewIndexIdKv.getValueArray(),
                 viewIndexIdKv.getValueOffset(), SortOrder.getDefault());
     }
 
-    private PDataType getViewIndexType(Cell[] tableKeyValues) {
+    private PDataType getViewIndexIdType(Cell[] tableKeyValues) {
         Cell dataTypeKv = tableKeyValues[VIEW_INDEX_ID_DATA_TYPE_INDEX];
         return dataTypeKv == null ?
                 MetaDataUtil.getLegacyViewIndexIdDataType() :
@@ -2349,6 +2379,12 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     return;
                 }
 
+                if (tableType == PTableType.VIEW) {
+                    // Pass in the parent's PTable so that we only tag cells corresponding to the
+                    // view's property in case they are different from the parent
+                    addTagsToPutsForViewAlteredProperties(tableMetadata, parentTable);
+                }
+
                 // When we drop a view we first drop the view metadata and then drop the parent->child linking row
                 List<Mutation> localMutations =
                         Lists.newArrayListWithExpectedSize(tableMetadata.size());
@@ -2406,7 +2442,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 builder.setReturnCode(MetaDataProtos.MutationCode.TABLE_NOT_FOUND);
                 if (indexId != null) {
                    builder.setViewIndexId(indexId);
-                   builder.setViewIndexType(PLong.INSTANCE.getSqlType());
+                   builder.setViewIndexIdType(PLong.INSTANCE.getSqlType());
                 }
                 builder.setMutationTime(currentTimeStamp);
                 done.run(builder.build());
@@ -3514,19 +3550,23 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                         }
                     }
                     tableMetaData.addAll(additionalTableMetadataMutations);
-                    if (type == PTableType.VIEW
-                                && EncodedColumnsUtil.usesEncodedColumnNames(table) && addingCol
+                    if (type == PTableType.VIEW) {
+                        if (EncodedColumnsUtil.usesEncodedColumnNames(table) && addingCol
                                 && !table.isAppendOnlySchema()) {
-                                // When adding a column to a view that uses encoded column name
-                                // scheme, we need to modify the CQ counters stored in the view's
-                                // physical table. So to make sure clients get the latest PTable, we
-                                // need to invalidate the cache entry.
-                                // If the table uses APPEND_ONLY_SCHEMA we use the position of the
-                                // column as the encoded column qualifier and so we don't need to
-                                // update the CQ counter in the view physical table (see
-                                // PHOENIX-4737)
-                                invalidateList.add(new ImmutableBytesPtr(
-                                        MetaDataUtil.getPhysicalTableRowForView(table)));
+                            // When adding a column to a view that uses encoded column name
+                            // scheme, we need to modify the CQ counters stored in the view's
+                            // physical table. So to make sure clients get the latest PTable, we
+                            // need to invalidate the cache entry.
+                            // If the table uses APPEND_ONLY_SCHEMA we use the position of the
+                            // column as the encoded column qualifier and so we don't need to
+                            // update the CQ counter in the view physical table (see
+                            // PHOENIX-4737)
+                            invalidateList.add(new ImmutableBytesPtr(
+                                    MetaDataUtil.getPhysicalTableRowForView(table)));
+                        }
+                        // Pass in null as the parent PTable, since we always want to tag the cells
+                        // in this case, irrespective of the property values of the parent
+                        addTagsToPutsForViewAlteredProperties(tableMetaData, null);
                     }
                     return null;
                 }
@@ -3541,6 +3581,44 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         }
     }
 
+    /**
+     * See PHOENIX-4763. If we are modifying any table-level properties that are mutable on a view,
+     * we mark these cells in SYSTEM.CATALOG with tags to indicate that this view property should
+     * not be kept in-sync with the base table and so we shouldn't propagate the base table's
+     * property value when resolving the view
+     * @param tableMetaData list of mutations on the view
+     * @param parent PTable of the parent or null
+     */
+    private void addTagsToPutsForViewAlteredProperties(List<Mutation> tableMetaData,
+            PTable parent) {
+        byte[] parentUpdateCacheFreqBytes = null;
+        byte[] parentUseStatsForParallelizationBytes = null;
+        if (parent != null) {
+            parentUpdateCacheFreqBytes = new byte[PLong.INSTANCE.getByteSize()];
+            PLong.INSTANCE.getCodec().encodeLong(parent.getUpdateCacheFrequency(),
+                    parentUpdateCacheFreqBytes, 0);
+            if (parent.useStatsForParallelization() != null) {
+                parentUseStatsForParallelizationBytes =
+                        PBoolean.INSTANCE.toBytes(parent.useStatsForParallelization());
+            }
+        }
+        for (Mutation m: tableMetaData) {
+            if (m instanceof Put) {
+                MetaDataUtil.conditionallyAddTagsToPutCells((Put)m,
+                        PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES,
+                        PhoenixDatabaseMetaData.UPDATE_CACHE_FREQUENCY_BYTES,
+                        parentUpdateCacheFreqBytes,
+                        VIEW_MODIFIED_PROPERTY_BYTES);
+                MetaDataUtil.conditionallyAddTagsToPutCells((Put)m,
+                        PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES,
+                        PhoenixDatabaseMetaData.USE_STATS_FOR_PARALLELIZATION_BYTES,
+                        parentUseStatsForParallelizationBytes,
+                        VIEW_MODIFIED_PROPERTY_BYTES);
+            }
+
+        }
+    }
+
     private PTable doGetTable(byte[] tenantId, byte[] schemaName, byte[] tableName,
             long clientTimeStamp, int clientVersion) throws IOException, SQLException {
         return doGetTable(tenantId, schemaName, tableName, clientTimeStamp, null, clientVersion,
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
index 86878c2..4160b8b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
@@ -175,7 +175,7 @@ public abstract class MetaDataProtocol extends MetaDataService {
         private PName tableName;
         private List<PColumn> columns;
         private List<PName> physicalNames;
-        private PDataType viewIndexType;
+        private PDataType viewIndexIdType;
         private Long viewIndexId;
         
         public SharedTableState(PTable table) {
@@ -184,7 +184,7 @@ public abstract class MetaDataProtocol extends MetaDataService {
             this.tableName = table.getTableName();
             this.columns = table.getColumns();
             this.physicalNames = table.getPhysicalNames();
-            this.viewIndexType = table.getViewIndexType();
+            this.viewIndexIdType = table.getviewIndexIdType();
             this.viewIndexId = table.getViewIndexId();
         }
         
@@ -208,8 +208,8 @@ public abstract class MetaDataProtocol extends MetaDataService {
                 }
             });
             this.viewIndexId = sharedTable.getViewIndexId();
-            this.viewIndexType = sharedTable.hasViewIndexType()
-                    ? PDataType.fromTypeId(sharedTable.getViewIndexType())
+            this.viewIndexIdType = sharedTable.hasViewIndexIdType()
+                    ? PDataType.fromTypeId(sharedTable.getViewIndexIdType())
                     : MetaDataUtil.getLegacyViewIndexIdDataType();
         }
 
@@ -237,8 +237,8 @@ public abstract class MetaDataProtocol extends MetaDataService {
             return viewIndexId;
         }
 
-        public PDataType getViewIndexType() {
-          return viewIndexType;
+        public PDataType getViewIndexIdType() {
+          return viewIndexIdType;
         }
   }
     
@@ -253,7 +253,7 @@ public abstract class MetaDataProtocol extends MetaDataService {
         private boolean wasUpdated;
         private PSchema schema;
         private Long viewIndexId;
-        private PDataType viewIndexType;
+        private PDataType viewIndexIdType;
         private List<PFunction> functions = new ArrayList<PFunction>(1);
         private long autoPartitionNum;
 
@@ -298,10 +298,10 @@ public abstract class MetaDataProtocol extends MetaDataService {
             this.tableNamesToDelete = tableNamesToDelete;
         }
         
-        public MetaDataMutationResult(MutationCode returnCode, int currentTime, PTable table, long viewIndexId, PDataType viewIndexType ) {
+        public MetaDataMutationResult(MutationCode returnCode, int currentTime, PTable table, long viewIndexId, PDataType viewIndexIdType) {
             this(returnCode, currentTime, table, Collections.<byte[]> emptyList());
             this.viewIndexId = viewIndexId;
-            this.viewIndexType = viewIndexType;
+            this.viewIndexIdType = viewIndexIdType;
         }
         
         public MetaDataMutationResult(MutationCode returnCode, long currentTime, PTable table, List<byte[]> tableNamesToDelete, List<SharedTableState> sharedTablesToDelete) {
@@ -361,8 +361,8 @@ public abstract class MetaDataProtocol extends MetaDataService {
             return viewIndexId;
         }
 
-      public PDataType getViewIndexType() {
-          return viewIndexType;
+      public PDataType getViewIndexIdType() {
+          return viewIndexIdType;
       }
 
         public static MetaDataMutationResult constructFromProto(MetaDataResponse proto) {
@@ -410,8 +410,8 @@ public abstract class MetaDataProtocol extends MetaDataService {
                result.viewIndexId = proto.getViewIndexId();
           }
 
-          result.viewIndexType = proto.hasViewIndexType()
-                    ? PDataType.fromTypeId(proto.getViewIndexType())
+          result.viewIndexIdType = proto.hasViewIndexIdType()
+                    ? PDataType.fromTypeId(proto.getViewIndexIdType())
                     : MetaDataUtil.getLegacyViewIndexIdDataType();
           return result;
         }
@@ -453,7 +453,7 @@ public abstract class MetaDataProtocol extends MetaDataService {
                 sharedTableStateBuilder.setSchemaName(ByteStringer.wrap(sharedTableState.getSchemaName().getBytes()));
                 sharedTableStateBuilder.setTableName(ByteStringer.wrap(sharedTableState.getTableName().getBytes()));
                 sharedTableStateBuilder.setViewIndexId(sharedTableState.getViewIndexId());
-                sharedTableStateBuilder.setViewIndexType(sharedTableState.viewIndexType.getSqlType());
+                sharedTableStateBuilder.setViewIndexIdType(sharedTableState.viewIndexIdType.getSqlType());
                 builder.addSharedTablesToDelete(sharedTableStateBuilder.build());
               }
             }
@@ -464,9 +464,9 @@ public abstract class MetaDataProtocol extends MetaDataService {
             if (result.getViewIndexId() != null) {
                 builder.setViewIndexId(result.getViewIndexId());
             }
-            builder.setViewIndexType(result.getViewIndexType() == null
+            builder.setViewIndexIdType(result.getViewIndexIdType() == null
 					  ? MetaDataUtil.getLegacyViewIndexIdDataType().getSqlType()
-					  : result.getViewIndexType().getSqlType());
+					  : result.getViewIndexIdType().getSqlType());
           }
           return builder.build();
         }
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/MetaDataProtos.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/MetaDataProtos.java
index 360dd77..d8152a9 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/MetaDataProtos.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/MetaDataProtos.java
@@ -379,15 +379,15 @@ public final class MetaDataProtos {
      */
     long getViewIndexId();
 
-    // optional int32 viewIndexType = 7 [default = 5];
+    // optional int32 viewIndexIdType = 7 [default = 5];
     /**
-     * <code>optional int32 viewIndexType = 7 [default = 5];</code>
+     * <code>optional int32 viewIndexIdType = 7 [default = 5];</code>
      */
-    boolean hasViewIndexType();
+    boolean hasViewIndexIdType();
     /**
-     * <code>optional int32 viewIndexType = 7 [default = 5];</code>
+     * <code>optional int32 viewIndexIdType = 7 [default = 5];</code>
      */
-    int getViewIndexType();
+    int getViewIndexIdType();
   }
   /**
    * Protobuf type {@code SharedTableState}
@@ -478,7 +478,7 @@ public final class MetaDataProtos {
             }
             case 56: {
               bitField0_ |= 0x00000010;
-              viewIndexType_ = input.readInt32();
+              viewIndexIdType_ = input.readInt32();
               break;
             }
           }
@@ -650,20 +650,20 @@ public final class MetaDataProtos {
       return viewIndexId_;
     }
 
-    // optional int32 viewIndexType = 7 [default = 5];
-    public static final int VIEWINDEXTYPE_FIELD_NUMBER = 7;
-    private int viewIndexType_;
+    // optional int32 viewIndexIdType = 7 [default = 5];
+    public static final int VIEWINDEXIDTYPE_FIELD_NUMBER = 7;
+    private int viewIndexIdType_;
     /**
-     * <code>optional int32 viewIndexType = 7 [default = 5];</code>
+     * <code>optional int32 viewIndexIdType = 7 [default = 5];</code>
      */
-    public boolean hasViewIndexType() {
+    public boolean hasViewIndexIdType() {
       return ((bitField0_ & 0x00000010) == 0x00000010);
     }
     /**
-     * <code>optional int32 viewIndexType = 7 [default = 5];</code>
+     * <code>optional int32 viewIndexIdType = 7 [default = 5];</code>
      */
-    public int getViewIndexType() {
-      return viewIndexType_;
+    public int getViewIndexIdType() {
+      return viewIndexIdType_;
     }
 
     private void initFields() {
@@ -673,7 +673,7 @@ public final class MetaDataProtos {
       columns_ = java.util.Collections.emptyList();
       physicalNames_ = java.util.Collections.emptyList();
       viewIndexId_ = 0L;
-      viewIndexType_ = 5;
+      viewIndexIdType_ = 5;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -724,7 +724,7 @@ public final class MetaDataProtos {
         output.writeInt64(6, viewIndexId_);
       }
       if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        output.writeInt32(7, viewIndexType_);
+        output.writeInt32(7, viewIndexIdType_);
       }
       getUnknownFields().writeTo(output);
     }
@@ -766,7 +766,7 @@ public final class MetaDataProtos {
       }
       if (((bitField0_ & 0x00000010) == 0x00000010)) {
         size += com.google.protobuf.CodedOutputStream
-          .computeInt32Size(7, viewIndexType_);
+          .computeInt32Size(7, viewIndexIdType_);
       }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
@@ -815,10 +815,10 @@ public final class MetaDataProtos {
         result = result && (getViewIndexId()
             == other.getViewIndexId());
       }
-      result = result && (hasViewIndexType() == other.hasViewIndexType());
-      if (hasViewIndexType()) {
-        result = result && (getViewIndexType()
-            == other.getViewIndexType());
+      result = result && (hasViewIndexIdType() == other.hasViewIndexIdType());
+      if (hasViewIndexIdType()) {
+        result = result && (getViewIndexIdType()
+            == other.getViewIndexIdType());
       }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
@@ -857,9 +857,9 @@ public final class MetaDataProtos {
         hash = (37 * hash) + VIEWINDEXID_FIELD_NUMBER;
         hash = (53 * hash) + hashLong(getViewIndexId());
       }
-      if (hasViewIndexType()) {
-        hash = (37 * hash) + VIEWINDEXTYPE_FIELD_NUMBER;
-        hash = (53 * hash) + getViewIndexType();
+      if (hasViewIndexIdType()) {
+        hash = (37 * hash) + VIEWINDEXIDTYPE_FIELD_NUMBER;
+        hash = (53 * hash) + getViewIndexIdType();
       }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
@@ -987,7 +987,7 @@ public final class MetaDataProtos {
         bitField0_ = (bitField0_ & ~0x00000010);
         viewIndexId_ = 0L;
         bitField0_ = (bitField0_ & ~0x00000020);
-        viewIndexType_ = 5;
+        viewIndexIdType_ = 5;
         bitField0_ = (bitField0_ & ~0x00000040);
         return this;
       }
@@ -1050,7 +1050,7 @@ public final class MetaDataProtos {
         if (((from_bitField0_ & 0x00000040) == 0x00000040)) {
           to_bitField0_ |= 0x00000010;
         }
-        result.viewIndexType_ = viewIndexType_;
+        result.viewIndexIdType_ = viewIndexIdType_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -1115,8 +1115,8 @@ public final class MetaDataProtos {
         if (other.hasViewIndexId()) {
           setViewIndexId(other.getViewIndexId());
         }
-        if (other.hasViewIndexType()) {
-          setViewIndexType(other.getViewIndexType());
+        if (other.hasViewIndexIdType()) {
+          setViewIndexIdType(other.getViewIndexIdType());
         }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
@@ -1616,35 +1616,35 @@ public final class MetaDataProtos {
         return this;
       }
 
-      // optional int32 viewIndexType = 7 [default = 5];
-      private int viewIndexType_ = 5;
+      // optional int32 viewIndexIdType = 7 [default = 5];
+      private int viewIndexIdType_ = 5;
       /**
-       * <code>optional int32 viewIndexType = 7 [default = 5];</code>
+       * <code>optional int32 viewIndexIdType = 7 [default = 5];</code>
        */
-      public boolean hasViewIndexType() {
+      public boolean hasViewIndexIdType() {
         return ((bitField0_ & 0x00000040) == 0x00000040);
       }
       /**
-       * <code>optional int32 viewIndexType = 7 [default = 5];</code>
+       * <code>optional int32 viewIndexIdType = 7 [default = 5];</code>
        */
-      public int getViewIndexType() {
-        return viewIndexType_;
+      public int getViewIndexIdType() {
+        return viewIndexIdType_;
       }
       /**
-       * <code>optional int32 viewIndexType = 7 [default = 5];</code>
+       * <code>optional int32 viewIndexIdType = 7 [default = 5];</code>
        */
-      public Builder setViewIndexType(int value) {
+      public Builder setViewIndexIdType(int value) {
         bitField0_ |= 0x00000040;
-        viewIndexType_ = value;
+        viewIndexIdType_ = value;
         onChanged();
         return this;
       }
       /**
-       * <code>optional int32 viewIndexType = 7 [default = 5];</code>
+       * <code>optional int32 viewIndexIdType = 7 [default = 5];</code>
        */
-      public Builder clearViewIndexType() {
+      public Builder clearViewIndexIdType() {
         bitField0_ = (bitField0_ & ~0x00000040);
-        viewIndexType_ = 5;
+        viewIndexIdType_ = 5;
         onChanged();
         return this;
       }
@@ -1825,15 +1825,15 @@ public final class MetaDataProtos {
      */
     long getViewIndexId();
 
-    // optional int32 viewIndexType = 13 [default = 5];
+    // optional int32 viewIndexIdType = 13 [default = 5];
     /**
-     * <code>optional int32 viewIndexType = 13 [default = 5];</code>
+     * <code>optional int32 viewIndexIdType = 13 [default = 5];</code>
      */
-    boolean hasViewIndexType();
+    boolean hasViewIndexIdType();
     /**
-     * <code>optional int32 viewIndexType = 13 [default = 5];</code>
+     * <code>optional int32 viewIndexIdType = 13 [default = 5];</code>
      */
-    int getViewIndexType();
+    int getViewIndexIdType();
   }
   /**
    * Protobuf type {@code MetaDataResponse}
@@ -1979,7 +1979,7 @@ public final class MetaDataProtos {
             }
             case 104: {
               bitField0_ |= 0x00000200;
-              viewIndexType_ = input.readInt32();
+              viewIndexIdType_ = input.readInt32();
               break;
             }
           }
@@ -2282,20 +2282,20 @@ public final class MetaDataProtos {
       return viewIndexId_;
     }
 
-    // optional int32 viewIndexType = 13 [default = 5];
-    public static final int VIEWINDEXTYPE_FIELD_NUMBER = 13;
-    private int viewIndexType_;
+    // optional int32 viewIndexIdType = 13 [default = 5];
+    public static final int VIEWINDEXIDTYPE_FIELD_NUMBER = 13;
+    private int viewIndexIdType_;
     /**
-     * <code>optional int32 viewIndexType = 13 [default = 5];</code>
+     * <code>optional int32 viewIndexIdType = 13 [default = 5];</code>
      */
-    public boolean hasViewIndexType() {
+    public boolean hasViewIndexIdType() {
       return ((bitField0_ & 0x00000200) == 0x00000200);
     }
     /**
-     * <code>optional int32 viewIndexType = 13 [default = 5];</code>
+     * <code>optional int32 viewIndexIdType = 13 [default = 5];</code>
      */
-    public int getViewIndexType() {
-      return viewIndexType_;
+    public int getViewIndexIdType() {
+      return viewIndexIdType_;
     }
 
     private void initFields() {
@@ -2311,7 +2311,7 @@ public final class MetaDataProtos {
       schema_ = org.apache.phoenix.coprocessor.generated.PSchemaProtos.PSchema.getDefaultInstance();
       autoPartitionNum_ = 0L;
       viewIndexId_ = 0L;
-      viewIndexType_ = 5;
+      viewIndexIdType_ = 5;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -2386,7 +2386,7 @@ public final class MetaDataProtos {
         output.writeInt64(12, viewIndexId_);
       }
       if (((bitField0_ & 0x00000200) == 0x00000200)) {
-        output.writeInt32(13, viewIndexType_);
+        output.writeInt32(13, viewIndexIdType_);
       }
       getUnknownFields().writeTo(output);
     }
@@ -2452,7 +2452,7 @@ public final class MetaDataProtos {
       }
       if (((bitField0_ & 0x00000200) == 0x00000200)) {
         size += com.google.protobuf.CodedOutputStream
-          .computeInt32Size(13, viewIndexType_);
+          .computeInt32Size(13, viewIndexIdType_);
       }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
@@ -2528,10 +2528,10 @@ public final class MetaDataProtos {
         result = result && (getViewIndexId()
             == other.getViewIndexId());
       }
-      result = result && (hasViewIndexType() == other.hasViewIndexType());
-      if (hasViewIndexType()) {
-        result = result && (getViewIndexType()
-            == other.getViewIndexType());
+      result = result && (hasViewIndexIdType() == other.hasViewIndexIdType());
+      if (hasViewIndexIdType()) {
+        result = result && (getViewIndexIdType()
+            == other.getViewIndexIdType());
       }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
@@ -2594,9 +2594,9 @@ public final class MetaDataProtos {
         hash = (37 * hash) + VIEWINDEXID_FIELD_NUMBER;
         hash = (53 * hash) + hashLong(getViewIndexId());
       }
-      if (hasViewIndexType()) {
-        hash = (37 * hash) + VIEWINDEXTYPE_FIELD_NUMBER;
-        hash = (53 * hash) + getViewIndexType();
+      if (hasViewIndexIdType()) {
+        hash = (37 * hash) + VIEWINDEXIDTYPE_FIELD_NUMBER;
+        hash = (53 * hash) + getViewIndexIdType();
       }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
@@ -2751,7 +2751,7 @@ public final class MetaDataProtos {
         bitField0_ = (bitField0_ & ~0x00000400);
         viewIndexId_ = 0L;
         bitField0_ = (bitField0_ & ~0x00000800);
-        viewIndexType_ = 5;
+        viewIndexIdType_ = 5;
         bitField0_ = (bitField0_ & ~0x00001000);
         return this;
       }
@@ -2851,7 +2851,7 @@ public final class MetaDataProtos {
         if (((from_bitField0_ & 0x00001000) == 0x00001000)) {
           to_bitField0_ |= 0x00000200;
         }
-        result.viewIndexType_ = viewIndexType_;
+        result.viewIndexIdType_ = viewIndexIdType_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -2957,8 +2957,8 @@ public final class MetaDataProtos {
         if (other.hasViewIndexId()) {
           setViewIndexId(other.getViewIndexId());
         }
-        if (other.hasViewIndexType()) {
-          setViewIndexType(other.getViewIndexType());
+        if (other.hasViewIndexIdType()) {
+          setViewIndexIdType(other.getViewIndexIdType());
         }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
@@ -4040,35 +4040,35 @@ public final class MetaDataProtos {
         return this;
       }
 
-      // optional int32 viewIndexType = 13 [default = 5];
-      private int viewIndexType_ = 5;
+      // optional int32 viewIndexIdType = 13 [default = 5];
+      private int viewIndexIdType_ = 5;
       /**
-       * <code>optional int32 viewIndexType = 13 [default = 5];</code>
+       * <code>optional int32 viewIndexIdType = 13 [default = 5];</code>
        */
-      public boolean hasViewIndexType() {
+      public boolean hasViewIndexIdType() {
         return ((bitField0_ & 0x00001000) == 0x00001000);
       }
       /**
-       * <code>optional int32 viewIndexType = 13 [default = 5];</code>
+       * <code>optional int32 viewIndexIdType = 13 [default = 5];</code>
        */
-      public int getViewIndexType() {
-        return viewIndexType_;
+      public int getViewIndexIdType() {
+        return viewIndexIdType_;
       }
       /**
-       * <code>optional int32 viewIndexType = 13 [default = 5];</code>
+       * <code>optional int32 viewIndexIdType = 13 [default = 5];</code>
        */
-      public Builder setViewIndexType(int value) {
+      public Builder setViewIndexIdType(int value) {
         bitField0_ |= 0x00001000;
-        viewIndexType_ = value;
+        viewIndexIdType_ = value;
         onChanged();
         return this;
       }
       /**
-       * <code>optional int32 viewIndexType = 13 [default = 5];</code>
+       * <code>optional int32 viewIndexIdType = 13 [default = 5];</code>
        */
-      public Builder clearViewIndexType() {
+      public Builder clearViewIndexIdType() {
         bitField0_ = (bitField0_ & ~0x00001000);
-        viewIndexType_ = 5;
+        viewIndexIdType_ = 5;
         onChanged();
         return this;
       }
@@ -17827,105 +17827,105 @@ public final class MetaDataProtos {
   static {
     java.lang.String[] descriptorData = {
       "\n\025MetaDataService.proto\032\014PTable.proto\032\017P" +
-      "Function.proto\032\rPSchema.proto\"\254\001\n\020Shared" +
+      "Function.proto\032\rPSchema.proto\"\256\001\n\020Shared" +
       "TableState\022\020\n\010tenantId\030\001 \001(\014\022\022\n\nschemaNa" +
       "me\030\002 \002(\014\022\021\n\ttableName\030\003 \002(\014\022\031\n\007columns\030\004" +
       " \003(\0132\010.PColumn\022\025\n\rphysicalNames\030\005 \003(\014\022\023\n" +
-      "\013viewIndexId\030\006 \002(\003\022\030\n\rviewIndexType\030\007 \001(" +
-      "\005:\0015\"\353\002\n\020MetaDataResponse\022!\n\nreturnCode\030" +
-      "\001 \001(\0162\r.MutationCode\022\024\n\014mutationTime\030\002 \001" +
-      "(\003\022\026\n\005table\030\003 \001(\0132\007.PTable\022\026\n\016tablesToDe" +
-      "lete\030\004 \003(\014\022\022\n\ncolumnName\030\005 \001(\014\022\022\n\nfamily",
-      "Name\030\006 \001(\014\022\024\n\014functionName\030\007 \001(\014\022\034\n\010func" +
-      "tion\030\010 \003(\0132\n.PFunction\022/\n\024sharedTablesTo" +
-      "Delete\030\t \003(\0132\021.SharedTableState\022\030\n\006schem" +
-      "a\030\n \001(\0132\010.PSchema\022\030\n\020autoPartitionNum\030\013 " +
-      "\001(\003\022\023\n\013viewIndexId\030\014 \001(\003\022\030\n\rviewIndexTyp" +
-      "e\030\r \001(\005:\0015\"\364\001\n\017GetTableRequest\022\020\n\010tenant" +
-      "Id\030\001 \002(\014\022\022\n\nschemaName\030\002 \002(\014\022\021\n\ttableNam" +
-      "e\030\003 \002(\014\022\026\n\016tableTimestamp\030\004 \002(\003\022\027\n\017clien" +
-      "tTimestamp\030\005 \002(\003\022\025\n\rclientVersion\030\006 \001(\005\022" +
-      "\037\n\027skipAddingParentColumns\030\007 \001(\010\022\031\n\021skip",
-      "AddingIndexes\030\010 \001(\010\022$\n\023lockedAncestorTab" +
-      "le\030\t \001(\0132\007.PTable\"\212\001\n\023GetFunctionsReques" +
-      "t\022\020\n\010tenantId\030\001 \002(\014\022\025\n\rfunctionNames\030\002 \003" +
-      "(\014\022\032\n\022functionTimestamps\030\003 \003(\003\022\027\n\017client" +
-      "Timestamp\030\004 \002(\003\022\025\n\rclientVersion\030\005 \001(\005\"V" +
-      "\n\020GetSchemaRequest\022\022\n\nschemaName\030\001 \002(\t\022\027" +
-      "\n\017clientTimestamp\030\002 \002(\003\022\025\n\rclientVersion" +
-      "\030\003 \002(\005\"d\n\022CreateTableRequest\022\036\n\026tableMet" +
+      "\013viewIndexId\030\006 \002(\003\022\032\n\017viewIndexIdType\030\007 " +
+      "\001(\005:\0015\"\355\002\n\020MetaDataResponse\022!\n\nreturnCod" +
+      "e\030\001 \001(\0162\r.MutationCode\022\024\n\014mutationTime\030\002" +
+      " \001(\003\022\026\n\005table\030\003 \001(\0132\007.PTable\022\026\n\016tablesTo" +
+      "Delete\030\004 \003(\014\022\022\n\ncolumnName\030\005 \001(\014\022\022\n\nfami",
+      "lyName\030\006 \001(\014\022\024\n\014functionName\030\007 \001(\014\022\034\n\010fu" +
+      "nction\030\010 \003(\0132\n.PFunction\022/\n\024sharedTables" +
+      "ToDelete\030\t \003(\0132\021.SharedTableState\022\030\n\006sch" +
+      "ema\030\n \001(\0132\010.PSchema\022\030\n\020autoPartitionNum\030" +
+      "\013 \001(\003\022\023\n\013viewIndexId\030\014 \001(\003\022\032\n\017viewIndexI" +
+      "dType\030\r \001(\005:\0015\"\364\001\n\017GetTableRequest\022\020\n\010te" +
+      "nantId\030\001 \002(\014\022\022\n\nschemaName\030\002 \002(\014\022\021\n\ttabl" +
+      "eName\030\003 \002(\014\022\026\n\016tableTimestamp\030\004 \002(\003\022\027\n\017c" +
+      "lientTimestamp\030\005 \002(\003\022\025\n\rclientVersion\030\006 " +
+      "\001(\005\022\037\n\027skipAddingParentColumns\030\007 \001(\010\022\031\n\021",
+      "skipAddingIndexes\030\010 \001(\010\022$\n\023lockedAncesto" +
+      "rTable\030\t \001(\0132\007.PTable\"\212\001\n\023GetFunctionsRe" +
+      "quest\022\020\n\010tenantId\030\001 \002(\014\022\025\n\rfunctionNames" +
+      "\030\002 \003(\014\022\032\n\022functionTimestamps\030\003 \003(\003\022\027\n\017cl" +
+      "ientTimestamp\030\004 \002(\003\022\025\n\rclientVersion\030\005 \001" +
+      "(\005\"V\n\020GetSchemaRequest\022\022\n\nschemaName\030\001 \002" +
+      "(\t\022\027\n\017clientTimestamp\030\002 \002(\003\022\025\n\rclientVer" +
+      "sion\030\003 \002(\005\"d\n\022CreateTableRequest\022\036\n\026tabl" +
+      "eMetadataMutations\030\001 \003(\014\022\025\n\rclientVersio" +
+      "n\030\002 \001(\005\022\027\n\017allocateIndexId\030\003 \001(\010\"r\n\025Crea",
+      "teFunctionRequest\022\036\n\026tableMetadataMutati" +
+      "ons\030\001 \003(\014\022\021\n\ttemporary\030\002 \002(\010\022\017\n\007replace\030" +
+      "\003 \001(\010\022\025\n\rclientVersion\030\004 \001(\005\"`\n\023CreateSc" +
+      "hemaRequest\022\036\n\026tableMetadataMutations\030\001 " +
+      "\003(\014\022\022\n\nschemaName\030\002 \002(\t\022\025\n\rclientVersion" +
+      "\030\003 \002(\005\"\216\001\n\020DropTableRequest\022\036\n\026tableMeta" +
+      "dataMutations\030\001 \003(\014\022\021\n\ttableType\030\002 \002(\t\022\017" +
+      "\n\007cascade\030\003 \001(\010\022\025\n\rclientVersion\030\004 \001(\005\022\037" +
+      "\n\027skipAddingParentColumns\030\005 \001(\010\"_\n\021DropS" +
+      "chemaRequest\022\037\n\027schemaMetadataMutations\030",
+      "\001 \003(\014\022\022\n\nschemaName\030\002 \002(\t\022\025\n\rclientVersi" +
+      "on\030\003 \002(\005\"I\n\020AddColumnRequest\022\036\n\026tableMet" +
       "adataMutations\030\001 \003(\014\022\025\n\rclientVersion\030\002 " +
-      "\001(\005\022\027\n\017allocateIndexId\030\003 \001(\010\"r\n\025CreateFu",
-      "nctionRequest\022\036\n\026tableMetadataMutations\030" +
-      "\001 \003(\014\022\021\n\ttemporary\030\002 \002(\010\022\017\n\007replace\030\003 \001(" +
-      "\010\022\025\n\rclientVersion\030\004 \001(\005\"`\n\023CreateSchema" +
-      "Request\022\036\n\026tableMetadataMutations\030\001 \003(\014\022" +
-      "\022\n\nschemaName\030\002 \002(\t\022\025\n\rclientVersion\030\003 \002" +
-      "(\005\"\216\001\n\020DropTableRequest\022\036\n\026tableMetadata" +
-      "Mutations\030\001 \003(\014\022\021\n\ttableType\030\002 \002(\t\022\017\n\007ca" +
-      "scade\030\003 \001(\010\022\025\n\rclientVersion\030\004 \001(\005\022\037\n\027sk" +
-      "ipAddingParentColumns\030\005 \001(\010\"_\n\021DropSchem" +
-      "aRequest\022\037\n\027schemaMetadataMutations\030\001 \003(",
-      "\014\022\022\n\nschemaName\030\002 \002(\t\022\025\n\rclientVersion\030\003" +
-      " \002(\005\"I\n\020AddColumnRequest\022\036\n\026tableMetadat" +
+      "\001(\005\"J\n\021DropColumnRequest\022\036\n\026tableMetadat" +
       "aMutations\030\001 \003(\014\022\025\n\rclientVersion\030\002 \001(\005\"" +
-      "J\n\021DropColumnRequest\022\036\n\026tableMetadataMut" +
-      "ations\030\001 \003(\014\022\025\n\rclientVersion\030\002 \001(\005\"^\n\023D" +
-      "ropFunctionRequest\022\036\n\026tableMetadataMutat" +
-      "ions\030\001 \003(\014\022\020\n\010ifExists\030\002 \001(\010\022\025\n\rclientVe" +
-      "rsion\030\003 \001(\005\"P\n\027UpdateIndexStateRequest\022\036" +
-      "\n\026tableMetadataMutations\030\001 \003(\014\022\025\n\rclient" +
-      "Version\030\002 \001(\005\"*\n\021ClearCacheRequest\022\025\n\rcl",
-      "ientVersion\030\001 \001(\005\"*\n\022ClearCacheResponse\022" +
-      "\024\n\014unfreedBytes\030\001 \001(\003\"*\n\021GetVersionReque" +
-      "st\022\025\n\rclientVersion\030\001 \001(\005\"E\n\022GetVersionR" +
-      "esponse\022\017\n\007version\030\001 \002(\003\022\036\n\026systemCatalo" +
-      "gTimestamp\030\002 \001(\003\"\205\001\n\032ClearTableFromCache" +
-      "Request\022\020\n\010tenantId\030\001 \002(\014\022\022\n\nschemaName\030" +
-      "\002 \002(\014\022\021\n\ttableName\030\003 \002(\014\022\027\n\017clientTimest" +
-      "amp\030\004 \002(\003\022\025\n\rclientVersion\030\005 \001(\005\"\035\n\033Clea" +
-      "rTableFromCacheResponse*\271\005\n\014MutationCode" +
-      "\022\030\n\024TABLE_ALREADY_EXISTS\020\000\022\023\n\017TABLE_NOT_",
-      "FOUND\020\001\022\024\n\020COLUMN_NOT_FOUND\020\002\022\031\n\025COLUMN_" +
-      "ALREADY_EXISTS\020\003\022\035\n\031CONCURRENT_TABLE_MUT" +
-      "ATION\020\004\022\027\n\023TABLE_NOT_IN_REGION\020\005\022\025\n\021NEWE" +
-      "R_TABLE_FOUND\020\006\022\034\n\030UNALLOWED_TABLE_MUTAT" +
-      "ION\020\007\022\021\n\rNO_PK_COLUMNS\020\010\022\032\n\026PARENT_TABLE" +
-      "_NOT_FOUND\020\t\022\033\n\027FUNCTION_ALREADY_EXISTS\020" +
-      "\n\022\026\n\022FUNCTION_NOT_FOUND\020\013\022\030\n\024NEWER_FUNCT" +
-      "ION_FOUND\020\014\022\032\n\026FUNCTION_NOT_IN_REGION\020\r\022" +
-      "\031\n\025SCHEMA_ALREADY_EXISTS\020\016\022\026\n\022NEWER_SCHE" +
-      "MA_FOUND\020\017\022\024\n\020SCHEMA_NOT_FOUND\020\020\022\030\n\024SCHE",
-      "MA_NOT_IN_REGION\020\021\022\032\n\026TABLES_EXIST_ON_SC" +
-      "HEMA\020\022\022\035\n\031UNALLOWED_SCHEMA_MUTATION\020\023\022%\n" +
-      "!AUTO_PARTITION_SEQUENCE_NOT_FOUND\020\024\022#\n\037" +
-      "CANNOT_COERCE_AUTO_PARTITION_ID\020\025\022\024\n\020TOO" +
-      "_MANY_INDEXES\020\026\022\037\n\033UNABLE_TO_CREATE_CHIL" +
-      "D_LINK\020\027\022!\n\035UNABLE_TO_UPDATE_PARENT_TABL" +
-      "E\020\0302\345\006\n\017MetaDataService\022/\n\010getTable\022\020.Ge" +
-      "tTableRequest\032\021.MetaDataResponse\0227\n\014getF" +
-      "unctions\022\024.GetFunctionsRequest\032\021.MetaDat" +
-      "aResponse\0221\n\tgetSchema\022\021.GetSchemaReques",
-      "t\032\021.MetaDataResponse\0225\n\013createTable\022\023.Cr" +
-      "eateTableRequest\032\021.MetaDataResponse\022;\n\016c" +
-      "reateFunction\022\026.CreateFunctionRequest\032\021." +
-      "MetaDataResponse\0227\n\014createSchema\022\024.Creat" +
-      "eSchemaRequest\032\021.MetaDataResponse\0221\n\tdro" +
-      "pTable\022\021.DropTableRequest\032\021.MetaDataResp" +
-      "onse\0223\n\ndropSchema\022\022.DropSchemaRequest\032\021" +
-      ".MetaDataResponse\0227\n\014dropFunction\022\024.Drop" +
-      "FunctionRequest\032\021.MetaDataResponse\0221\n\tad" +
-      "dColumn\022\021.AddColumnRequest\032\021.MetaDataRes",
-      "ponse\0223\n\ndropColumn\022\022.DropColumnRequest\032" +
-      "\021.MetaDataResponse\022?\n\020updateIndexState\022\030" +
-      ".UpdateIndexStateRequest\032\021.MetaDataRespo" +
-      "nse\0225\n\nclearCache\022\022.ClearCacheRequest\032\023." +
-      "ClearCacheResponse\0225\n\ngetVersion\022\022.GetVe" +
-      "rsionRequest\032\023.GetVersionResponse\022P\n\023cle" +
-      "arTableFromCache\022\033.ClearTableFromCacheRe" +
-      "quest\032\034.ClearTableFromCacheResponseBB\n(o" +
-      "rg.apache.phoenix.coprocessor.generatedB" +
-      "\016MetaDataProtosH\001\210\001\001\240\001\001"
+      "^\n\023DropFunctionRequest\022\036\n\026tableMetadataM" +
+      "utations\030\001 \003(\014\022\020\n\010ifExists\030\002 \001(\010\022\025\n\rclie" +
+      "ntVersion\030\003 \001(\005\"P\n\027UpdateIndexStateReque" +
+      "st\022\036\n\026tableMetadataMutations\030\001 \003(\014\022\025\n\rcl" +
+      "ientVersion\030\002 \001(\005\"*\n\021ClearCacheRequest\022\025",
+      "\n\rclientVersion\030\001 \001(\005\"*\n\022ClearCacheRespo" +
+      "nse\022\024\n\014unfreedBytes\030\001 \001(\003\"*\n\021GetVersionR" +
+      "equest\022\025\n\rclientVersion\030\001 \001(\005\"E\n\022GetVers" +
+      "ionResponse\022\017\n\007version\030\001 \002(\003\022\036\n\026systemCa" +
+      "talogTimestamp\030\002 \001(\003\"\205\001\n\032ClearTableFromC" +
+      "acheRequest\022\020\n\010tenantId\030\001 \002(\014\022\022\n\nschemaN" +
+      "ame\030\002 \002(\014\022\021\n\ttableName\030\003 \002(\014\022\027\n\017clientTi" +
+      "mestamp\030\004 \002(\003\022\025\n\rclientVersion\030\005 \001(\005\"\035\n\033" +
+      "ClearTableFromCacheResponse*\271\005\n\014Mutation" +
+      "Code\022\030\n\024TABLE_ALREADY_EXISTS\020\000\022\023\n\017TABLE_",
+      "NOT_FOUND\020\001\022\024\n\020COLUMN_NOT_FOUND\020\002\022\031\n\025COL" +
+      "UMN_ALREADY_EXISTS\020\003\022\035\n\031CONCURRENT_TABLE" +
+      "_MUTATION\020\004\022\027\n\023TABLE_NOT_IN_REGION\020\005\022\025\n\021" +
+      "NEWER_TABLE_FOUND\020\006\022\034\n\030UNALLOWED_TABLE_M" +
+      "UTATION\020\007\022\021\n\rNO_PK_COLUMNS\020\010\022\032\n\026PARENT_T" +
+      "ABLE_NOT_FOUND\020\t\022\033\n\027FUNCTION_ALREADY_EXI" +
+      "STS\020\n\022\026\n\022FUNCTION_NOT_FOUND\020\013\022\030\n\024NEWER_F" +
+      "UNCTION_FOUND\020\014\022\032\n\026FUNCTION_NOT_IN_REGIO" +
+      "N\020\r\022\031\n\025SCHEMA_ALREADY_EXISTS\020\016\022\026\n\022NEWER_" +
+      "SCHEMA_FOUND\020\017\022\024\n\020SCHEMA_NOT_FOUND\020\020\022\030\n\024",
+      "SCHEMA_NOT_IN_REGION\020\021\022\032\n\026TABLES_EXIST_O" +
+      "N_SCHEMA\020\022\022\035\n\031UNALLOWED_SCHEMA_MUTATION\020" +
+      "\023\022%\n!AUTO_PARTITION_SEQUENCE_NOT_FOUND\020\024" +
+      "\022#\n\037CANNOT_COERCE_AUTO_PARTITION_ID\020\025\022\024\n" +
+      "\020TOO_MANY_INDEXES\020\026\022\037\n\033UNABLE_TO_CREATE_" +
+      "CHILD_LINK\020\027\022!\n\035UNABLE_TO_UPDATE_PARENT_" +
+      "TABLE\020\0302\345\006\n\017MetaDataService\022/\n\010getTable\022" +
+      "\020.GetTableRequest\032\021.MetaDataResponse\0227\n\014" +
+      "getFunctions\022\024.GetFunctionsRequest\032\021.Met" +
+      "aDataResponse\0221\n\tgetSchema\022\021.GetSchemaRe",
+      "quest\032\021.MetaDataResponse\0225\n\013createTable\022" +
+      "\023.CreateTableRequest\032\021.MetaDataResponse\022" +
+      ";\n\016createFunction\022\026.CreateFunctionReques" +
+      "t\032\021.MetaDataResponse\0227\n\014createSchema\022\024.C" +
+      "reateSchemaRequest\032\021.MetaDataResponse\0221\n" +
+      "\tdropTable\022\021.DropTableRequest\032\021.MetaData" +
+      "Response\0223\n\ndropSchema\022\022.DropSchemaReque" +
+      "st\032\021.MetaDataResponse\0227\n\014dropFunction\022\024." +
+      "DropFunctionRequest\032\021.MetaDataResponse\0221" +
+      "\n\taddColumn\022\021.AddColumnRequest\032\021.MetaDat",
+      "aResponse\0223\n\ndropColumn\022\022.DropColumnRequ" +
+      "est\032\021.MetaDataResponse\022?\n\020updateIndexSta" +
+      "te\022\030.UpdateIndexStateRequest\032\021.MetaDataR" +
+      "esponse\0225\n\nclearCache\022\022.ClearCacheReques" +
+      "t\032\023.ClearCacheResponse\0225\n\ngetVersion\022\022.G" +
+      "etVersionRequest\032\023.GetVersionResponse\022P\n" +
+      "\023clearTableFromCache\022\033.ClearTableFromCac" +
+      "heRequest\032\034.ClearTableFromCacheResponseB" +
+      "B\n(org.apache.phoenix.coprocessor.genera" +
+      "tedB\016MetaDataProtosH\001\210\001\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -17937,13 +17937,13 @@ public final class MetaDataProtos {
           internal_static_SharedTableState_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_SharedTableState_descriptor,
-              new java.lang.String[] { "TenantId", "SchemaName", "TableName", "Columns", "PhysicalNames", "ViewIndexId", "ViewIndexType", });
+              new java.lang.String[] { "TenantId", "SchemaName", "TableName", "Columns", "PhysicalNames", "ViewIndexId", "ViewIndexIdType", });
           internal_static_MetaDataResponse_descriptor =
             getDescriptor().getMessageTypes().get(1);
           internal_static_MetaDataResponse_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_MetaDataResponse_descriptor,
-              new java.lang.String[] { "ReturnCode", "MutationTime", "Table", "TablesToDelete", "ColumnName", "FamilyName", "FunctionName", "Function", "SharedTablesToDelete", "Schema", "AutoPartitionNum", "ViewIndexId", "ViewIndexType", });
+              new java.lang.String[] { "ReturnCode", "MutationTime", "Table", "TablesToDelete", "ColumnName", "FamilyName", "FunctionName", "Function", "SharedTablesToDelete", "Schema", "AutoPartitionNum", "ViewIndexId", "ViewIndexIdType", });
           internal_static_GetTableRequest_descriptor =
             getDescriptor().getMessageTypes().get(2);
           internal_static_GetTableRequest_fieldAccessorTable = new
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PTableProtos.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PTableProtos.java
index 938ae1f..4a3b43f 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PTableProtos.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PTableProtos.java
@@ -3695,15 +3695,15 @@ public final class PTableProtos {
      */
     int getTransactionProvider();
 
-    // optional int32 viewIndexType = 39 [default = 5];
+    // optional int32 viewIndexIdType = 39 [default = 5];
     /**
-     * <code>optional int32 viewIndexType = 39 [default = 5];</code>
+     * <code>optional int32 viewIndexIdType = 39 [default = 5];</code>
      */
-    boolean hasViewIndexType();
+    boolean hasViewIndexIdType();
     /**
-     * <code>optional int32 viewIndexType = 39 [default = 5];</code>
+     * <code>optional int32 viewIndexIdType = 39 [default = 5];</code>
      */
-    int getViewIndexType();
+    int getViewIndexIdType();
   }
   /**
    * Protobuf type {@code PTable}
@@ -3962,7 +3962,7 @@ public final class PTableProtos {
             }
             case 312: {
               bitField1_ |= 0x00000002;
-              viewIndexType_ = input.readInt32();
+              viewIndexIdType_ = input.readInt32();
               break;
             }
           }
@@ -4745,20 +4745,20 @@ public final class PTableProtos {
       return transactionProvider_;
     }
 
-    // optional int32 viewIndexType = 39 [default = 5];
-    public static final int VIEWINDEXTYPE_FIELD_NUMBER = 39;
-    private int viewIndexType_;
+    // optional int32 viewIndexIdType = 39 [default = 5];
+    public static final int VIEWINDEXIDTYPE_FIELD_NUMBER = 39;
+    private int viewIndexIdType_;
     /**
-     * <code>optional int32 viewIndexType = 39 [default = 5];</code>
+     * <code>optional int32 viewIndexIdType = 39 [default = 5];</code>
      */
-    public boolean hasViewIndexType() {
+    public boolean hasViewIndexIdType() {
       return ((bitField1_ & 0x00000002) == 0x00000002);
     }
     /**
-     * <code>optional int32 viewIndexType = 39 [default = 5];</code>
+     * <code>optional int32 viewIndexIdType = 39 [default = 5];</code>
      */
-    public int getViewIndexType() {
-      return viewIndexType_;
+    public int getViewIndexIdType() {
+      return viewIndexIdType_;
     }
 
     private void initFields() {
@@ -4799,7 +4799,7 @@ public final class PTableProtos {
       encodedCQCounters_ = java.util.Collections.emptyList();
       useStatsForParallelization_ = false;
       transactionProvider_ = 0;
-      viewIndexType_ = 5;
+      viewIndexIdType_ = 5;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -4979,7 +4979,7 @@ public final class PTableProtos {
         output.writeInt32(38, transactionProvider_);
       }
       if (((bitField1_ & 0x00000002) == 0x00000002)) {
-        output.writeInt32(39, viewIndexType_);
+        output.writeInt32(39, viewIndexIdType_);
       }
       getUnknownFields().writeTo(output);
     }
@@ -5145,7 +5145,7 @@ public final class PTableProtos {
       }
       if (((bitField1_ & 0x00000002) == 0x00000002)) {
         size += com.google.protobuf.CodedOutputStream
-          .computeInt32Size(39, viewIndexType_);
+          .computeInt32Size(39, viewIndexIdType_);
       }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
@@ -5343,10 +5343,10 @@ public final class PTableProtos {
         result = result && (getTransactionProvider()
             == other.getTransactionProvider());
       }
-      result = result && (hasViewIndexType() == other.hasViewIndexType());
-      if (hasViewIndexType()) {
-        result = result && (getViewIndexType()
-            == other.getViewIndexType());
+      result = result && (hasViewIndexIdType() == other.hasViewIndexIdType());
+      if (hasViewIndexIdType()) {
+        result = result && (getViewIndexIdType()
+            == other.getViewIndexIdType());
       }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
@@ -5509,9 +5509,9 @@ public final class PTableProtos {
         hash = (37 * hash) + TRANSACTIONPROVIDER_FIELD_NUMBER;
         hash = (53 * hash) + getTransactionProvider();
       }
-      if (hasViewIndexType()) {
-        hash = (37 * hash) + VIEWINDEXTYPE_FIELD_NUMBER;
-        hash = (53 * hash) + getViewIndexType();
+      if (hasViewIndexIdType()) {
+        hash = (37 * hash) + VIEWINDEXIDTYPE_FIELD_NUMBER;
+        hash = (53 * hash) + getViewIndexIdType();
       }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
@@ -5711,7 +5711,7 @@ public final class PTableProtos {
         bitField1_ = (bitField1_ & ~0x00000008);
         transactionProvider_ = 0;
         bitField1_ = (bitField1_ & ~0x00000010);
-        viewIndexType_ = 5;
+        viewIndexIdType_ = 5;
         bitField1_ = (bitField1_ & ~0x00000020);
         return this;
       }
@@ -5910,7 +5910,7 @@ public final class PTableProtos {
         if (((from_bitField1_ & 0x00000020) == 0x00000020)) {
           to_bitField1_ |= 0x00000002;
         }
-        result.viewIndexType_ = viewIndexType_;
+        result.viewIndexIdType_ = viewIndexIdType_;
         result.bitField0_ = to_bitField0_;
         result.bitField1_ = to_bitField1_;
         onBuilt();
@@ -6119,8 +6119,8 @@ public final class PTableProtos {
         if (other.hasTransactionProvider()) {
           setTransactionProvider(other.getTransactionProvider());
         }
-        if (other.hasViewIndexType()) {
-          setViewIndexType(other.getViewIndexType());
+        if (other.hasViewIndexIdType()) {
+          setViewIndexIdType(other.getViewIndexIdType());
         }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
@@ -8234,35 +8234,35 @@ public final class PTableProtos {
         return this;
       }
 
-      // optional int32 viewIndexType = 39 [default = 5];
-      private int viewIndexType_ = 5;
+      // optional int32 viewIndexIdType = 39 [default = 5];
+      private int viewIndexIdType_ = 5;
       /**
-       * <code>optional int32 viewIndexType = 39 [default = 5];</code>
+       * <code>optional int32 viewIndexIdType = 39 [default = 5];</code>
        */
-      public boolean hasViewIndexType() {
+      public boolean hasViewIndexIdType() {
         return ((bitField1_ & 0x00000020) == 0x00000020);
       }
       /**
-       * <code>optional int32 viewIndexType = 39 [default = 5];</code>
+       * <code>optional int32 viewIndexIdType = 39 [default = 5];</code>
        */
-      public int getViewIndexType() {
-        return viewIndexType_;
+      public int getViewIndexIdType() {
+        return viewIndexIdType_;
       }
       /**
-       * <code>optional int32 viewIndexType = 39 [default = 5];</code>
+       * <code>optional int32 viewIndexIdType = 39 [default = 5];</code>
        */
-      public Builder setViewIndexType(int value) {
+      public Builder setViewIndexIdType(int value) {
         bitField1_ |= 0x00000020;
-        viewIndexType_ = value;
+        viewIndexIdType_ = value;
         onChanged();
         return this;
       }
       /**
-       * <code>optional int32 viewIndexType = 39 [default = 5];</code>
+       * <code>optional int32 viewIndexIdType = 39 [default = 5];</code>
        */
-      public Builder clearViewIndexType() {
+      public Builder clearViewIndexIdType() {
         bitField1_ = (bitField1_ & ~0x00000020);
-        viewIndexType_ = 5;
+        viewIndexIdType_ = 5;
         onChanged();
         return this;
       }
@@ -8936,7 +8936,7 @@ public final class PTableProtos {
       "es\030\002 \003(\014\022\033\n\023guidePostsByteCount\030\003 \001(\003\022\025\n" +
       "\rkeyBytesCount\030\004 \001(\003\022\027\n\017guidePostsCount\030" +
       "\005 \001(\005\022!\n\013pGuidePosts\030\006 \001(\0132\014.PGuidePosts" +
-      "\"\307\007\n\006PTable\022\027\n\017schemaNameBytes\030\001 \002(\014\022\026\n\016" +
+      "\"\311\007\n\006PTable\022\027\n\017schemaNameBytes\030\001 \002(\014\022\026\n\016" +
       "tableNameBytes\030\002 \002(\014\022\036\n\ttableType\030\003 \002(\0162" +
       "\013.PTableType\022\022\n\nindexState\030\004 \001(\t\022\026\n\016sequ" +
       "enceNumber\030\005 \002(\003\022\021\n\ttimeStamp\030\006 \002(\003\022\023\n\013p" +
@@ -8959,12 +8959,13 @@ public final class PTableProtos {
       "eme\030\" \001(\014\022\026\n\016encodingScheme\030# \001(\014\022,\n\021enc" +
       "odedCQCounters\030$ \003(\0132\021.EncodedCQCounter\022" +
       "\"\n\032useStatsForParallelization\030% \001(\010\022\033\n\023t" +
-      "ransactionProvider\030& \001(\005\022\030\n\rviewIndexTyp" +
-      "e\030\' \001(\005:\0015\"6\n\020EncodedCQCounter\022\021\n\tcolFam" +
-      "ily\030\001 \002(\t\022\017\n\007counter\030\002 \002(\005*A\n\nPTableType",
-      "\022\n\n\006SYSTEM\020\000\022\010\n\004USER\020\001\022\010\n\004VIEW\020\002\022\t\n\005INDE" +
-      "X\020\003\022\010\n\004JOIN\020\004B@\n(org.apache.phoenix.copr" +
-      "ocessor.generatedB\014PTableProtosH\001\210\001\001\240\001\001"
+      "ransactionProvider\030& \001(\005\022\032\n\017viewIndexIdT" +
+      "ype\030\' \001(\005:\0015\"6\n\020EncodedCQCounter\022\021\n\tcolF" +
+      "amily\030\001 \002(\t\022\017\n\007counter\030\002 \002(\005*A\n\nPTableTy",
+      "pe\022\n\n\006SYSTEM\020\000\022\010\n\004USER\020\001\022\010\n\004VIEW\020\002\022\t\n\005IN" +
+      "DEX\020\003\022\010\n\004JOIN\020\004B@\n(org.apache.phoenix.co" +
+      "processor.generatedB\014PTableProtosH\001\210\001\001\240\001" +
+      "\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -8988,7 +8989,7 @@ public final class PTableProtos {
           internal_static_PTable_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_PTable_descriptor,
-              new java.lang.String[] { "SchemaNameBytes", "TableNameBytes", "TableType", "IndexState", "SequenceNumber", "TimeStamp", "PkNameBytes", "BucketNum", "Columns", "Indexes", "IsImmutableRows", "DataTableNameBytes", "DefaultFamilyName", "DisableWAL", "MultiTenant", "ViewType", "ViewStatement", "PhysicalNames", "TenantId", "ViewIndexId", "IndexType", "StatsTimeStamp", "StoreNulls", "BaseColumnCount", "RowKeyOrderOptimizable", "Transactional", "UpdateCacheFrequency", "IndexDisable [...]
+              new java.lang.String[] { "SchemaNameBytes", "TableNameBytes", "TableType", "IndexState", "SequenceNumber", "TimeStamp", "PkNameBytes", "BucketNum", "Columns", "Indexes", "IsImmutableRows", "DataTableNameBytes", "DefaultFamilyName", "DisableWAL", "MultiTenant", "ViewType", "ViewStatement", "PhysicalNames", "TenantId", "ViewIndexId", "IndexType", "StatsTimeStamp", "StoreNulls", "BaseColumnCount", "RowKeyOrderOptimizable", "Transactional", "UpdateCacheFrequency", "IndexDisable [...]
           internal_static_EncodedCQCounter_descriptor =
             getDescriptor().getMessageTypes().get(3);
           internal_static_EncodedCQCounter_fieldAccessorTable = new
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/ServerCachingProtos.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/ServerCachingProtos.java
index c42b9df..138be15 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/ServerCachingProtos.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/ServerCachingProtos.java
@@ -2158,15 +2158,15 @@ public final class ServerCachingProtos {
      */
     int getImmutableStorageScheme();
 
-    // optional int32 viewIndexType = 22;
+    // optional int32 viewIndexIdType = 22;
     /**
-     * <code>optional int32 viewIndexType = 22;</code>
+     * <code>optional int32 viewIndexIdType = 22;</code>
      */
-    boolean hasViewIndexType();
+    boolean hasViewIndexIdType();
     /**
-     * <code>optional int32 viewIndexType = 22;</code>
+     * <code>optional int32 viewIndexIdType = 22;</code>
      */
-    int getViewIndexType();
+    int getViewIndexIdType();
   }
   /**
    * Protobuf type {@code IndexMaintainer}
@@ -2362,7 +2362,7 @@ public final class ServerCachingProtos {
             }
             case 176: {
               bitField0_ |= 0x00010000;
-              viewIndexType_ = input.readInt32();
+              viewIndexIdType_ = input.readInt32();
               break;
             }
           }
@@ -2849,20 +2849,20 @@ public final class ServerCachingProtos {
       return immutableStorageScheme_;
     }
 
-    // optional int32 viewIndexType = 22;
-    public static final int VIEWINDEXTYPE_FIELD_NUMBER = 22;
-    private int viewIndexType_;
+    // optional int32 viewIndexIdType = 22;
+    public static final int VIEWINDEXIDTYPE_FIELD_NUMBER = 22;
+    private int viewIndexIdType_;
     /**
-     * <code>optional int32 viewIndexType = 22;</code>
+     * <code>optional int32 viewIndexIdType = 22;</code>
      */
-    public boolean hasViewIndexType() {
+    public boolean hasViewIndexIdType() {
       return ((bitField0_ & 0x00010000) == 0x00010000);
     }
     /**
-     * <code>optional int32 viewIndexType = 22;</code>
+     * <code>optional int32 viewIndexIdType = 22;</code>
      */
-    public int getViewIndexType() {
-      return viewIndexType_;
+    public int getViewIndexIdType() {
+      return viewIndexIdType_;
     }
 
     private void initFields() {
@@ -2887,7 +2887,7 @@ public final class ServerCachingProtos {
       indexedColumnInfo_ = java.util.Collections.emptyList();
       encodingScheme_ = 0;
       immutableStorageScheme_ = 0;
-      viewIndexType_ = 0;
+      viewIndexIdType_ = 0;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -3049,7 +3049,7 @@ public final class ServerCachingProtos {
         output.writeInt32(21, immutableStorageScheme_);
       }
       if (((bitField0_ & 0x00010000) == 0x00010000)) {
-        output.writeInt32(22, viewIndexType_);
+        output.writeInt32(22, viewIndexIdType_);
       }
       getUnknownFields().writeTo(output);
     }
@@ -3151,7 +3151,7 @@ public final class ServerCachingProtos {
       }
       if (((bitField0_ & 0x00010000) == 0x00010000)) {
         size += com.google.protobuf.CodedOutputStream
-          .computeInt32Size(22, viewIndexType_);
+          .computeInt32Size(22, viewIndexIdType_);
       }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
@@ -3266,10 +3266,10 @@ public final class ServerCachingProtos {
         result = result && (getImmutableStorageScheme()
             == other.getImmutableStorageScheme());
       }
-      result = result && (hasViewIndexType() == other.hasViewIndexType());
-      if (hasViewIndexType()) {
-        result = result && (getViewIndexType()
-            == other.getViewIndexType());
+      result = result && (hasViewIndexIdType() == other.hasViewIndexIdType());
+      if (hasViewIndexIdType()) {
+        result = result && (getViewIndexIdType()
+            == other.getViewIndexIdType());
       }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
@@ -3368,9 +3368,9 @@ public final class ServerCachingProtos {
         hash = (37 * hash) + IMMUTABLESTORAGESCHEME_FIELD_NUMBER;
         hash = (53 * hash) + getImmutableStorageScheme();
       }
-      if (hasViewIndexType()) {
-        hash = (37 * hash) + VIEWINDEXTYPE_FIELD_NUMBER;
-        hash = (53 * hash) + getViewIndexType();
+      if (hasViewIndexIdType()) {
+        hash = (37 * hash) + VIEWINDEXIDTYPE_FIELD_NUMBER;
+        hash = (53 * hash) + getViewIndexIdType();
       }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
@@ -3548,7 +3548,7 @@ public final class ServerCachingProtos {
         bitField0_ = (bitField0_ & ~0x00080000);
         immutableStorageScheme_ = 0;
         bitField0_ = (bitField0_ & ~0x00100000);
-        viewIndexType_ = 0;
+        viewIndexIdType_ = 0;
         bitField0_ = (bitField0_ & ~0x00200000);
         return this;
       }
@@ -3690,7 +3690,7 @@ public final class ServerCachingProtos {
         if (((from_bitField0_ & 0x00200000) == 0x00200000)) {
           to_bitField0_ |= 0x00010000;
         }
-        result.viewIndexType_ = viewIndexType_;
+        result.viewIndexIdType_ = viewIndexIdType_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -3869,8 +3869,8 @@ public final class ServerCachingProtos {
         if (other.hasImmutableStorageScheme()) {
           setImmutableStorageScheme(other.getImmutableStorageScheme());
         }
-        if (other.hasViewIndexType()) {
-          setViewIndexType(other.getViewIndexType());
+        if (other.hasViewIndexIdType()) {
+          setViewIndexIdType(other.getViewIndexIdType());
         }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
@@ -5636,35 +5636,35 @@ public final class ServerCachingProtos {
         return this;
       }
 
-      // optional int32 viewIndexType = 22;
-      private int viewIndexType_ ;
+      // optional int32 viewIndexIdType = 22;
+      private int viewIndexIdType_ ;
       /**
-       * <code>optional int32 viewIndexType = 22;</code>
+       * <code>optional int32 viewIndexIdType = 22;</code>
        */
-      public boolean hasViewIndexType() {
+      public boolean hasViewIndexIdType() {
         return ((bitField0_ & 0x00200000) == 0x00200000);
       }
       /**
-       * <code>optional int32 viewIndexType = 22;</code>
+       * <code>optional int32 viewIndexIdType = 22;</code>
        */
-      public int getViewIndexType() {
-        return viewIndexType_;
+      public int getViewIndexIdType() {
+        return viewIndexIdType_;
       }
       /**
-       * <code>optional int32 viewIndexType = 22;</code>
+       * <code>optional int32 viewIndexIdType = 22;</code>
        */
-      public Builder setViewIndexType(int value) {
+      public Builder setViewIndexIdType(int value) {
         bitField0_ |= 0x00200000;
-        viewIndexType_ = value;
+        viewIndexIdType_ = value;
         onChanged();
         return this;
       }
       /**
-       * <code>optional int32 viewIndexType = 22;</code>
+       * <code>optional int32 viewIndexIdType = 22;</code>
        */
-      public Builder clearViewIndexType() {
+      public Builder clearViewIndexIdType() {
         bitField0_ = (bitField0_ & ~0x00200000);
-        viewIndexType_ = 0;
+        viewIndexIdType_ = 0;
         onChanged();
         return this;
       }
@@ -8795,7 +8795,7 @@ public final class ServerCachingProtos {
       "ength\030\003 \002(\005\"4\n\017ColumnReference\022\016\n\006family" +
       "\030\001 \002(\014\022\021\n\tqualifier\030\002 \002(\014\"4\n\nColumnInfo\022" +
       "\022\n\nfamilyName\030\001 \001(\t\022\022\n\ncolumnName\030\002 \002(\t\"" +
-      "\335\005\n\017IndexMaintainer\022\023\n\013saltBuckets\030\001 \002(\005" +
+      "\337\005\n\017IndexMaintainer\022\023\n\013saltBuckets\030\001 \002(\005" +
       "\022\025\n\risMultiTenant\030\002 \002(\010\022\023\n\013viewIndexId\030\003" +
       " \001(\014\022(\n\016indexedColumns\030\004 \003(\0132\020.ColumnRef" +
       "erence\022 \n\030indexedColumnTypeOrdinal\030\005 \003(\005",
@@ -8812,24 +8812,24 @@ public final class ServerCachingProtos {
       "ed\030\020 \002(\010\022\033\n\023indexRowKeyByteSize\030\021 \002(\005\022\021\n" +
       "\timmutable\030\022 \002(\010\022&\n\021indexedColumnInfo\030\023 " +
       "\003(\0132\013.ColumnInfo\022\026\n\016encodingScheme\030\024 \002(\005" +
-      "\022\036\n\026immutableStorageScheme\030\025 \002(\005\022\025\n\rview" +
-      "IndexType\030\026 \001(\005\"\370\001\n\025AddServerCacheReques" +
-      "t\022\020\n\010tenantId\030\001 \001(\014\022\017\n\007cacheId\030\002 \002(\014\022)\n\010" +
-      "cachePtr\030\003 \002(\0132\027.ImmutableBytesWritable\022" +
-      ")\n\014cacheFactory\030\004 \002(\0132\023.ServerCacheFacto" +
-      "ry\022\017\n\007txState\030\005 \001(\014\022\"\n\032hasProtoBufIndexM" +
-      "aintainer\030\006 \001(\010\022\025\n\rclientVersion\030\007 \001(\005\022\032",
-      "\n\022usePersistentCache\030\010 \001(\010\"(\n\026AddServerC" +
-      "acheResponse\022\016\n\006return\030\001 \002(\010\"=\n\030RemoveSe" +
-      "rverCacheRequest\022\020\n\010tenantId\030\001 \001(\014\022\017\n\007ca" +
-      "cheId\030\002 \002(\014\"+\n\031RemoveServerCacheResponse" +
-      "\022\016\n\006return\030\001 \002(\0102\245\001\n\024ServerCachingServic" +
-      "e\022A\n\016addServerCache\022\026.AddServerCacheRequ" +
-      "est\032\027.AddServerCacheResponse\022J\n\021removeSe" +
-      "rverCache\022\031.RemoveServerCacheRequest\032\032.R" +
-      "emoveServerCacheResponseBG\n(org.apache.p" +
-      "hoenix.coprocessor.generatedB\023ServerCach",
-      "ingProtosH\001\210\001\001\240\001\001"
+      "\022\036\n\026immutableStorageScheme\030\025 \002(\005\022\027\n\017view" +
+      "IndexIdType\030\026 \001(\005\"\370\001\n\025AddServerCacheRequ" +
+      "est\022\020\n\010tenantId\030\001 \001(\014\022\017\n\007cacheId\030\002 \002(\014\022)" +
+      "\n\010cachePtr\030\003 \002(\0132\027.ImmutableBytesWritabl" +
+      "e\022)\n\014cacheFactory\030\004 \002(\0132\023.ServerCacheFac" +
+      "tory\022\017\n\007txState\030\005 \001(\014\022\"\n\032hasProtoBufInde" +
+      "xMaintainer\030\006 \001(\010\022\025\n\rclientVersion\030\007 \001(\005",
+      "\022\032\n\022usePersistentCache\030\010 \001(\010\"(\n\026AddServe" +
+      "rCacheResponse\022\016\n\006return\030\001 \002(\010\"=\n\030Remove" +
+      "ServerCacheRequest\022\020\n\010tenantId\030\001 \001(\014\022\017\n\007" +
+      "cacheId\030\002 \002(\014\"+\n\031RemoveServerCacheRespon" +
+      "se\022\016\n\006return\030\001 \002(\0102\245\001\n\024ServerCachingServ" +
+      "ice\022A\n\016addServerCache\022\026.AddServerCacheRe" +
+      "quest\032\027.AddServerCacheResponse\022J\n\021remove" +
+      "ServerCache\022\031.RemoveServerCacheRequest\032\032" +
+      ".RemoveServerCacheResponseBG\n(org.apache" +
+      ".phoenix.coprocessor.generatedB\023ServerCa",
+      "chingProtosH\001\210\001\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -8859,7 +8859,7 @@ public final class ServerCachingProtos {
           internal_static_IndexMaintainer_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_IndexMaintainer_descriptor,
-              new java.lang.String[] { "SaltBuckets", "IsMultiTenant", "ViewIndexId", "IndexedColumns", "IndexedColumnTypeOrdinal", "DataTableColRefForCoveredColumns", "IndexTableColRefForCoveredColumns", "IsLocalIndex", "IndexTableName", "RowKeyOrderOptimizable", "DataTableEmptyKeyValueColFamily", "EmptyKeyValueColFamily", "IndexedExpressions", "RowKeyMetadata", "NumDataTableColFamilies", "IndexWalDisabled", "IndexRowKeyByteSize", "Immutable", "IndexedColumnInfo", "EncodingScheme", "Imm [...]
+              new java.lang.String[] { "SaltBuckets", "IsMultiTenant", "ViewIndexId", "IndexedColumns", "IndexedColumnTypeOrdinal", "DataTableColRefForCoveredColumns", "IndexTableColRefForCoveredColumns", "IsLocalIndex", "IndexTableName", "RowKeyOrderOptimizable", "DataTableEmptyKeyValueColFamily", "EmptyKeyValueColFamily", "IndexedExpressions", "RowKeyMetadata", "NumDataTableColFamilies", "IndexWalDisabled", "IndexRowKeyByteSize", "Immutable", "IndexedColumnInfo", "EncodingScheme", "Imm [...]
           internal_static_AddServerCacheRequest_descriptor =
             getDescriptor().getMessageTypes().get(4);
           internal_static_AddServerCacheRequest_fieldAccessorTable = new
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java b/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java
index d1da5f1..d3d14d8 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java
@@ -318,7 +318,7 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
     }
     
     private byte[] viewIndexId;
-    private PDataType viewIndexType;
+    private PDataType viewIndexIdType;
     private boolean isMultiTenant;
     // indexed expressions that are not present in the row key of the data table, the expression can also refer to a regular column
     private List<Expression> indexedExpressions;
@@ -376,8 +376,8 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
         this(dataTable.getRowKeySchema(), dataTable.getBucketNum() != null);
         this.rowKeyOrderOptimizable = index.rowKeyOrderOptimizable();
         this.isMultiTenant = dataTable.isMultiTenant();
-        this.viewIndexId = index.getViewIndexId() == null ? null : index.getViewIndexType().toBytes(index.getViewIndexId());
-        this.viewIndexType = index.getViewIndexType();
+        this.viewIndexId = index.getViewIndexId() == null ? null : index.getviewIndexIdType().toBytes(index.getViewIndexId());
+        this.viewIndexIdType = index.getviewIndexIdType();
         this.isLocalIndex = index.getIndexType() == IndexType.LOCAL;
         this.encodingScheme = index.getEncodingScheme();
         
@@ -829,7 +829,7 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
 
                 @Override
                 public PDataType getDataType() {
-                    return viewIndexType;
+                    return viewIndexIdType;
                 }
 
                 @Override
@@ -1228,7 +1228,7 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
             // Fixed length
             //Use legacy viewIndexIdType for clients older than 4.10 release
             viewIndexId = new byte[MetaDataUtil.getLegacyViewIndexIdDataType().getByteSize()];
-            viewIndexType = MetaDataUtil.getLegacyViewIndexIdDataType();
+            viewIndexIdType = MetaDataUtil.getLegacyViewIndexIdDataType();
             input.readFully(viewIndexId);
         }
         int nIndexedColumns = Math.abs(encodedIndexedColumnsAndViewId) - 1;
@@ -1345,8 +1345,8 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
         maintainer.nIndexSaltBuckets = proto.getSaltBuckets();
         maintainer.isMultiTenant = proto.getIsMultiTenant();
         maintainer.viewIndexId = proto.hasViewIndexId() ? proto.getViewIndexId().toByteArray() : null;
-        maintainer.viewIndexType = proto.hasViewIndexType()
-                ? PDataType.fromTypeId(proto.getViewIndexType())
+        maintainer.viewIndexIdType = proto.hasViewIndexIdType()
+                ? PDataType.fromTypeId(proto.getViewIndexIdType())
                 : MetaDataUtil.getLegacyViewIndexIdDataType();
         List<ServerCachingProtos.ColumnReference> indexedColumnsList = proto.getIndexedColumnsList();
         maintainer.indexedColumns = new HashSet<ColumnReference>(indexedColumnsList.size());
@@ -1467,7 +1467,7 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
         builder.setIsMultiTenant(maintainer.isMultiTenant);
         if (maintainer.viewIndexId != null) {
             builder.setViewIndexId(ByteStringer.wrap(maintainer.viewIndexId));
-            builder.setViewIndexType(maintainer.viewIndexType.getSqlType());
+            builder.setViewIndexIdType(maintainer.viewIndexIdType.getSqlType());
         }
         for (ColumnReference colRef : maintainer.indexedColumns) {
             ServerCachingProtos.ColumnReference.Builder cRefBuilder =  ServerCachingProtos.ColumnReference.newBuilder();
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 2d0c22c..172c2d6 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
@@ -320,7 +320,7 @@ public class PhoenixIndexFailurePolicy extends DelegateIndexFailurePolicy {
             for (PTable index : indexes) {
                 if (index.getIndexType() == IndexType.LOCAL) {
                     localIndex = index;
-                    localIndexNames.put(new ImmutableBytesWritable(index.getViewIndexType().toBytes(
+                    localIndexNames.put(new ImmutableBytesWritable(index.getviewIndexIdType().toBytes(
                             index.getViewIndexId())), index.getName().getString());
                 }
             }
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateTable.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateTable.java
index bf2bc59..34a973f 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateTable.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateTable.java
@@ -218,8 +218,8 @@ public class DelegateTable implements PTable {
     }
 
     @Override
-    public PDataType getViewIndexType() {
-        return delegate.getViewIndexType();
+    public PDataType getviewIndexIdType() {
+        return delegate.getviewIndexIdType();
     }
 
     @Override
@@ -332,4 +332,12 @@ public class DelegateTable implements PTable {
     public Boolean useStatsForParallelization() {
         return delegate.useStatsForParallelization();
     }
+
+    @Override public boolean hasViewModifiedUpdateCacheFrequency() {
+        return delegate.hasViewModifiedUpdateCacheFrequency();
+    }
+
+    @Override public boolean hasViewModifiedUseStatsForParallelization() {
+        return delegate.hasViewModifiedUseStatsForParallelization();
+    }
 }
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 4b7c9af..67a4928 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
@@ -99,6 +99,7 @@ import static org.apache.phoenix.query.QueryConstants.ENCODED_CQ_COUNTER_INITIAL
 import static org.apache.phoenix.query.QueryServices.DROP_METADATA_ATTRIB;
 import static org.apache.phoenix.query.QueryServicesOptions.DEFAULT_DROP_METADATA;
 import static org.apache.phoenix.query.QueryServicesOptions.DEFAULT_RUN_UPDATE_STATS_ASYNC;
+import static org.apache.phoenix.query.QueryServicesOptions.DEFAULT_USE_STATS_FOR_PARALLELIZATION;
 import static org.apache.phoenix.schema.PTable.EncodedCQCounter.NULL_COUNTER;
 import static org.apache.phoenix.schema.PTable.ImmutableStorageScheme.ONE_CELL_PER_COLUMN;
 import static org.apache.phoenix.schema.PTable.ImmutableStorageScheme.SINGLE_CELL_ARRAY_WITH_OFFSETS;
@@ -1077,7 +1078,7 @@ public class MetaDataClient {
         }
     }
     
-    public MutationState createTable(CreateTableStatement statement, byte[][] splits, PTable parent, String viewStatement, ViewType viewType, PDataType viewIndexType, byte[][] viewColumnConstants, BitSet isViewColumnReferenced) throws SQLException {
+    public MutationState createTable(CreateTableStatement statement, byte[][] splits, PTable parent, String viewStatement, ViewType viewType, PDataType viewIndexIdType, byte[][] viewColumnConstants, BitSet isViewColumnReferenced) throws SQLException {
         TableName tableName = statement.getTableName();
         Map<String,Object> tableProps = Maps.newHashMapWithExpectedSize(statement.getProps().size());
         Map<String,Object> commonFamilyProps = Maps.newHashMapWithExpectedSize(statement.getProps().size() + 1);
@@ -1086,19 +1087,20 @@ public class MetaDataClient {
         boolean isAppendOnlySchema = false;
         long updateCacheFrequency = connection.getQueryServices().getProps().getLong(
             QueryServices.DEFAULT_UPDATE_CACHE_FREQUENCY_ATRRIB, QueryServicesOptions.DEFAULT_UPDATE_CACHE_FREQUENCY);
+        Long updateCacheFrequencyProp = (Long) TableProperty.UPDATE_CACHE_FREQUENCY.getValue(tableProps);
         if (parent==null) {
 	        Boolean appendOnlySchemaProp = (Boolean) TableProperty.APPEND_ONLY_SCHEMA.getValue(tableProps);
 	        if (appendOnlySchemaProp != null) {
 	            isAppendOnlySchema = appendOnlySchemaProp;
 	        }
-	        Long updateCacheFrequencyProp = (Long) TableProperty.UPDATE_CACHE_FREQUENCY.getValue(tableProps);
 	        if (updateCacheFrequencyProp != null) {
 	            updateCacheFrequency = updateCacheFrequencyProp;
 	        }
         }
         else {
         	isAppendOnlySchema = parent.isAppendOnlySchema();
-        	updateCacheFrequency = parent.getUpdateCacheFrequency();
+        	updateCacheFrequency = (updateCacheFrequencyProp != null) ?
+                    updateCacheFrequencyProp : parent.getUpdateCacheFrequency();
         }
         // updateCacheFrequency cannot be set to ALWAYS if isAppendOnlySchema is true
         if (isAppendOnlySchema && updateCacheFrequency==0) {
@@ -1138,7 +1140,7 @@ public class MetaDataClient {
                         true, NamedTableNode.create(statement.getTableName()), statement.getTableType());
             }
         }
-        table = createTableInternal(statement, splits, parent, viewStatement, viewType, viewIndexType, viewColumnConstants, isViewColumnReferenced, false, null, null, tableProps, commonFamilyProps);
+        table = createTableInternal(statement, splits, parent, viewStatement, viewType, viewIndexIdType, viewColumnConstants, isViewColumnReferenced, false, null, null, tableProps, commonFamilyProps);
 
         if (table == null || table.getType() == PTableType.VIEW /*|| table.isTransactional()*/) {
             return new MutationState(0, 0, connection);
@@ -1986,7 +1988,7 @@ public class MetaDataClient {
     }
 
     private PTable createTableInternal(CreateTableStatement statement, byte[][] splits,
-            final PTable parent, String viewStatement, ViewType viewType, PDataType viewIndexType,
+            final PTable parent, String viewStatement, ViewType viewType, PDataType viewIndexIdType,
             final byte[][] viewColumnConstants, final BitSet isViewColumnReferenced, boolean allocateIndexId,
             IndexType indexType, Date asyncCreatedDate,
             Map<String,Object> tableProps,
@@ -2680,7 +2682,7 @@ public class MetaDataClient {
                         .setDisableWAL(Boolean.TRUE.equals(disableWAL))
                         .setMultiTenant(false)
                         .setStoreNulls(false)
-                        .setViewIndexType(viewIndexType)
+                        .setViewIndexIdType(viewIndexIdType)
                         .setIndexType(indexType)
                         .setUpdateCacheFrequency(0)
                         .setNamespaceMapped(isNamespaceMapped)
@@ -2999,7 +3001,7 @@ public class MetaDataClient {
                         .setMultiTenant(multiTenant)
                         .setStoreNulls(storeNulls)
                         .setViewType(viewType)
-                        .setViewIndexType(viewIndexType)
+                        .setViewIndexIdType(viewIndexIdType)
                         .setViewIndexId(result.getViewIndexId())
                         .setIndexType(indexType)
                         .setTransactionProvider(transactionProvider)
@@ -3029,6 +3031,13 @@ public class MetaDataClient {
                         .setPhysicalNames(physicalNames == null ?
                                 ImmutableList.<PName>of() : ImmutableList.copyOf(physicalNames))
                         .setColumns(columns.values())
+                        .setViewModifiedUpdateCacheFrequency(tableType ==  PTableType.VIEW &&
+                                parent != null &&
+                                parent.getUpdateCacheFrequency() != updateCacheFrequency)
+                        .setViewModifiedUseStatsForParallelization(tableType ==  PTableType.VIEW &&
+                                parent != null &&
+                                parent.useStatsForParallelization()
+                                        != useStatsForParallelizationProp)
                         .build();
                 result = new MetaDataMutationResult(code, result.getMutationTime(), table, true);
                 addTableToCache(result);
@@ -4178,7 +4187,7 @@ public class MetaDataClient {
                                         .setType(PTableType.INDEX)
                                         .setTimeStamp(ts)
                                         .setMultiTenant(table.isMultiTenant())
-                                        .setViewIndexType(sharedTableState.getViewIndexType())
+                                        .setViewIndexIdType(sharedTableState.getViewIndexIdType())
                                         .setViewIndexId(sharedTableState.getViewIndexId())
                                         .setNamespaceMapped(table.isNamespaceMapped())
                                         .setAppendOnlySchema(false)
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTable.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTable.java
index 6dfe411..3e22225 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTable.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTable.java
@@ -736,7 +736,7 @@ public interface PTable extends PMetaDataEntity {
     ViewType getViewType();
     String getViewStatement();
     Long getViewIndexId();
-    PDataType getViewIndexType();
+    PDataType getviewIndexIdType();
     PTableKey getKey();
 
     IndexType getIndexType();
@@ -774,7 +774,9 @@ public interface PTable extends PMetaDataEntity {
     QualifierEncodingScheme getEncodingScheme();
     EncodedCQCounter getEncodedCQCounter();
     Boolean useStatsForParallelization();
-    
+    boolean hasViewModifiedUpdateCacheFrequency();
+    boolean hasViewModifiedUseStatsForParallelization();
+
     /**
      * Class to help track encoded column qualifier counters per column family.
      */
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
index 7939b97..8b2b4ff 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
@@ -26,6 +26,7 @@ import java.sql.DriverManager;
 import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.BitSet;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
@@ -101,6 +102,8 @@ import com.google.common.collect.Maps;
  */
 public class PTableImpl implements PTable {
     private static final Integer NO_SALTING = -1;
+    private static final int VIEW_MODIFIED_UPDATE_CACHE_FREQUENCY_BIT_SET_POS = 0;
+    private static final int VIEW_MODIFIED_USE_STATS_FOR_PARALLELIZATION_BIT_SET_POS = 1;
 
     private IndexMaintainer indexMaintainer;
     private ImmutableBytesWritable indexMaintainersPtr;
@@ -143,7 +146,7 @@ public class PTableImpl implements PTable {
     private final boolean storeNulls;
     private final TransactionFactory.Provider transactionProvider;
     private final ViewType viewType;
-    private final PDataType viewIndexType;
+    private final PDataType viewIndexIdType;
     private final Long viewIndexId;
     private final int estimatedSize;
     private final IndexType indexType;
@@ -159,6 +162,7 @@ public class PTableImpl implements PTable {
     private final QualifierEncodingScheme qualifierEncodingScheme;
     private final EncodedCQCounter encodedCQCounter;
     private final Boolean useStatsForParallelization;
+    private final BitSet viewModifiedPropSet;
 
     public static class Builder {
         private PTableKey key;
@@ -197,7 +201,7 @@ public class PTableImpl implements PTable {
         private boolean storeNulls;
         private TransactionFactory.Provider transactionProvider;
         private ViewType viewType;
-        private PDataType viewIndexType;
+        private PDataType viewIndexIdType;
         private Long viewIndexId;
         private int estimatedSize;
         private IndexType indexType;
@@ -213,6 +217,8 @@ public class PTableImpl implements PTable {
         private QualifierEncodingScheme qualifierEncodingScheme;
         private EncodedCQCounter encodedCQCounter;
         private Boolean useStatsForParallelization;
+        // Used to denote which properties a view has explicitly modified
+        private BitSet viewModifiedPropSet = new BitSet(2);
         // Optionally set columns for the builder, but not for the actual PTable
         private Collection<PColumn> columns;
 
@@ -407,8 +413,8 @@ public class PTableImpl implements PTable {
             return this;
         }
 
-        public Builder setViewIndexType(PDataType viewIndexType) {
-            this.viewIndexType = viewIndexType;
+        public Builder setViewIndexIdType(PDataType viewIndexIdType) {
+            this.viewIndexIdType = viewIndexIdType;
             return this;
         }
 
@@ -487,6 +493,18 @@ public class PTableImpl implements PTable {
             return this;
         }
 
+        public Builder setViewModifiedUpdateCacheFrequency(boolean modified) {
+            this.viewModifiedPropSet.set(VIEW_MODIFIED_UPDATE_CACHE_FREQUENCY_BIT_SET_POS,
+                    modified);
+            return this;
+        }
+
+        public Builder setViewModifiedUseStatsForParallelization(boolean modified) {
+            this.viewModifiedPropSet.set(VIEW_MODIFIED_USE_STATS_FOR_PARALLELIZATION_BIT_SET_POS,
+                    modified);
+            return this;
+        }
+
         /**
          * Note: When set in the builder, we must call {@link Builder#initDerivedAttributes()}
          * before building the PTable in order to correctly populate other attributes of the PTable
@@ -741,7 +759,7 @@ public class PTableImpl implements PTable {
         this.storeNulls = builder.storeNulls;
         this.transactionProvider = builder.transactionProvider;
         this.viewType = builder.viewType;
-        this.viewIndexType = builder.viewIndexType;
+        this.viewIndexIdType = builder.viewIndexIdType;
         this.viewIndexId = builder.viewIndexId;
         this.estimatedSize = builder.estimatedSize;
         this.indexType = builder.indexType;
@@ -757,6 +775,7 @@ public class PTableImpl implements PTable {
         this.qualifierEncodingScheme = builder.qualifierEncodingScheme;
         this.encodedCQCounter = builder.encodedCQCounter;
         this.useStatsForParallelization = builder.useStatsForParallelization;
+        this.viewModifiedPropSet = builder.viewModifiedPropSet;
     }
 
     // When cloning table, ignore the salt column as it will be added back in the constructor
@@ -791,7 +810,7 @@ public class PTableImpl implements PTable {
                 .setMultiTenant(table.isMultiTenant())
                 .setStoreNulls(table.getStoreNulls())
                 .setViewType(table.getViewType())
-                .setViewIndexType(table.getViewIndexType())
+                .setViewIndexIdType(table.getviewIndexIdType())
                 .setViewIndexId(table.getViewIndexId())
                 .setIndexType(table.getIndexType())
                 .setTransactionProvider(table.getTransactionProvider())
@@ -820,7 +839,10 @@ public class PTableImpl implements PTable {
                 .setParentSchemaName(table.getParentSchemaName())
                 .setParentTableName(table.getParentTableName())
                 .setPhysicalNames(table.getPhysicalNames() == null ?
-                        ImmutableList.<PName>of() : ImmutableList.copyOf(table.getPhysicalNames()));
+                        ImmutableList.<PName>of() : ImmutableList.copyOf(table.getPhysicalNames()))
+                .setViewModifiedUseStatsForParallelization(table
+                        .hasViewModifiedUseStatsForParallelization())
+                .setViewModifiedUpdateCacheFrequency(table.hasViewModifiedUpdateCacheFrequency());
     }
 
     @Override
@@ -1442,8 +1464,8 @@ public class PTableImpl implements PTable {
     }
 
     @Override
-    public PDataType getViewIndexType() {
-        return viewIndexType;
+    public PDataType getviewIndexIdType() {
+        return viewIndexIdType;
     }
 
     @Override
@@ -1478,8 +1500,8 @@ public class PTableImpl implements PTable {
         if (table.hasViewIndexId()) {
             viewIndexId = table.getViewIndexId();
         }
-        PDataType viewIndexType = table.hasViewIndexType()
-                ? PDataType.fromTypeId(table.getViewIndexType())
+        PDataType viewIndexIdType = table.hasViewIndexIdType()
+                ? PDataType.fromTypeId(table.getViewIndexIdType())
                 : MetaDataUtil.getLegacyViewIndexIdDataType();
         IndexType indexType = IndexType.getDefault();
         if(table.hasIndexType()){
@@ -1602,7 +1624,7 @@ public class PTableImpl implements PTable {
                     .setMultiTenant(multiTenant)
                     .setStoreNulls(storeNulls)
                     .setViewType(viewType)
-                    .setViewIndexType(viewIndexType)
+                    .setViewIndexIdType(viewIndexIdType)
                     .setViewIndexId(viewIndexId)
                     .setIndexType(indexType)
                     .setTransactionProvider(transactionProvider)
@@ -1653,7 +1675,7 @@ public class PTableImpl implements PTable {
         }
         if(table.getViewIndexId() != null) {
           builder.setViewIndexId(table.getViewIndexId());
-          builder.setViewIndexType(table.getViewIndexType().getSqlType());
+          builder.setViewIndexIdType(table.getviewIndexIdType().getSqlType());
 		}
         if(table.getIndexType() != null) {
             builder.setIndexType(ByteStringer.wrap(new byte[]{table.getIndexType().getSerializedValue()}));
@@ -1833,6 +1855,14 @@ public class PTableImpl implements PTable {
         return useStatsForParallelization;
     }
 
+    @Override public boolean hasViewModifiedUpdateCacheFrequency() {
+        return viewModifiedPropSet.get(VIEW_MODIFIED_UPDATE_CACHE_FREQUENCY_BIT_SET_POS);
+    }
+
+    @Override public boolean hasViewModifiedUseStatsForParallelization() {
+        return viewModifiedPropSet.get(VIEW_MODIFIED_USE_STATS_FOR_PARALLELIZATION_BIT_SET_POS);
+    }
+
     private static final class KVColumnFamilyQualifier {
         @Nonnull
         private final String colFamilyName;
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java
index 9aa5bd3..0b3248a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java
@@ -33,12 +33,14 @@ import java.util.NavigableMap;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.HConnection;
 import org.apache.hadoop.hbase.client.HConnectionManager;
@@ -258,6 +260,48 @@ public class MetaDataUtil {
         }
     }
 
+    /**
+     * Iterates over the cells that are mutated by the put operation for the given column family and
+     * column qualifier and conditionally modifies those cells to add a tags list. We only add tags
+     * if the cell value does not match the passed valueArray. If we always want to add tags to
+     * these cells, we can pass in a null valueArray
+     * @param somePut Put operation
+     * @param family column family of the cells
+     * @param qualifier column qualifier of the cells
+     * @param valueArray byte array of values or null
+     * @param tagArray byte array of tags to add to the cells
+     */
+    public static void conditionallyAddTagsToPutCells(Put somePut, byte[] family, byte[] qualifier,
+            byte[] valueArray, byte[] tagArray) {
+        NavigableMap<byte[], List<Cell>> familyCellMap = somePut.getFamilyCellMap();
+        List<Cell> cells = familyCellMap.get(family);
+        List<Cell> newCells = Lists.newArrayList();
+        if (cells != null) {
+            for (Cell cell : cells) {
+                if (Bytes.compareTo(cell.getQualifierArray(), cell.getQualifierOffset(),
+                        cell.getQualifierLength(), qualifier, 0, qualifier.length) == 0 &&
+                        (valueArray == null || !CellUtil.matchingValue(cell, valueArray))) {
+                    final byte[] combinedTags =
+                            ByteUtil.concat(CellUtil.getTagArray(cell), tagArray);
+                    Cell newCell = CellUtil.createCell(
+                            CellUtil.cloneRow(cell),
+                            CellUtil.cloneFamily(cell),
+                            CellUtil.cloneQualifier(cell),
+                            cell.getTimestamp(),
+                            KeyValue.Type.codeToType(cell.getTypeByte()),
+                            CellUtil.cloneValue(cell),
+                            combinedTags);
+                    // Replace existing cell with a cell that has the custom tags list
+                    newCells.add(newCell);
+                } else {
+                    // Add cell as is
+                    newCells.add(cell);
+                }
+            }
+            familyCellMap.put(family, newCells);
+        }
+    }
+
     public static Put cloneDeleteToPutAndAddColumn(Delete delete, byte[] family, byte[] qualifier, byte[] value) {
         NavigableMap<byte[], List<Cell>> familyCellMap = delete.getFamilyCellMap();
         List<Cell> cells = familyCellMap.get(family);
@@ -357,7 +401,7 @@ public class MetaDataUtil {
         }
         return 0;
     }
-    
+
     public static long getParentSequenceNumber(List<Mutation> tableMetaData) {
         return getSequenceNumber(getParentTableHeaderRow(tableMetaData));
     }
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/util/MetaDataUtilTest.java b/phoenix-core/src/test/java/org/apache/phoenix/util/MetaDataUtilTest.java
index 485a21f..d934a04 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/util/MetaDataUtilTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/util/MetaDataUtilTest.java
@@ -19,11 +19,14 @@ package org.apache.phoenix.util;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 
 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.Tag;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.VersionInfo;
@@ -32,14 +35,22 @@ import org.apache.phoenix.hbase.index.util.GenericKeyValueBuilder;
 import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.hbase.index.util.KeyValueBuilder;
 import org.apache.phoenix.hbase.index.util.VersionUtil;
-import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.query.HBaseFactoryProvider;
 import org.apache.phoenix.query.QueryServices;
 import org.junit.Test;
 
+import java.util.Arrays;
+
+import static org.apache.hadoop.hbase.HConstants.EMPTY_BYTE_ARRAY;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES;
 
 public class MetaDataUtilTest {
 
+    private static final byte[] ROW = Bytes.toBytes("row");
+    private static final byte[] QUALIFIER = Bytes.toBytes("qual");
+    private static final byte[] ORIGINAL_VALUE = Bytes.toBytes("generic-value");
+    private static final byte[] DUMMY_TAGS = Bytes.toBytes("tags");
+
     @Test
     public void testEncode() {
         assertEquals(VersionUtil.encodeVersion("0.94.5"),VersionUtil.encodeVersion("0.94.5-mapR"));
@@ -65,21 +76,63 @@ public class MetaDataUtilTest {
 
     @Test
     public void testMutatingAPut() throws Exception {
-        String version = VersionInfo.getVersion();
-        KeyValueBuilder builder = KeyValueBuilder.get(version);
-        byte[] row = Bytes.toBytes("row");
-        byte[] family = PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES;
-        byte[] qualifier = Bytes.toBytes("qual");
-        byte[] value = Bytes.toBytes("generic-value");
-        KeyValue kv = builder.buildPut(wrap(row), wrap(family), wrap(qualifier), wrap(value));
-        Put put = new Put(row);
-        KeyValueBuilder.addQuietly(put, builder, kv);
+        Put put = generateOriginalPut();
         byte[] newValue = Bytes.toBytes("new-value");
-        Cell cell = put.get(family, qualifier).get(0);
-        assertEquals(Bytes.toString(value), Bytes.toString(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength()));
-        MetaDataUtil.mutatePutValue(put, family, qualifier, newValue);
-        cell = put.get(family, qualifier).get(0);
-        assertEquals(Bytes.toString(newValue), Bytes.toString(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength()));
+        Cell cell = put.get(TABLE_FAMILY_BYTES, QUALIFIER).get(0);
+        assertEquals(Bytes.toString(ORIGINAL_VALUE),
+                Bytes.toString(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength()));
+        MetaDataUtil.mutatePutValue(put, TABLE_FAMILY_BYTES, QUALIFIER, newValue);
+        cell = put.get(TABLE_FAMILY_BYTES, QUALIFIER).get(0);
+        assertEquals(Bytes.toString(newValue),
+                Bytes.toString(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength()));
+    }
+
+    @Test
+    public void testTaggingAPutWrongQualifier() throws Exception {
+        Put put = generateOriginalPut();
+        Cell initialCell = put.get(TABLE_FAMILY_BYTES, QUALIFIER).get(0);
+
+        // Different qualifier, so no tags should be set
+        MetaDataUtil.conditionallyAddTagsToPutCells(put, TABLE_FAMILY_BYTES, EMPTY_BYTE_ARRAY,
+                EMPTY_BYTE_ARRAY, DUMMY_TAGS);
+        Cell newCell = put.getFamilyCellMap().get(TABLE_FAMILY_BYTES).get(0);
+        assertEquals(initialCell, newCell);
+        assertNull(Tag.carryForwardTags(newCell));
+    }
+
+    @Test
+    public void testTaggingAPutUnconditionally() throws Exception {
+        Put put = generateOriginalPut();
+
+        // valueArray is null so we always set tags
+        MetaDataUtil.conditionallyAddTagsToPutCells(put, TABLE_FAMILY_BYTES, QUALIFIER,
+                null, DUMMY_TAGS);
+        Cell newCell = put.getFamilyCellMap().get(TABLE_FAMILY_BYTES).get(0);
+        assertTrue(Arrays.equals(DUMMY_TAGS, CellUtil.getTagArray(newCell)));
+    }
+
+    @Test
+    public void testSkipTaggingAPutDueToSameCellValue() throws Exception {
+        Put put = generateOriginalPut();
+        Cell initialCell = put.get(TABLE_FAMILY_BYTES, QUALIFIER).get(0);
+
+        // valueArray is set as the value stored in the cell, so we skip tagging the cell
+        MetaDataUtil.conditionallyAddTagsToPutCells(put, TABLE_FAMILY_BYTES, QUALIFIER,
+                ORIGINAL_VALUE, DUMMY_TAGS);
+        Cell newCell = put.getFamilyCellMap().get(TABLE_FAMILY_BYTES).get(0);
+        assertEquals(initialCell, newCell);
+        assertNull(Tag.carryForwardTags(newCell));
+    }
+
+    @Test
+    public void testTaggingAPutDueToDifferentCellValue() throws Exception {
+        Put put = generateOriginalPut();
+
+        // valueArray is set to a value different than the one in the cell, so we tag the cell
+        MetaDataUtil.conditionallyAddTagsToPutCells(put, TABLE_FAMILY_BYTES, QUALIFIER,
+                EMPTY_BYTE_ARRAY, DUMMY_TAGS);
+        Cell newCell = put.getFamilyCellMap().get(TABLE_FAMILY_BYTES).get(0);
+        assertTrue(Arrays.equals(DUMMY_TAGS, CellUtil.getTagArray(newCell)));
     }
 
     /**
@@ -90,31 +143,28 @@ public class MetaDataUtilTest {
   public void testGetMutationKeyValue() throws Exception {
     String version = VersionInfo.getVersion();
     KeyValueBuilder builder = KeyValueBuilder.get(version);
-    byte[] row = Bytes.toBytes("row");
-    byte[] family = PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES;
-    byte[] qualifier = Bytes.toBytes("qual");
-    byte[] value = Bytes.toBytes("generic-value");
-    KeyValue kv = builder.buildPut(wrap(row), wrap(family), wrap(qualifier), wrap(value));
-    Put put = new Put(row);
+    KeyValue kv = builder.buildPut(wrap(ROW), wrap(TABLE_FAMILY_BYTES), wrap(QUALIFIER),
+            wrap(ORIGINAL_VALUE));
+    Put put = new Put(ROW);
     KeyValueBuilder.addQuietly(put, builder, kv);
 
     // read back out the value
     ImmutableBytesPtr ptr = new ImmutableBytesPtr();
-    assertTrue(MetaDataUtil.getMutationValue(put, qualifier, builder, ptr));
+    assertTrue(MetaDataUtil.getMutationValue(put, QUALIFIER, builder, ptr));
     assertEquals("Value returned doesn't match stored value for " + builder.getClass().getName()
         + "!", 0,
-      ByteUtil.BYTES_PTR_COMPARATOR.compare(ptr, wrap(value)));
+      ByteUtil.BYTES_PTR_COMPARATOR.compare(ptr, wrap(ORIGINAL_VALUE)));
 
     // try again, this time with the clientkeyvalue builder
     if (builder != GenericKeyValueBuilder.INSTANCE) {
         builder = GenericKeyValueBuilder.INSTANCE;
-        value = Bytes.toBytes("client-value");
-        kv = builder.buildPut(wrap(row), wrap(family), wrap(qualifier), wrap(value));
-        put = new Put(row);
+        byte[] value = Bytes.toBytes("client-value");
+        kv = builder.buildPut(wrap(ROW), wrap(TABLE_FAMILY_BYTES), wrap(QUALIFIER), wrap(value));
+        put = new Put(ROW);
         KeyValueBuilder.addQuietly(put, builder, kv);
     
         // read back out the value
-        assertTrue(MetaDataUtil.getMutationValue(put, qualifier, builder, ptr));
+        assertTrue(MetaDataUtil.getMutationValue(put, QUALIFIER, builder, ptr));
         assertEquals("Value returned doesn't match stored value for " + builder.getClass().getName()
             + "!", 0,
           ByteUtil.BYTES_PTR_COMPARATOR.compare(ptr, wrap(value)));
@@ -159,5 +209,16 @@ public class MetaDataUtilTest {
                 MetaDataProtocol.PHOENIX_MINOR_VERSION, MetaDataProtocol.PHOENIX_PATCH_NUMBER);
         assertEquals(expectedPhoenixVersion, phoenixVersion);
     }
+
+    private Put generateOriginalPut() {
+        String version = VersionInfo.getVersion();
+        KeyValueBuilder builder = KeyValueBuilder.get(version);
+        KeyValue kv = builder.buildPut(wrap(ROW), wrap(TABLE_FAMILY_BYTES), wrap(QUALIFIER),
+                wrap(ORIGINAL_VALUE));
+        Put put = new Put(ROW);
+        KeyValueBuilder.addQuietly(put, builder, kv);
+        return put;
+    }
+
 }
 
diff --git a/phoenix-protocol/src/main/MetaDataService.proto b/phoenix-protocol/src/main/MetaDataService.proto
index f10aa3c..290a65d 100644
--- a/phoenix-protocol/src/main/MetaDataService.proto
+++ b/phoenix-protocol/src/main/MetaDataService.proto
@@ -61,7 +61,7 @@ message SharedTableState {
   repeated PColumn columns = 4;  
   repeated bytes physicalNames = 5;
   required int64 viewIndexId = 6;
-  optional int32 viewIndexType = 7 [default = 5];
+  optional int32 viewIndexIdType = 7 [default = 5];
 }
 
 message MetaDataResponse {
@@ -77,7 +77,7 @@ message MetaDataResponse {
   optional PSchema schema = 10;
   optional int64 autoPartitionNum = 11;
   optional int64 viewIndexId = 12;
-  optional int32 viewIndexType = 13 [default = 5];
+  optional int32 viewIndexIdType = 13 [default = 5];
 }
 
 message GetTableRequest {
diff --git a/phoenix-protocol/src/main/PTable.proto b/phoenix-protocol/src/main/PTable.proto
index be771a9..12dde49 100644
--- a/phoenix-protocol/src/main/PTable.proto
+++ b/phoenix-protocol/src/main/PTable.proto
@@ -103,7 +103,7 @@ message PTable {
   repeated EncodedCQCounter encodedCQCounters = 36;
   optional bool useStatsForParallelization = 37;
   optional int32 transactionProvider = 38;
-  optional int32 viewIndexType = 39 [default = 5];
+  optional int32 viewIndexIdType = 39 [default = 5];
 }
 
 message EncodedCQCounter {
diff --git a/phoenix-protocol/src/main/ServerCachingService.proto b/phoenix-protocol/src/main/ServerCachingService.proto
index 0d2d1d2..5891d25 100644
--- a/phoenix-protocol/src/main/ServerCachingService.proto
+++ b/phoenix-protocol/src/main/ServerCachingService.proto
@@ -62,7 +62,7 @@ message IndexMaintainer {
   repeated ColumnInfo indexedColumnInfo = 19;
   required int32 encodingScheme = 20;
   required int32 immutableStorageScheme = 21;
-  optional int32 viewIndexType = 22 ;
+  optional int32 viewIndexIdType = 22 ;
 }
 
 message AddServerCacheRequest {


[phoenix] 14/18: PHOENIX-5084 Changes from Transactional Tables are not visible to query in different client.

Posted by pb...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

pboado pushed a commit to branch 4.x-cdh5.16
in repository https://gitbox.apache.org/repos/asf/phoenix.git

commit 58732144d28e1af4143b6554c0f01f7e1e0f1669
Author: Lars Hofhansl <la...@apache.org>
AuthorDate: Wed Jan 2 08:52:52 2019 +0000

    PHOENIX-5084 Changes from Transactional Tables are not visible to query in different client.
---
 .../org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java     | 12 ++++++++++++
 1 file changed, 12 insertions(+)

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 3ff62e2..61ba0fc 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
@@ -718,6 +718,7 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData {
     @Override
     public ResultSet getColumns(String catalog, String schemaPattern, String tableNamePattern, String columnNamePattern)
             throws SQLException {
+        try {
         boolean isTenantSpecificConnection = connection.getTenantId() != null;
         List<Tuple> tuples = Lists.newArrayListWithExpectedSize(10);
         ResultSet rs = getTables(catalog, schemaPattern, tableNamePattern, null);
@@ -893,6 +894,11 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData {
             }
         }
         return new PhoenixResultSet(new MaterializedResultIterator(tuples), GET_COLUMNS_ROW_PROJECTOR, new StatementContext(new PhoenixStatement(connection), false));
+        } finally {
+            if (connection.getAutoCommit()) {
+                connection.commit();
+            }
+        }
     }
 
     @Override
@@ -1142,6 +1148,7 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData {
         if (tableName == null || tableName.length() == 0) {
             return emptyResultSet;
         }
+        try {
         List<Tuple> tuples = Lists.newArrayListWithExpectedSize(10);
         ResultSet rs = getTables(catalog, schemaName, tableName, null);
         while (rs.next()) {
@@ -1219,6 +1226,11 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData {
         return new PhoenixResultSet(new MaterializedResultIterator(tuples),
                 GET_PRIMARY_KEYS_ROW_PROJECTOR,
                 new StatementContext(new PhoenixStatement(connection), false));
+        } finally {
+            if (connection.getAutoCommit()) {
+                connection.commit();
+            }
+        }
     }
 
     @Override


[phoenix] 03/18: PHOENIX-4832: Add Canary Test Tool for Phoenix Query Server.

Posted by pb...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

pboado pushed a commit to branch 4.x-cdh5.16
in repository https://gitbox.apache.org/repos/asf/phoenix.git

commit 1d9073c1a326f4317b0ee2960668c90f0234b003
Author: s.kadam <s....@gus.com>
AuthorDate: Thu Dec 6 00:11:07 2018 +0000

    PHOENIX-4832: Add Canary Test Tool for Phoenix Query Server.
---
 phoenix-core/pom.xml                               |   7 +
 .../org/apache/phoenix/tool/CanaryTestResult.java  |  86 ++++
 .../org/apache/phoenix/tool/PhoenixCanaryTool.java | 477 +++++++++++++++++++++
 .../resources/phoenix-canary-file-sink.properties  |  17 +
 .../apache/phoenix/tool/PhoenixCanaryToolTest.java | 140 ++++++
 5 files changed, 727 insertions(+)

diff --git a/phoenix-core/pom.xml b/phoenix-core/pom.xml
index 97091b9..f8112fe 100644
--- a/phoenix-core/pom.xml
+++ b/phoenix-core/pom.xml
@@ -26,6 +26,7 @@
 
   <properties>
     <top.dir>${project.basedir}/..</top.dir>
+    <argparse4j.version>0.8.1</argparse4j.version>
   </properties>
 
   <build>
@@ -228,6 +229,12 @@
       <artifactId>sqlline</artifactId>
     </dependency>
     <dependency>
+      <groupId>net.sourceforge.argparse4j</groupId>
+      <artifactId>argparse4j</artifactId>
+      <version>${argparse4j.version}</version>
+    </dependency>
+
+    <dependency>
       <groupId>com.google.guava</groupId>
       <artifactId>guava</artifactId>
     </dependency>
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/tool/CanaryTestResult.java b/phoenix-core/src/main/java/org/apache/phoenix/tool/CanaryTestResult.java
new file mode 100644
index 0000000..b72439c
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/tool/CanaryTestResult.java
@@ -0,0 +1,86 @@
+/*
+ * 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.tool;
+
+public class CanaryTestResult {
+
+    private boolean isSuccessful;
+    private long startTime;
+    private long executionTime;
+    private String message;
+    private String testName;
+    private String timestamp;
+    private Object miscellaneous;
+
+    public Object getMiscellaneous() {
+        return miscellaneous;
+    }
+
+    public void setMiscellaneous(Object miscellaneous) {
+        this.miscellaneous = miscellaneous;
+    }
+
+    public long getStartTime() {
+        return startTime;
+    }
+
+    public void setStartTime(long startTime) {
+        this.startTime = startTime;
+    }
+
+    public String getTimestamp() {
+        return timestamp;
+    }
+
+    public void setTimestamp(String timestamp) {
+        this.timestamp = timestamp;
+    }
+
+    public boolean isSuccessful() {
+        return isSuccessful;
+    }
+
+    public void setSuccessful(boolean successful) {
+        isSuccessful = successful;
+    }
+
+    public long getExecutionTime() {
+        return executionTime;
+    }
+
+    public void setExecutionTime(long executionTime) {
+        this.executionTime = executionTime;
+    }
+
+    public String getMessage() {
+        return message;
+    }
+
+    public void setMessage(String message) {
+        this.message = message;
+    }
+
+    public String getTestName() {
+        return testName;
+    }
+
+    public void setTestName(String testName) {
+        this.testName = testName;
+    }
+
+}
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/tool/PhoenixCanaryTool.java b/phoenix-core/src/main/java/org/apache/phoenix/tool/PhoenixCanaryTool.java
new file mode 100644
index 0000000..405f54f
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/tool/PhoenixCanaryTool.java
@@ -0,0 +1,477 @@
+/*
+ * 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.tool;
+
+import com.google.common.base.Throwables;
+import com.google.common.io.Files;
+import com.google.common.util.concurrent.SimpleTimeLimiter;
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import net.sourceforge.argparse4j.ArgumentParsers;
+import net.sourceforge.argparse4j.inf.ArgumentParser;
+import net.sourceforge.argparse4j.inf.ArgumentParserException;
+import net.sourceforge.argparse4j.inf.Namespace;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import java.io.File;
+import java.io.InputStream;
+import java.sql.Connection;
+import java.sql.DatabaseMetaData;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.Statement;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.List;
+import java.util.Properties;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * A Canary Tool to perform synthetic tests for Query Server
+ */
+public class PhoenixCanaryTool extends Configured implements Tool {
+
+    private static String TEST_SCHEMA_NAME = "TEST";
+    private static String TEST_TABLE_NAME = "PQSTEST";
+    private static String FQ_TABLE_NAME = "TEST.PQSTEST";
+    private boolean USE_NAMESPACE = true;
+
+    private Sink sink = new StdOutSink();
+
+    /**
+     * Base class for a Canary Test
+     */
+    private abstract static class CanaryTest {
+
+        CanaryTestResult result = new CanaryTestResult();
+
+        Connection connection = null;
+
+        private void onCreate(Connection connection) {
+            result.setTimestamp(getCurrentTimestamp());
+            result.setStartTime(System.currentTimeMillis());
+            this.connection = connection;
+        }
+
+        abstract void onExecute() throws Exception;
+
+        private void onExit() {
+            result.setExecutionTime(System.currentTimeMillis() - result.getStartTime());
+        }
+
+        CanaryTestResult runTest(Connection connection) {
+            try {
+                onCreate(connection);
+                onExecute();
+                result.setSuccessful(true);
+                result.setMessage("Test " + result.getTestName() + " successful");
+            } catch (Exception e) {
+                result.setSuccessful(false);
+                result.setMessage(Throwables.getStackTraceAsString(e));
+            } finally {
+                onExit();
+            }
+            return result;
+        }
+    }
+
+    /**
+     * Test which prepares environment before other tests run
+     */
+    static class PrepareTest extends CanaryTest {
+        void onExecute() throws Exception {
+            result.setTestName("prepare");
+            Statement statement = connection.createStatement();
+            DatabaseMetaData dbm = connection.getMetaData();
+            ResultSet tables = dbm.getTables(null, TEST_SCHEMA_NAME, TEST_TABLE_NAME, null);
+            if (tables.next()) {
+                // Drop test Table if exists
+                statement.executeUpdate("DROP TABLE IF EXISTS " + FQ_TABLE_NAME);
+            }
+
+            // Drop test schema if exists
+            if (TEST_SCHEMA_NAME != null) {
+                statement = connection.createStatement();
+                statement.executeUpdate("DROP SCHEMA IF EXISTS " + TEST_SCHEMA_NAME);
+            }
+        }
+    }
+
+    /**
+     * Create Schema Test
+     */
+    static class CreateSchemaTest extends CanaryTest {
+        void onExecute() throws Exception {
+            result.setTestName("createSchema");
+            Statement statement = connection.createStatement();
+            statement.executeUpdate("CREATE SCHEMA IF NOT EXISTS " + TEST_SCHEMA_NAME);
+        }
+    }
+
+    /**
+     * Create Table Test
+     */
+    static class CreateTableTest extends CanaryTest {
+        void onExecute() throws Exception {
+            result.setTestName("createTable");
+            Statement statement = connection.createStatement();
+            // Create Table
+            statement.executeUpdate("CREATE TABLE IF NOT EXISTS" + FQ_TABLE_NAME + " (mykey " + "INTEGER "
+                    + "NOT " + "NULL PRIMARY KEY, " + "mycolumn VARCHAR)");
+        }
+    }
+
+    /**
+     * Upsert Data into Table Test
+     */
+    static class UpsertTableTest extends CanaryTest {
+        void onExecute() throws Exception {
+            result.setTestName("upsertTable");
+            // Insert data
+            Statement statement = connection.createStatement();
+            statement.executeUpdate("UPSERT INTO " + FQ_TABLE_NAME + " VALUES (1, " +
+                    "'Hello" + " World')");
+            connection.commit();
+        }
+    }
+
+    /**
+     * Read data from Table Test
+     */
+    static class ReadTableTest extends CanaryTest {
+        void onExecute() throws Exception {
+            result.setTestName("readTable");
+            // Query for table
+            PreparedStatement ps = connection.prepareStatement("SELECT * FROM " + FQ_TABLE_NAME);
+            ResultSet rs = ps.executeQuery();
+
+            // Check correctness
+            int totalRows = 0;
+            while (rs.next()) {
+                totalRows += 1;
+                Integer myKey = rs.getInt(1);
+                String myColumn = rs.getString(2);
+                if (myKey != 1 || !myColumn.equals("Hello World")) {
+                    throw new Exception("Retrieved values do not match the inserted " + "values");
+                }
+            }
+            if (totalRows != 1) {
+                throw new Exception(totalRows + " rows fetched instead of just one.");
+            }
+            ps.close();
+            rs.close();
+        }
+    }
+
+    /**
+     * Delete test table Test
+     */
+    static class DeleteTableTest extends CanaryTest {
+        void onExecute() throws Exception {
+            result.setTestName("deleteTable");
+            Statement statement = connection.createStatement();
+            statement.executeUpdate("DROP TABLE IF EXISTS" + FQ_TABLE_NAME);
+
+            // Check if table dropped
+            DatabaseMetaData dbm = connection.getMetaData();
+            ResultSet tables = dbm.getTables(null, TEST_SCHEMA_NAME, TEST_TABLE_NAME, null);
+            if (tables.next()) {
+                throw new Exception("Test Table could not be dropped");
+            }
+        }
+    }
+
+    /**
+     * Delete test Schema Test
+     */
+    static class DeleteSchemaTest extends CanaryTest {
+        void onExecute() throws Exception {
+            result.setTestName("deleteSchema");
+            Statement statement = connection.createStatement();
+            statement.executeUpdate("DROP SCHEMA IF EXISTS " + TEST_SCHEMA_NAME);
+        }
+    }
+
+    /**
+     * Sink interface used by the canary to output information
+     */
+    public interface Sink {
+        List<CanaryTestResult> getResults();
+
+        void updateResults(CanaryTestResult result);
+
+        void publishResults() throws Exception;
+
+        void clearResults();
+    }
+
+    /**
+     * Implementation of Std Out Sink
+     */
+    public static class StdOutSink implements Sink {
+        private List<CanaryTestResult> results = new ArrayList<>();
+
+        @Override
+        public void updateResults(CanaryTestResult result) {
+            results.add(result);
+        }
+
+        @Override
+        public List<CanaryTestResult> getResults() {
+            return results;
+        }
+
+        @Override
+        public void publishResults() throws Exception {
+
+            Gson gson = new GsonBuilder().setPrettyPrinting().create();
+            String resultJson = gson.toJson(results);
+            System.out.println(resultJson);
+        }
+
+        @Override
+        public void clearResults() {
+            results.clear();
+        }
+    }
+
+    /**
+     * Implementation of File Out Sink
+     */
+    public static class FileOutSink implements Sink {
+        private List<CanaryTestResult> results = new ArrayList<>();
+        File dir;
+        String logfileName;
+        String propFileName = "phoenix-canary-file-sink.properties";
+
+        public FileOutSink() throws Exception {
+            Properties prop = new Properties();
+            InputStream input = ClassLoader.getSystemResourceAsStream(propFileName);
+            if (input == null) {
+                throw new Exception("Cannot load " + propFileName + " file for " + "FileOutSink.");
+            }
+            prop.load(input);
+            logfileName = prop.getProperty("file.name");
+            dir = new File(prop.getProperty("file.location"));
+            dir.mkdirs();
+        }
+
+        @Override
+        public void updateResults(CanaryTestResult result) {
+            results.add(result);
+        }
+
+        @Override
+        public List<CanaryTestResult> getResults() {
+            return results;
+        }
+
+        @Override
+        public void publishResults() throws Exception {
+            Gson gson = new GsonBuilder().setPrettyPrinting().create();
+            String resultJson = gson.toJson(results);
+            String fileName = logfileName + "-" + new SimpleDateFormat("yyyy.MM.dd.HH" + ".mm" +
+                    ".ss").format(new Date()) + ".log";
+            File file = new File(dir, fileName);
+            Files.write(Bytes.toBytes(resultJson), file);
+        }
+
+        @Override
+        public void clearResults() {
+            results.clear();
+        }
+    }
+
+    private static final Logger LOG = LoggerFactory.getLogger(PhoenixCanaryTool.class);
+
+    private static String getCurrentTimestamp() {
+        return new SimpleDateFormat("yyyy.MM.dd.HH.mm.ss.ms").format(new Date());
+    }
+
+    private static Namespace parseArgs(String[] args) {
+
+        ArgumentParser parser = ArgumentParsers.newFor("Phoenix Canary Test Tool").build()
+                .description("Phoenix Canary Test Tool");
+
+        parser.addArgument("--hostname", "-hn").type(String.class).nargs("?").help("Hostname on "
+                + "which Phoenix is running.");
+
+        parser.addArgument("--port", "-p").type(String.class).nargs("?").help("Port on " +
+                "which Phoenix is running.");
+
+        parser.addArgument("--constring", "-cs").type(String.class).nargs("?").help("Pass an " +
+                "explicit connection String to connect to Phoenix. " +
+                "default: jdbc:phoenix:thin:serialization=PROTOBUF;url=[hostName:port]");
+
+        parser.addArgument("--timeout", "-t").type(String.class).nargs("?").setDefault("60").help
+                ("Maximum time for which the app should run before returning error. default:" + "" +
+                        " 60 sec");
+
+        parser.addArgument("--testschema", "-ts").type(String.class).nargs("?").setDefault
+                (TEST_SCHEMA_NAME).help("Custom name for the test table. " + "default: " +
+                TEST_SCHEMA_NAME);
+
+        parser.addArgument("--testtable", "-tt").type(String.class).nargs("?").setDefault
+                (TEST_TABLE_NAME).help("Custom name for the test table." + " default: " +
+                TEST_TABLE_NAME);
+
+        parser.addArgument("--logsinkclass", "-lsc").type(String.class).nargs("?").setDefault
+                ("PhoenixCanaryTool$StdOutSink").help
+                ("Path to a Custom implementation for log sink class. default: stdout");
+
+        Namespace res = null;
+        try {
+            res = parser.parseKnownArgs(args, null);
+        } catch (ArgumentParserException e) {
+            parser.handleError(e);
+        }
+        return res;
+    }
+
+    private CanaryTestResult appInfo = new CanaryTestResult();
+    private Connection connection = null;
+
+    @Override
+    public int run(String[] args) throws Exception {
+
+        try {
+            Namespace cArgs = parseArgs(args);
+            if (cArgs == null) {
+                LOG.error("Argument parsing failed.");
+                throw new RuntimeException("Argument parsing failed");
+            }
+
+            final String hostName = cArgs.getString("hostname");
+            final String port = cArgs.getString("port");
+            final String timeout = cArgs.getString("timeout");
+            final String conString = cArgs.getString("constring");
+            final String testSchemaName = cArgs.getString("testschema");
+            final String testTableName = cArgs.getString("testtable");
+            final String logSinkClass = cArgs.getString("logsinkclass");
+
+            TEST_TABLE_NAME = testTableName;
+            TEST_SCHEMA_NAME = testSchemaName;
+            FQ_TABLE_NAME = testSchemaName + "." + testTableName;
+
+            // Check if at least one from host+port or con string is provided.
+            if ((hostName == null || port == null) && conString == null) {
+                throw new RuntimeException("Provide at least one from host+port or constring");
+            }
+
+            int timeoutVal = Integer.parseInt(timeout);
+
+            // Dynamically load a class for sink
+            sink = (Sink) ClassLoader.getSystemClassLoader().loadClass(logSinkClass).newInstance();
+
+            long startTime = System.currentTimeMillis();
+
+            String connectionURL = (conString != null) ? conString :
+                    "jdbc:phoenix:thin:serialization=PROTOBUF;url=" + hostName + ":" + port;
+
+            appInfo.setTestName("appInfo");
+            appInfo.setMiscellaneous(connectionURL);
+
+            Properties connProps = new Properties();
+            connProps.setProperty("phoenix.schema.mapSystemTablesToNamespace", "true");
+            connProps.setProperty("phoenix.schema.isNamespaceMappingEnabled", "true");
+
+            try {
+                connection = DriverManager.getConnection(connectionURL, connProps);
+            } catch (Exception e) {
+                LOG.info("Namespace mapping cannot be set. Using default schema");
+                USE_NAMESPACE = false;
+                connection = DriverManager.getConnection(connectionURL);
+                TEST_SCHEMA_NAME = null;
+                FQ_TABLE_NAME = TEST_TABLE_NAME;
+            }
+
+            SimpleTimeLimiter limiter = new SimpleTimeLimiter();
+
+            limiter.callWithTimeout(new Callable<Void>() {
+
+                public Void call() {
+
+                    sink.clearResults();
+
+                    // Execute tests
+
+                    LOG.info("Starting PrepareTest");
+                    sink.updateResults(new PrepareTest().runTest(connection));
+
+                    if (USE_NAMESPACE) {
+                        LOG.info("Starting CreateSchemaTest");
+                        sink.updateResults(new CreateSchemaTest().runTest(connection));
+                    }
+
+                    LOG.info("Starting CreateTableTest");
+                    sink.updateResults(new CreateTableTest().runTest(connection));
+
+                    LOG.info("Starting UpsertTableTest");
+                    sink.updateResults(new UpsertTableTest().runTest(connection));
+
+                    LOG.info("Starting ReadTableTest");
+                    sink.updateResults(new ReadTableTest().runTest(connection));
+
+                    LOG.info("Starting DeleteTableTest");
+                    sink.updateResults(new DeleteTableTest().runTest(connection));
+
+                    if (USE_NAMESPACE) {
+                        LOG.info("Starting DeleteSchemaTest");
+                        sink.updateResults(new DeleteSchemaTest().runTest(connection));
+                    }
+                    return null;
+                }
+            }, timeoutVal, TimeUnit.SECONDS, true);
+
+            long estimatedTime = System.currentTimeMillis() - startTime;
+
+            appInfo.setExecutionTime(estimatedTime);
+            appInfo.setSuccessful(true);
+
+        } catch (Exception e) {
+            LOG.error(Throwables.getStackTraceAsString(e));
+            appInfo.setMessage(Throwables.getStackTraceAsString(e));
+            appInfo.setSuccessful(false);
+
+        } finally {
+            sink.updateResults(appInfo);
+            sink.publishResults();
+            connection.close();
+        }
+
+        return 0;
+    }
+
+    public static void main(final String[] args) {
+        int result = 0;
+        try {
+            LOG.info("Starting Phoenix Canary Test tool...");
+            result = ToolRunner.run(new PhoenixCanaryTool(), args);
+        } catch (Exception e) {
+            LOG.error("Error in running Phoenix Canary Test tool. " + e);
+        }
+        LOG.info("Exiting Phoenix Canary Test tool...");
+    }
+}
diff --git a/phoenix-core/src/main/resources/phoenix-canary-file-sink.properties b/phoenix-core/src/main/resources/phoenix-canary-file-sink.properties
new file mode 100644
index 0000000..3c1ebae
--- /dev/null
+++ b/phoenix-core/src/main/resources/phoenix-canary-file-sink.properties
@@ -0,0 +1,17 @@
+# 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.
+
+file.name=phoenix-canary-logfile
+file.location=/tmp/logs/phoenix
\ No newline at end of file
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/tool/PhoenixCanaryToolTest.java b/phoenix-core/src/test/java/org/apache/phoenix/tool/PhoenixCanaryToolTest.java
new file mode 100644
index 0000000..bd2a3d1
--- /dev/null
+++ b/phoenix-core/src/test/java/org/apache/phoenix/tool/PhoenixCanaryToolTest.java
@@ -0,0 +1,140 @@
+/*
+ * 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.tool;
+
+import static org.junit.Assert.assertEquals;
+
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mock;
+import org.mockito.Mockito;
+import static org.mockito.Mockito.when;
+import org.mockito.MockitoAnnotations;
+import java.sql.Connection;
+import java.sql.DatabaseMetaData;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.Statement;
+
+public class PhoenixCanaryToolTest {
+
+    @Mock
+    private Connection connection;
+
+    @Mock
+    private Statement statement;
+
+    @Mock
+    private PreparedStatement ps;
+
+    @Mock
+    private ResultSet rs;
+
+    @Mock
+    private DatabaseMetaData dbm;
+
+    @Before
+    public void setUp() {
+        MockitoAnnotations.initMocks(this);
+    }
+
+    @Test
+    public void prepareTest() throws Exception {
+        when(connection.createStatement()).thenReturn(statement);
+        when(statement.executeUpdate(Mockito.anyString())).thenReturn(0);
+        when(connection.getMetaData()).thenReturn(dbm);
+        when(dbm.getTables(null, "TEST", "PQSTEST", null)).thenReturn(rs);
+        when(rs.next()).thenReturn(true).thenReturn(false);
+        when(connection.createStatement()).thenReturn(statement);
+        when(statement.executeUpdate(Mockito.anyString())).thenReturn(0);
+        CanaryTestResult result = new PhoenixCanaryTool.PrepareTest().runTest(connection);
+        assertEquals(true, result.isSuccessful());
+        assertEquals("Test prepare successful", result.getMessage());
+    }
+
+    @Test
+    public void createSchemaTest() throws Exception {
+        when(connection.createStatement()).thenReturn(statement);
+        when(statement.executeUpdate(Mockito.anyString())).thenReturn(0);
+        CanaryTestResult result = new PhoenixCanaryTool.CreateSchemaTest().runTest(connection);
+        assertEquals(true, result.isSuccessful());
+        assertEquals("Test createSchema successful", result.getMessage());
+    }
+
+    @Test
+    public void createTableTest() throws Exception {
+        when(connection.createStatement()).thenReturn(statement);
+        when(statement.executeUpdate(Mockito.anyString())).thenReturn(0);
+        CanaryTestResult result = new PhoenixCanaryTool.CreateTableTest().runTest(connection);
+        assertEquals(true, result.isSuccessful());
+        assertEquals("Test createTable successful", result.getMessage());
+    }
+
+    @Test
+    public void upsertTableTest() throws Exception {
+        when(connection.createStatement()).thenReturn(statement);
+        when(statement.executeUpdate(Mockito.anyString())).thenReturn(1);
+        CanaryTestResult result = new PhoenixCanaryTool.UpsertTableTest().runTest(connection);
+        assertEquals(true, result.isSuccessful());
+        assertEquals("Test upsertTable successful", result.getMessage());
+    }
+
+    @Test
+    public void readTableTest() throws Exception {
+        when(connection.prepareStatement(Mockito.anyString())).thenReturn(ps);
+        when(ps.executeQuery()).thenReturn(rs);
+        when(rs.next()).thenReturn(true).thenReturn(false);
+        when(rs.getInt(1)).thenReturn(1);
+        when(rs.getString(2)).thenReturn("Hello World");
+        CanaryTestResult result = new PhoenixCanaryTool.ReadTableTest().runTest(connection);
+        assertEquals(true, result.isSuccessful());
+        assertEquals("Test readTable successful", result.getMessage());
+    }
+
+    @Test
+    public void deleteTableTest() throws Exception {
+        when(connection.createStatement()).thenReturn(statement);
+        when(statement.executeUpdate(Mockito.anyString())).thenReturn(0);
+        when(connection.getMetaData()).thenReturn(dbm);
+        when(dbm.getTables(null, "TEST", "PQSTEST", null)).thenReturn(rs);
+        CanaryTestResult result = new PhoenixCanaryTool.DeleteTableTest().runTest(connection);
+        assertEquals(true, result.isSuccessful());
+        assertEquals("Test deleteTable successful", result.getMessage());
+    }
+
+    @Test
+    public void deleteSchemaTest() throws Exception {
+        when(connection.createStatement()).thenReturn(statement);
+        when(statement.executeUpdate(Mockito.anyString())).thenReturn(0);
+        CanaryTestResult result = new PhoenixCanaryTool.DeleteSchemaTest().runTest(connection);
+        assertEquals(true, result.isSuccessful());
+        assertEquals("Test deleteSchema successful", result.getMessage());
+    }
+
+    @Test
+    public void failTest() throws Exception {
+        when(connection.prepareStatement(Mockito.anyString())).thenReturn(ps);
+        when(ps.executeQuery()).thenReturn(rs);
+        when(rs.getInt(1)).thenReturn(3);
+        when(rs.getString(2)).thenReturn("Incorrect data");
+        when(rs.next()).thenReturn(true).thenReturn(false);
+        CanaryTestResult result = new PhoenixCanaryTool.ReadTableTest().runTest(connection);
+        assertEquals(false, result.isSuccessful());
+        assert (result.getMessage().contains("Retrieved values do not match the inserted values"));
+    }
+}
\ No newline at end of file


[phoenix] 17/18: Changes for CDH 5.16.x

Posted by pb...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

pboado pushed a commit to branch 4.x-cdh5.16
in repository https://gitbox.apache.org/repos/asf/phoenix.git

commit cea84e84d5b294bc1205a9b673037984b700ce63
Author: pboado <pe...@gmail.com>
AuthorDate: Tue May 28 23:45:56 2019 +0100

    Changes for CDH 5.16.x
---
 phoenix-assembly/pom.xml           |  2 +-
 phoenix-client/pom.xml             |  2 +-
 phoenix-core/pom.xml               |  2 +-
 phoenix-flume/pom.xml              |  2 +-
 phoenix-hive/pom.xml               |  2 +-
 phoenix-kafka/pom.xml              |  2 +-
 phoenix-load-balancer/pom.xml      |  2 +-
 phoenix-parcel/pom.xml             |  2 +-
 phoenix-pherf/pom.xml              |  2 +-
 phoenix-pig/pom.xml                |  2 +-
 phoenix-queryserver-client/pom.xml |  2 +-
 phoenix-queryserver/pom.xml        |  2 +-
 phoenix-server/pom.xml             |  2 +-
 phoenix-spark/pom.xml              |  2 +-
 phoenix-tracing-webapp/pom.xml     |  2 +-
 pom.xml                            | 10 +++++-----
 16 files changed, 20 insertions(+), 20 deletions(-)

diff --git a/phoenix-assembly/pom.xml b/phoenix-assembly/pom.xml
index 300b4f6..5c2aeb5 100644
--- a/phoenix-assembly/pom.xml
+++ b/phoenix-assembly/pom.xml
@@ -27,7 +27,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.15.0-cdh5.15.1</version>
+    <version>4.15.0-cdh5.16.2</version>
   </parent>
   <artifactId>phoenix-assembly</artifactId>
   <name>Phoenix Assembly</name>
diff --git a/phoenix-client/pom.xml b/phoenix-client/pom.xml
index cfed3ce..3028c81 100644
--- a/phoenix-client/pom.xml
+++ b/phoenix-client/pom.xml
@@ -27,7 +27,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.15.0-cdh5.15.1</version>
+    <version>4.15.0-cdh5.16.2</version>
   </parent>
   <artifactId>phoenix-client</artifactId>
   <name>Phoenix Client</name>
diff --git a/phoenix-core/pom.xml b/phoenix-core/pom.xml
index 8caf88f..043505a 100644
--- a/phoenix-core/pom.xml
+++ b/phoenix-core/pom.xml
@@ -4,7 +4,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.15.0-cdh5.15.1</version>
+    <version>4.15.0-cdh5.16.2</version>
   </parent>
   <artifactId>phoenix-core</artifactId>
   <name>Phoenix Core</name>
diff --git a/phoenix-flume/pom.xml b/phoenix-flume/pom.xml
index c67de23..5711714 100644
--- a/phoenix-flume/pom.xml
+++ b/phoenix-flume/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.15.0-cdh5.15.1</version>
+    <version>4.15.0-cdh5.16.2</version>
   </parent>
   <artifactId>phoenix-flume</artifactId>
   <name>Phoenix - Flume</name>
diff --git a/phoenix-hive/pom.xml b/phoenix-hive/pom.xml
index 50670e0..8af7c16 100644
--- a/phoenix-hive/pom.xml
+++ b/phoenix-hive/pom.xml
@@ -27,7 +27,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.15.0-cdh5.15.1</version>
+    <version>4.15.0-cdh5.16.2</version>
   </parent>
   <artifactId>phoenix-hive</artifactId>
   <name>Phoenix - Hive</name>
diff --git a/phoenix-kafka/pom.xml b/phoenix-kafka/pom.xml
index 460eb5a..6da5a58 100644
--- a/phoenix-kafka/pom.xml
+++ b/phoenix-kafka/pom.xml
@@ -26,7 +26,7 @@
 	<parent>
 		<groupId>org.apache.phoenix</groupId>
 		<artifactId>phoenix</artifactId>
-		<version>4.15.0-cdh5.15.1</version>
+		<version>4.15.0-cdh5.16.2</version>
 	</parent>
 	<artifactId>phoenix-kafka</artifactId>
 	<name>Phoenix - Kafka</name>
diff --git a/phoenix-load-balancer/pom.xml b/phoenix-load-balancer/pom.xml
index a8319e9..a59ee06 100644
--- a/phoenix-load-balancer/pom.xml
+++ b/phoenix-load-balancer/pom.xml
@@ -27,7 +27,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.15.0-cdh5.15.1</version>
+    <version>4.15.0-cdh5.16.2</version>
   </parent>
   <artifactId>phoenix-load-balancer</artifactId>
   <name>Phoenix Load Balancer</name>
diff --git a/phoenix-parcel/pom.xml b/phoenix-parcel/pom.xml
index 417a2db..eb2f254 100644
--- a/phoenix-parcel/pom.xml
+++ b/phoenix-parcel/pom.xml
@@ -27,7 +27,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.15.0-cdh5.15.1</version>
+    <version>4.15.0-cdh5.16.2</version>
   </parent>
   <artifactId>phoenix-parcel</artifactId>
   <name>Phoenix Parcels for CDH</name>
diff --git a/phoenix-pherf/pom.xml b/phoenix-pherf/pom.xml
index cb648e4..340bb58 100644
--- a/phoenix-pherf/pom.xml
+++ b/phoenix-pherf/pom.xml
@@ -15,7 +15,7 @@
 	<parent>
 		<groupId>org.apache.phoenix</groupId>
 		<artifactId>phoenix</artifactId>
-		<version>4.15.0-cdh5.15.1</version>
+		<version>4.15.0-cdh5.16.2</version>
 	</parent>
 
 	<artifactId>phoenix-pherf</artifactId>
diff --git a/phoenix-pig/pom.xml b/phoenix-pig/pom.xml
index 48ffb91..8f96d6f 100644
--- a/phoenix-pig/pom.xml
+++ b/phoenix-pig/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.15.0-cdh5.15.1</version>
+    <version>4.15.0-cdh5.16.2</version>
   </parent>
   <artifactId>phoenix-pig</artifactId>
   <name>Phoenix - Pig</name>
diff --git a/phoenix-queryserver-client/pom.xml b/phoenix-queryserver-client/pom.xml
index a87d338..ea386d7 100644
--- a/phoenix-queryserver-client/pom.xml
+++ b/phoenix-queryserver-client/pom.xml
@@ -27,7 +27,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.15.0-cdh5.15.1</version>
+    <version>4.15.0-cdh5.16.2</version>
   </parent>
   <artifactId>phoenix-queryserver-client</artifactId>
   <name>Phoenix Query Server Client</name>
diff --git a/phoenix-queryserver/pom.xml b/phoenix-queryserver/pom.xml
index f91fce5..0a19b6d 100644
--- a/phoenix-queryserver/pom.xml
+++ b/phoenix-queryserver/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.15.0-cdh5.15.1</version>
+    <version>4.15.0-cdh5.16.2</version>
   </parent>
   <artifactId>phoenix-queryserver</artifactId>
   <name>Phoenix Query Server</name>
diff --git a/phoenix-server/pom.xml b/phoenix-server/pom.xml
index def100c..18a5ab9 100644
--- a/phoenix-server/pom.xml
+++ b/phoenix-server/pom.xml
@@ -27,7 +27,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.15.0-cdh5.15.1</version>
+    <version>4.15.0-cdh5.16.2</version>
   </parent>
   <artifactId>phoenix-server</artifactId>
   <name>Phoenix Server</name>
diff --git a/phoenix-spark/pom.xml b/phoenix-spark/pom.xml
index 264ac77..e2790bd 100644
--- a/phoenix-spark/pom.xml
+++ b/phoenix-spark/pom.xml
@@ -28,7 +28,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.15.0-cdh5.15.1</version>
+    <version>4.15.0-cdh5.16.2</version>
   </parent>
   <artifactId>phoenix-spark</artifactId>
   <name>Phoenix - Spark</name>
diff --git a/phoenix-tracing-webapp/pom.xml b/phoenix-tracing-webapp/pom.xml
index d765b54..85a8d39 100755
--- a/phoenix-tracing-webapp/pom.xml
+++ b/phoenix-tracing-webapp/pom.xml
@@ -27,7 +27,7 @@
     <parent>
       <groupId>org.apache.phoenix</groupId>
       <artifactId>phoenix</artifactId>
-      <version>4.15.0-cdh5.15.1</version>
+      <version>4.15.0-cdh5.16.2</version>
     </parent>
 
     <artifactId>phoenix-tracing-webapp</artifactId>
diff --git a/pom.xml b/pom.xml
index 74f3381..a863f97 100644
--- a/pom.xml
+++ b/pom.xml
@@ -3,7 +3,7 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>org.apache.phoenix</groupId>
   <artifactId>phoenix</artifactId>
-  <version>4.15.0-cdh5.15.1</version>
+  <version>4.15.0-cdh5.16.2</version>
   <packaging>pom</packaging>
   <name>Apache Phoenix</name>
   <description>A SQL layer over HBase</description>
@@ -86,7 +86,7 @@
   <parent>
     <groupId>com.cloudera.cdh</groupId>
     <artifactId>cdh-root</artifactId>
-    <version>5.15.1</version>
+    <version>5.16.2</version>
   </parent>
 
   <scm>
@@ -145,9 +145,9 @@
     <jettyVersion>8.1.7.v20120910</jettyVersion>
     <tephra.version>0.15.0-incubating</tephra.version>
     <omid.version>1.0.0</omid.version>
-    <spark.version>${cdh.spark.version}</spark.version>
-    <scala.version>2.10.4</scala.version>
-    <scala.binary.version>2.10</scala.binary.version>
+    <spark.version>2.4.0.cloudera2</spark.version>
+    <scala.version>2.11.12</scala.version>
+    <scala.binary.version>2.11</scala.binary.version>
     <stream.version>2.9.5</stream.version>
     <i18n-util.version>1.0.4</i18n-util.version>
     <servlet.api.version>3.1.0</servlet.api.version>