You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ch...@apache.org on 2020/03/10 05:40:42 UTC

[phoenix] branch 4.x-HBase-1.4 updated: PHOENIX-5607 Client-server backward compatibility tests

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

chinmayskulkarni pushed a commit to branch 4.x-HBase-1.4
in repository https://gitbox.apache.org/repos/asf/phoenix.git


The following commit(s) were added to refs/heads/4.x-HBase-1.4 by this push:
     new bdc0609  PHOENIX-5607 Client-server backward compatibility tests
bdc0609 is described below

commit bdc0609c00797861fd2d93e981fb92d5113d010b
Author: Sandeep Guggilam <sg...@sandeepg-ltm.internal.salesforce.com>
AuthorDate: Tue Mar 3 13:04:39 2020 -0800

    PHOENIX-5607 Client-server backward compatibility tests
    
    Signed-off-by: Chinmay Kulkarni <ch...@apache.org>
---
 .../phoenix/end2end/BackwardCompatibilityIT.java   | 398 +++++++++++++++++++++
 .../resources/gold_files/gold_query_add_data.txt   |  43 +++
 .../resources/gold_files/gold_query_add_delete.txt |  22 ++
 .../resources/gold_files/gold_query_create_add.txt |  32 ++
 .../src/it/resources/sql_files/add_data.sql        |  27 ++
 .../src/it/resources/sql_files/add_delete.sql      |  26 ++
 .../src/it/resources/sql_files/create_add.sql      |  33 ++
 phoenix-core/src/it/resources/sql_files/query.sql  |  24 ++
 .../it/resources/sql_files/query_add_delete.sql    |  26 ++
 .../src/it/resources/sql_files/query_more.sql      |  25 ++
 phoenix-core/src/it/scripts/execute_query.sh       |  40 +++
 .../phoenix/coprocessor/MetaDataProtocol.java      |   5 +-
 12 files changed, 700 insertions(+), 1 deletion(-)

diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BackwardCompatibilityIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BackwardCompatibilityIT.java
new file mode 100644
index 0000000..dfa0032
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BackwardCompatibilityIT.java
@@ -0,0 +1,398 @@
+/*
+ * 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.query.BaseTest.setUpConfigForMiniCluster;
+import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assume.assumeFalse;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileReader;
+import java.io.FileWriter;
+import java.io.InputStreamReader;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.util.Collection;
+import java.util.List;
+import java.util.Properties;
+
+import org.apache.curator.shaded.com.google.common.collect.Lists;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.phoenix.coprocessor.MetaDataProtocol;
+import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
+import org.apache.phoenix.jdbc.PhoenixDriver;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.query.QueryServicesOptions;
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+/**
+ * This class is meant for testing all compatible client versions 
+ * against the current server version. It runs SQL queries with given 
+ * client versions and compares the output against gold files
+ */
+
+@RunWith(Parameterized.class)
+@Category(NeedsOwnMiniClusterTest.class)
+public class BackwardCompatibilityIT {
+
+    private static final String SQL_DIR = "src/it/resources/sql_files/";
+    private static final String RESULT_DIR = "src/it/resources/gold_files/";
+    private static final String RESULT_PREFIX = "result_";
+    private static final String SQL_EXTENSION = ".sql";
+    private static final String TEXT_EXTENSION = ".txt";
+    private static final String CREATE_ADD = "create_add";
+    private static final String ADD_DATA = "add_data";
+    private static final String ADD_DELETE = "add_delete";
+    private static final String QUERY = "query";
+    private static final String QUERY_MORE = "query_more";
+    private static final String QUERY_ADD_DELETE = "query_add_delete";
+
+    private final String compatibleClientVersion;
+    private static Configuration conf;
+    private static HBaseTestingUtility hbaseTestUtil;
+    private static String zkQuorum;
+    private static String url;
+
+    public BackwardCompatibilityIT(String compatibleClientVersion) {
+        this.compatibleClientVersion = compatibleClientVersion;
+    }
+
+    @Parameters(name = "BackwardCompatibilityIT_compatibleClientVersion={0}")
+    public static synchronized Collection<String> data() {
+        return MetaDataProtocol.COMPATIBLE_CLIENT_VERSIONS;
+    }
+
+    @Before
+    public synchronized void doSetup() throws Exception {
+        conf = HBaseConfiguration.create();
+        hbaseTestUtil = new HBaseTestingUtility(conf);
+        setUpConfigForMiniCluster(conf);
+        conf.set(QueryServices.EXTRA_JDBC_ARGUMENTS_ATTRIB, QueryServicesOptions.DEFAULT_EXTRA_JDBC_ARGUMENTS);
+        hbaseTestUtil.startMiniCluster();
+        zkQuorum = "localhost:" + hbaseTestUtil.getZkCluster().getClientPort();
+        url = PhoenixRuntime.JDBC_PROTOCOL + PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR + zkQuorum;
+        DriverManager.registerDriver(PhoenixDriver.INSTANCE);
+    }
+    
+    @After
+    public void cleanUpAfterTest() throws Exception {
+        try {
+            DriverManager.deregisterDriver(PhoenixDriver.INSTANCE);
+        } finally {
+            hbaseTestUtil.shutdownMiniCluster();
+        }
+    }
+
+    /**
+     * Scenario: 
+     * 1. Old Client connects to the updated server 
+     * 2. Old Client creates tables and inserts data 
+     * 3. New Client reads the data inserted by the old client
+     * 
+     * @throws Exception
+     */
+    @Test
+    public void testUpsertWithOldClient() throws Exception {
+        checkForPreConditions();
+        // Insert data with old client and read with new client
+        executeQueryWithClientVersion(compatibleClientVersion, CREATE_ADD);
+        executeQueriesWithCurrentVersion(QUERY);
+        assertTrue(compareOutput(CREATE_ADD, QUERY));
+    }
+
+    /**
+     * Scenario: 
+     * 1. New Client connects to the updated server 
+     * 2. New Client creates tables and inserts data 
+     * 3. Old Client reads the data inserted by the old client
+     * 
+     * @throws Exception
+     */
+    @Test
+    public void testSelectWithOldClient() throws Exception {
+        checkForPreConditions();
+        // Insert data with new client and read with old client
+        executeQueriesWithCurrentVersion(CREATE_ADD);
+        executeQueryWithClientVersion(compatibleClientVersion, QUERY);
+        assertTrue(compareOutput(CREATE_ADD, QUERY));
+    }
+
+    /**
+     * Scenario: 
+     * 1. Old Client connects to the updated server 
+     * 2. Old Client creates tables and inserts data 
+     * 3. New Client reads the data inserted by the old client 
+     * 4. New Client inserts more data into the tables created by old client 
+     * 5. Old Client reads the data inserted by new client
+     * 
+     * @throws Exception
+     */
+    @Test
+    public void testSelectUpsertWithNewClient() throws Exception {
+        checkForPreConditions();
+        // Insert data with old client and read with new client
+        executeQueryWithClientVersion(compatibleClientVersion, CREATE_ADD);
+        executeQueriesWithCurrentVersion(QUERY);
+        assertTrue(compareOutput(CREATE_ADD, QUERY));
+
+        // Insert more data with new client and read with old client
+        executeQueriesWithCurrentVersion(ADD_DATA);
+        executeQueryWithClientVersion(compatibleClientVersion, QUERY_MORE);
+        assertTrue(compareOutput(ADD_DATA, QUERY_MORE));
+    }
+
+    /**
+     * Scenario: 
+     * 1. New Client connects to the updated server 
+     * 2. New Client creates tables and inserts data 
+     * 3. Old Client reads the data inserted by the old client 
+     * 4. Old Client inserts more data into the tables created by old client 
+     * 5. New Client reads the data inserted by new client
+     * 
+     * @throws Exception
+     */
+    @Test
+    public void testSelectUpsertWithOldClient() throws Exception {
+        checkForPreConditions();
+        // Insert data with new client and read with old client
+        executeQueriesWithCurrentVersion(CREATE_ADD);
+        executeQueryWithClientVersion(compatibleClientVersion, QUERY);
+        assertTrue(compareOutput(CREATE_ADD, QUERY));
+
+        // Insert more data with old client and read with new client
+        executeQueryWithClientVersion(compatibleClientVersion, ADD_DATA);
+        executeQueriesWithCurrentVersion(QUERY_MORE);
+        assertTrue(compareOutput(ADD_DATA, QUERY_MORE));
+    }
+
+    /**
+     * Scenario: 
+     * 1. Old Client connects to the updated server 
+     * 2. Old Client creates tables and inserts data 
+     * 3. New Client reads the data inserted by the old client 
+     * 4. Old Client creates and deletes the data
+     * 
+     * @throws Exception
+     */
+    @Test
+    public void testUpsertDeleteWithOldClient() throws Exception {
+        checkForPreConditions();
+        // Insert data with old client and read with new client
+        executeQueryWithClientVersion(compatibleClientVersion, CREATE_ADD);
+        executeQueriesWithCurrentVersion(QUERY);
+        assertTrue(compareOutput(CREATE_ADD, QUERY));
+
+        // Deletes with the old client
+        executeQueryWithClientVersion(compatibleClientVersion, ADD_DELETE);
+        executeQueryWithClientVersion(compatibleClientVersion, QUERY_ADD_DELETE);
+        assertTrue(compareOutput(ADD_DELETE, QUERY_ADD_DELETE));
+    }
+
+    /**
+     * Scenario: 
+     * 1. New Client connects to the updated server 
+     * 2. New Client creates tables and inserts data 
+     * 3. Old Client reads the data inserted by the old client 
+     * 4. New Client creates and deletes the data
+     * 
+     * @throws Exception
+     */
+    @Test
+    public void testUpsertDeleteWithNewClient() throws Exception {
+        checkForPreConditions();
+        // Insert data with old client and read with new client
+        executeQueriesWithCurrentVersion(CREATE_ADD);
+        executeQueryWithClientVersion(compatibleClientVersion, QUERY);
+        assertTrue(compareOutput(CREATE_ADD, QUERY));
+
+        // Deletes with the new client
+        executeQueriesWithCurrentVersion(ADD_DELETE);
+        executeQueriesWithCurrentVersion(QUERY_ADD_DELETE);
+        assertTrue(compareOutput(ADD_DELETE, QUERY_ADD_DELETE));
+    }
+    
+    private void checkForPreConditions() throws Exception {
+        // For the first code cut of any major version, there wouldn't be any backward compatible
+        // clients. Hence the test wouldn't run and just return true when the client  
+        // version to be tested is same as current version
+        assumeFalse(compatibleClientVersion.contains(MetaDataProtocol.CURRENT_CLIENT_VERSION));
+        // Make sure that cluster is clean before test execution with no system tables
+        try (org.apache.hadoop.hbase.client.Connection conn = 
+                ConnectionFactory.createConnection(conf)) {
+            Admin admin = conn.getAdmin();
+            assertFalse(admin.tableExists(TableName.valueOf(QueryConstants.SYSTEM_SCHEMA_NAME, 
+                    PhoenixDatabaseMetaData.SYSTEM_CATALOG_TABLE)));
+        }       
+    }
+
+    // Executes the queries listed in the operation file with a given client version
+    private void executeQueryWithClientVersion(String clientVersion, String operation)
+            throws Exception {
+        String BASH = "/bin/bash";
+        String EXECUTE_QUERY_SH = "src/it/scripts/execute_query.sh";
+
+        List<String> cmdParams = Lists.newArrayList();
+        cmdParams.add(BASH);
+        cmdParams.add(EXECUTE_QUERY_SH);
+        cmdParams.add(zkQuorum);
+        cmdParams.add(clientVersion);
+
+        cmdParams.add(new File(SQL_DIR + operation + SQL_EXTENSION).getAbsolutePath());
+        cmdParams.add(
+            new File(RESULT_DIR + RESULT_PREFIX + operation + TEXT_EXTENSION).getAbsolutePath());
+        cmdParams.add(System.getProperty("java.io.tmpdir"));
+
+        if (System.getProperty("maven.home") != null) {
+            cmdParams.add(System.getProperty("maven.home"));
+        }
+
+        ProcessBuilder pb = new ProcessBuilder(cmdParams);
+        final Process p = pb.start();
+        final StringBuffer sb = new StringBuffer();
+        //Capture the error stream if any from the execution of the script
+        Thread errorStreamThread = new Thread() {
+            @Override
+            public void run() {
+                try {
+                    BufferedReader reader = new BufferedReader(
+                            new InputStreamReader(p.getErrorStream()));
+                    String line = null;
+                    while ((line = reader.readLine()) != null) {
+                        sb.append(line);
+                    }
+                    reader.close();
+                } catch (final Exception e) {
+                    e.printStackTrace();
+                }
+            }
+        };
+        errorStreamThread.start();
+        assertEquals(sb.toString(), 0, p.waitFor());
+    }
+
+    // Executes the SQL commands listed in the given operation file from the sql_files directory
+    private ResultSet executeQueriesWithCurrentVersion(String operation) throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        try (Connection conn = DriverManager.getConnection(url, props)) {
+            StringBuilder sb = new StringBuilder();
+            BufferedReader reader =
+                    new BufferedReader(new FileReader(SQL_DIR + operation + SQL_EXTENSION));
+            String sqlCommand;
+            ResultSet rs = null;
+            while ((sqlCommand = reader.readLine()) != null) {
+                sqlCommand = sqlCommand.trim();
+                if (sqlCommand.length() == 0 || sqlCommand.startsWith("/") || sqlCommand.startsWith("*")) continue;
+                sb.append(sqlCommand);
+            }
+            reader.close();
+            
+            String[] sqlCommands = sb.toString().split(";");
+            try (BufferedWriter br = new BufferedWriter(
+                new FileWriter(RESULT_DIR + RESULT_PREFIX + operation + TEXT_EXTENSION))) {
+                for (int i = 0; i < sqlCommands.length; i++) {
+                    PreparedStatement stmt = conn.prepareStatement(sqlCommands[i]);
+                    stmt.execute();
+                    rs = stmt.getResultSet();
+                    if (rs != null) {
+                        saveResultSet(rs, br);
+                    }
+                    conn.commit();
+                }
+                return rs;
+            }
+        }        
+    }
+
+    // Saves the result set to a text file to be compared against the gold file for difference
+    private void saveResultSet(ResultSet rs, BufferedWriter br) throws Exception {
+        ResultSetMetaData rsm = rs.getMetaData();
+        int columnCount = rsm.getColumnCount();
+        String row = formatStringWithQuotes(rsm.getColumnName(1));
+        for (int i = 2; i <= columnCount; i++) {
+            row = row + "," + formatStringWithQuotes(rsm.getColumnName(i));
+        }
+        br.write(row);
+        br.write("\n");
+        while (rs.next()) {
+            row = formatStringWithQuotes(rs.getString(1));
+            for (int i = 2; i <= columnCount; i++) {
+                row = row + "," + formatStringWithQuotes(rs.getString(i));
+            }
+            br.write(row);
+            br.write("\n");
+        }
+    }
+
+    private String formatStringWithQuotes(String str) {
+        return (str != null) ? String.format("\'%s\'", str) : "\'\'";
+    }
+
+    // Compares the result file against the gold file to match for the expected output
+    // for the given operation
+    private boolean compareOutput(String gold, String result) throws Exception {
+        BufferedReader goldFileReader = new BufferedReader(new FileReader(
+                        new File(RESULT_DIR + "gold_query_" + gold + TEXT_EXTENSION)));
+        BufferedReader resultFileReader = new BufferedReader(new FileReader(
+                        new File(RESULT_DIR + RESULT_PREFIX + result + TEXT_EXTENSION)));
+
+        List<String> resultFile = Lists.newArrayList();
+        List<String> goldFile = Lists.newArrayList();
+
+        String line = null;
+        while ((line = resultFileReader.readLine()) != null) {
+            resultFile.add(line.trim());
+        }
+        resultFileReader.close();
+
+        while ((line = goldFileReader.readLine()) != null) {
+            line = line.trim();
+            if ( !(line.isEmpty() || line.startsWith("*") || line.startsWith("/"))) {
+                goldFile.add(line);
+            }           
+        }
+        goldFileReader.close();
+
+        // We take the first line in gold file and match against the result file to exclude any
+        // other WARNING messages that comes as a result of the query execution
+        int index = resultFile.indexOf(goldFile.get(0));
+        resultFile = resultFile.subList(index, resultFile.size());
+        return resultFile.equals(goldFile);
+    }
+}
\ No newline at end of file
diff --git a/phoenix-core/src/it/resources/gold_files/gold_query_add_data.txt b/phoenix-core/src/it/resources/gold_files/gold_query_add_data.txt
new file mode 100644
index 0000000..70bc53f
--- /dev/null
+++ b/phoenix-core/src/it/resources/gold_files/gold_query_add_data.txt
@@ -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.
+ */
+ 
+ 'COUNT(1)'
+'7'
+'ID','NAME'
+'a','a_name'
+'b','b_name'
+'c','a_name'
+'d','a_name'
+'x','a_name'
+'y','a_name'
+'z','b_name'
+'ID'
+'a'
+'c'
+'d'
+'x'
+'y'
+'ID'
+'b'
+'z'
+'COUNT(1)'
+'3'
+'ID','NAME'
+'a','a_name'
+'b','b_name'
+'x','x_name'
diff --git a/phoenix-core/src/it/resources/gold_files/gold_query_add_delete.txt b/phoenix-core/src/it/resources/gold_files/gold_query_add_delete.txt
new file mode 100644
index 0000000..4fbc21a
--- /dev/null
+++ b/phoenix-core/src/it/resources/gold_files/gold_query_add_delete.txt
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+ 
+'COUNT(1)'
+'4'
+'COUNT(1)'
+'3'
diff --git a/phoenix-core/src/it/resources/gold_files/gold_query_create_add.txt b/phoenix-core/src/it/resources/gold_files/gold_query_create_add.txt
new file mode 100644
index 0000000..5b19443
--- /dev/null
+++ b/phoenix-core/src/it/resources/gold_files/gold_query_create_add.txt
@@ -0,0 +1,32 @@
+/*
+ * 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.
+ */
+ 
+'COUNT(1)'
+'3'
+'ID','NAME'
+'a','a_name'
+'b','b_name'
+'c','a_name'
+'ID'
+'a'
+'c'
+'COUNT(1)'
+'2'
+'ID','NAME'
+'a','a_name'
+'b','b_name'
diff --git a/phoenix-core/src/it/resources/sql_files/add_data.sql b/phoenix-core/src/it/resources/sql_files/add_data.sql
new file mode 100644
index 0000000..c952b31
--- /dev/null
+++ b/phoenix-core/src/it/resources/sql_files/add_data.sql
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+ 
+UPSERT INTO my_schema.my_table values ('x','a_name');
+UPSERT INTO my_table_view (id, entity_id) VALUES ('y', 'y_entity');
+UPSERT INTO my_schema.my_table_immutable values ('x','x_name');
+
+CREATE VIEW IF NOT EXISTS my_table_second_view (entity_id VARCHAR) 
+                    AS SELECT * FROM  my_schema.my_table WHERE name='b_name';
+UPSERT INTO my_table_second_view (id, entity_id) values ('z', 'z_entity');
+
+UPSERT INTO my_table_view (id, entity_id) values ('d', 'd_entity');
\ No newline at end of file
diff --git a/phoenix-core/src/it/resources/sql_files/add_delete.sql b/phoenix-core/src/it/resources/sql_files/add_delete.sql
new file mode 100644
index 0000000..94bfc95
--- /dev/null
+++ b/phoenix-core/src/it/resources/sql_files/add_delete.sql
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+ 
+CREATE TABLE my_schema.my_table_delete 
+               (id VARCHAR not null primary key, name VARCHAR) SALT_BUCKETS=4;
+UPSERT INTO my_schema.my_table_delete values ('a','a_name');
+DELETE FROM my_schema.my_table_delete;
+DROP TABLE my_schema.my_table_delete;
+
+UPSERT INTO my_schema.my_table values ('y','y_name');
+UPSERT INTO my_schema.my_table_immutable values ('y','y_name');
diff --git a/phoenix-core/src/it/resources/sql_files/create_add.sql b/phoenix-core/src/it/resources/sql_files/create_add.sql
new file mode 100644
index 0000000..9c3f984
--- /dev/null
+++ b/phoenix-core/src/it/resources/sql_files/create_add.sql
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+ 
+CREATE TABLE IF NOT EXISTS my_schema.my_table 
+                    (id VARCHAR NOT NULL PRIMARY KEY, 
+                     name VARCHAR) VERSIONS=1;
+UPSERT INTO my_schema.my_table values ('a','a_name');
+UPSERT INTO my_schema.my_table values ('b','b_name');
+
+CREATE VIEW IF NOT EXISTS my_table_view (entity_id VARCHAR) 
+                    AS SELECT * FROM  my_schema.my_table WHERE name='a_name';
+UPSERT INTO my_table_view (id, entity_id) values ('c', 'c_entity');
+
+CREATE TABLE IF NOT EXISTS my_schema.my_table_immutable 
+                    (id VARCHAR NOT NULL PRIMARY KEY, name VARCHAR) IMMUTABLE_ROWS=true;
+UPSERT INTO my_schema.my_table_immutable values ('a','a_name');
+UPSERT INTO my_schema.my_table_immutable values ('b','b_name');
+
diff --git a/phoenix-core/src/it/resources/sql_files/query.sql b/phoenix-core/src/it/resources/sql_files/query.sql
new file mode 100644
index 0000000..ebf154a
--- /dev/null
+++ b/phoenix-core/src/it/resources/sql_files/query.sql
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ */
+ 
+SELECT COUNT(*) from my_schema.my_table;
+SELECT * FROM my_schema.my_table;
+SELECT id from my_table_view;
+
+SELECT COUNT(*) from my_schema.my_table_immutable;
+SELECT * FROM my_schema.my_table_immutable;
diff --git a/phoenix-core/src/it/resources/sql_files/query_add_delete.sql b/phoenix-core/src/it/resources/sql_files/query_add_delete.sql
new file mode 100644
index 0000000..c99d54c
--- /dev/null
+++ b/phoenix-core/src/it/resources/sql_files/query_add_delete.sql
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+ 
+CREATE TABLE my_schema.my_table_delete 
+             (id VARCHAR not null primary key, name VARCHAR) SALT_BUCKETS=4;
+UPSERT INTO my_schema.my_table_delete values ('a','a_name');
+DELETE FROM my_schema.my_table_delete;
+DROP TABLE my_schema.my_table_delete;
+
+SELECT COUNT(*) from my_schema.my_table;
+SELECT COUNT(*) from my_schema.my_table_immutable;
diff --git a/phoenix-core/src/it/resources/sql_files/query_more.sql b/phoenix-core/src/it/resources/sql_files/query_more.sql
new file mode 100644
index 0000000..061505f
--- /dev/null
+++ b/phoenix-core/src/it/resources/sql_files/query_more.sql
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+ 
+SELECT COUNT(*) from my_schema.my_table;
+SELECT * FROM my_schema.my_table;
+SELECT id from my_table_view;
+SELECT id from my_table_second_view;
+
+SELECT COUNT(*) from my_schema.my_table_immutable;
+SELECT * FROM my_schema.my_table_immutable;
diff --git a/phoenix-core/src/it/scripts/execute_query.sh b/phoenix-core/src/it/scripts/execute_query.sh
new file mode 100644
index 0000000..7dba6e9
--- /dev/null
+++ b/phoenix-core/src/it/scripts/execute_query.sh
@@ -0,0 +1,40 @@
+#!/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.
+###########################################################################
+
+# This script is intended to run the sql queries in a file with the given client version 
+
+zk_url=$1
+client_version=$2
+sqlfile=$3
+resultfile=$4
+tmp_dir=$5
+maven_home=$6
+
+if [ -n $maven_home ]; then
+   export PATH=$maven_home/bin:$PATH
+fi
+
+mvn dependency:get -Dartifact=org.apache.phoenix:phoenix-client:$client_version
+mvn dependency:copy -Dartifact=org.apache.phoenix:phoenix-client:$client_version \
+-DoutputDirectory=$tmp_dir
+
+phoenix_client_jar=$tmp_dir/phoenix-client-$client_version.jar
+java -cp ".:$phoenix_client_jar" sqlline.SqlLine -d org.apache.phoenix.jdbc.PhoenixDriver \
+-u jdbc:phoenix:$zk_url -n none -p none --color=false --fastConnect=true --outputformat=csv \
+--silent=true --verbose=false --isolation=TRANSACTION_READ_COMMITTED --run=$sqlfile &> $resultfile
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 3f5e2fb..2982f25 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
@@ -18,6 +18,7 @@
 package org.apache.phoenix.coprocessor;
 
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
@@ -65,7 +66,7 @@ import com.google.protobuf.ByteString;
  */
 public abstract class MetaDataProtocol extends MetaDataService {
     public static final int PHOENIX_MAJOR_VERSION = 4;
-    public static final int PHOENIX_MINOR_VERSION = 15;
+    public static final int PHOENIX_MINOR_VERSION = 16;
     public static final int PHOENIX_PATCH_NUMBER = 0;
     public static final int PHOENIX_VERSION =
             VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER);
@@ -145,6 +146,8 @@ public abstract class MetaDataProtocol extends MetaDataService {
     }
     
     public static final String CURRENT_CLIENT_VERSION = PHOENIX_MAJOR_VERSION + "." + PHOENIX_MINOR_VERSION + "." + PHOENIX_PATCH_NUMBER;
+    public static final List<String> COMPATIBLE_CLIENT_VERSIONS = 
+            Arrays.asList("4.14.3-HBase-1.3", "4.15.0-HBase-1.3");
      
     
     // TODO: pare this down to minimum, as we don't need duplicates for both table and column errors, nor should we need