You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ja...@apache.org on 2017/12/17 00:43:23 UTC

[01/16] phoenix git commit: PHOENIX-4424 Allow users to create "DEFAULT" and "HBASE" Schema (Uppercase Schema Names)

Repository: phoenix
Updated Branches:
  refs/heads/4.x-HBase-1.2 0b1f22749 -> f6f7b4678


PHOENIX-4424 Allow users to create "DEFAULT" and "HBASE" Schema (Uppercase Schema Names)


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

Branch: refs/heads/4.x-HBase-1.2
Commit: f6f7b467883e6089ab564b94d32042340186472d
Parents: a3db335
Author: Karan Mehta <ka...@gmail.com>
Authored: Sat Nov 4 03:13:53 2017 +0000
Committer: James Taylor <jt...@salesforce.com>
Committed: Sat Dec 16 16:42:54 2017 -0800

----------------------------------------------------------------------
 .../phoenix/end2end/ChangePermissionsIT.java    |  5 +-
 .../apache/phoenix/end2end/CreateSchemaIT.java  | 64 ++++++++++++++------
 phoenix-core/src/main/antlr3/PhoenixSQL.g       |  2 +-
 .../phoenix/parse/CreateSchemaStatement.java    |  2 +-
 .../apache/phoenix/query/QueryConstants.java    |  1 -
 .../apache/phoenix/schema/MetaDataClient.java   |  8 ++-
 .../org/apache/phoenix/util/SchemaUtil.java     |  5 +-
 .../apache/phoenix/parse/QueryParserTest.java   | 13 ++++
 8 files changed, 73 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/f6f7b467/phoenix-core/src/it/java/org/apache/phoenix/end2end/ChangePermissionsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ChangePermissionsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ChangePermissionsIT.java
index c023440..2bf7fe1 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ChangePermissionsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ChangePermissionsIT.java
@@ -23,6 +23,7 @@ import org.apache.hadoop.hbase.security.User;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.TableNotFoundException;
+import org.apache.phoenix.util.SchemaUtil;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -144,7 +145,7 @@ public class ChangePermissionsIT extends BasePermissionsIT {
             verifyAllowed(createSchema(SCHEMA_NAME), superUser1);
             verifyAllowed(grantPermissions("C", regularUser1, SCHEMA_NAME, true), superUser1);
         } else {
-            verifyAllowed(grantPermissions("C", regularUser1, "\"" + QueryConstants.HBASE_DEFAULT_SCHEMA_NAME + "\"", true), superUser1);
+            verifyAllowed(grantPermissions("C", regularUser1, "\"" + SchemaUtil.SCHEMA_FOR_DEFAULT_NAMESPACE + "\"", true), superUser1);
         }
 
         // Create new table. Create indexes, views and view indexes on top of it. Verify the contents by querying it
@@ -235,7 +236,7 @@ public class ChangePermissionsIT extends BasePermissionsIT {
             verifyAllowed(createSchema(SCHEMA_NAME), superUser1);
             verifyAllowed(grantPermissions("C", regularUser1, SCHEMA_NAME, true), superUser1);
         } else {
-            verifyAllowed(grantPermissions("C", regularUser1, "\"" + QueryConstants.HBASE_DEFAULT_SCHEMA_NAME + "\"", true), superUser1);
+            verifyAllowed(grantPermissions("C", regularUser1, "\"" + SchemaUtil.SCHEMA_FOR_DEFAULT_NAMESPACE + "\"", true), superUser1);
         }
 
         // Create MultiTenant Table (View Index Table should be automatically created)

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f6f7b467/phoenix-core/src/it/java/org/apache/phoenix/end2end/CreateSchemaIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CreateSchemaIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CreateSchemaIT.java
index fe09dcd..8002dc1 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CreateSchemaIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CreateSchemaIT.java
@@ -43,31 +43,61 @@ public class CreateSchemaIT extends ParallelStatsDisabledIT {
         Properties props = PropertiesUtil.deepCopy(TestUtil.TEST_PROPERTIES);
         props.setProperty(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, Boolean.toString(true));
         String schemaName = generateUniqueName();
-        String ddl = "CREATE SCHEMA " + schemaName;
+        String schemaName1 = schemaName.toLowerCase();
+        String schemaName2 = schemaName.toLowerCase();
+        // Create unique name schema and verify that it exists
+        // ddl1 should create lowercase schemaName since it is passed in with double-quotes
+        // ddl2 should create uppercase schemaName since Phoenix upper-cases identifiers without quotes
+        // Both the statements should succeed
+        String ddl1 = "CREATE SCHEMA \"" + schemaName1 + "\"";
+        String ddl2 = "CREATE SCHEMA " + schemaName2;
         try (Connection conn = DriverManager.getConnection(getUrl(), props);
                 HBaseAdmin admin = conn.unwrap(PhoenixConnection.class).getQueryServices().getAdmin();) {
-            conn.createStatement().execute(ddl);
-            assertNotNull(admin.getNamespaceDescriptor(schemaName));
+            conn.createStatement().execute(ddl1);
+            assertNotNull(admin.getNamespaceDescriptor(schemaName1));
+            conn.createStatement().execute(ddl2);
+            assertNotNull(admin.getNamespaceDescriptor(schemaName2.toUpperCase()));
         }
+        // Try creating it again and verify that it throws SchemaAlreadyExistsException
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-            conn.createStatement().execute(ddl);
+            conn.createStatement().execute(ddl1);
             fail();
         } catch (SchemaAlreadyExistsException e) {
             // expected
         }
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        try {
-            conn.createStatement().execute("CREATE SCHEMA " + SchemaUtil.SCHEMA_FOR_DEFAULT_NAMESPACE);
-            fail();
-        } catch (SQLException e) {
-            assertEquals(SQLExceptionCode.SCHEMA_NOT_ALLOWED.getErrorCode(), e.getErrorCode());
-        }
-        try {
-            conn.createStatement().execute("CREATE SCHEMA " + SchemaUtil.HBASE_NAMESPACE);
-            fail();
-        } catch (SQLException e) {
-            assertEquals(SQLExceptionCode.SCHEMA_NOT_ALLOWED.getErrorCode(), e.getErrorCode());
+
+        // See PHOENIX-4424
+        // Create schema DEFAULT and HBASE (Should allow since they are upper-cased) and verify that it exists
+        // Create schema default and hbase and it should fail
+        try (Connection conn = DriverManager.getConnection(getUrl(), props);
+             HBaseAdmin admin = conn.unwrap(PhoenixConnection.class).getQueryServices().getAdmin();) {
+
+            // default is a SQL keyword, hence it should always be passed in double-quotes
+            try {
+                conn.createStatement().execute("CREATE SCHEMA \""
+                        + SchemaUtil.SCHEMA_FOR_DEFAULT_NAMESPACE + "\"");
+                fail();
+            } catch (SQLException e) {
+                assertEquals(SQLExceptionCode.SCHEMA_NOT_ALLOWED.getErrorCode(), e.getErrorCode());
+            }
+
+            try {
+                conn.createStatement().execute("CREATE SCHEMA \""
+                        + SchemaUtil.HBASE_NAMESPACE + "\"");
+                fail();
+            } catch (SQLException e) {
+                assertEquals(SQLExceptionCode.SCHEMA_NOT_ALLOWED.getErrorCode(), e.getErrorCode());
+            }
+
+            // default is a SQL keyword, hence it should always be passed in double-quotes
+            conn.createStatement().execute("CREATE SCHEMA \""
+                    + SchemaUtil.SCHEMA_FOR_DEFAULT_NAMESPACE.toUpperCase() + "\"");
+            conn.createStatement().execute("CREATE SCHEMA \""
+                    + SchemaUtil.HBASE_NAMESPACE.toUpperCase() + "\"");
+
+            assertNotNull(admin.getNamespaceDescriptor(SchemaUtil.SCHEMA_FOR_DEFAULT_NAMESPACE.toUpperCase()));
+            assertNotNull(admin.getNamespaceDescriptor(SchemaUtil.HBASE_NAMESPACE.toUpperCase()));
+
         }
-        conn.close();
     }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f6f7b467/phoenix-core/src/main/antlr3/PhoenixSQL.g
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/antlr3/PhoenixSQL.g b/phoenix-core/src/main/antlr3/PhoenixSQL.g
index ccf654b..87153cd 100644
--- a/phoenix-core/src/main/antlr3/PhoenixSQL.g
+++ b/phoenix-core/src/main/antlr3/PhoenixSQL.g
@@ -459,7 +459,7 @@ create_table_node returns [CreateTableStatement ret]
    
 // Parse a create schema statement.
 create_schema_node returns [CreateSchemaStatement ret]
-    :   CREATE SCHEMA (IF NOT ex=EXISTS)? (DEFAULT | s=identifier)
+    :   CREATE SCHEMA (IF NOT ex=EXISTS)? s=identifier
         {ret = factory.createSchema(s, ex!=null); }
     ;
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f6f7b467/phoenix-core/src/main/java/org/apache/phoenix/parse/CreateSchemaStatement.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/CreateSchemaStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/CreateSchemaStatement.java
index 7c255cb..f5ab3f6 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/CreateSchemaStatement.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/CreateSchemaStatement.java
@@ -24,7 +24,7 @@ public class CreateSchemaStatement extends MutableStatement {
 	private final boolean ifNotExists;
 	
 	public CreateSchemaStatement(String schemaName,boolean ifNotExists) {
-		this.schemaName = null == schemaName ? SchemaUtil.SCHEMA_FOR_DEFAULT_NAMESPACE : schemaName;
+		this.schemaName = schemaName;
 		this.ifNotExists = ifNotExists;
 	}
 	

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f6f7b467/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
index 851ba9a..7607388 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
@@ -149,7 +149,6 @@ public interface QueryConstants {
     public enum JoinType {INNER, LEFT_OUTER}
     public final static String SYSTEM_SCHEMA_NAME = "SYSTEM";
     public final static byte[] SYSTEM_SCHEMA_NAME_BYTES = Bytes.toBytes(SYSTEM_SCHEMA_NAME);
-    public final static String HBASE_DEFAULT_SCHEMA_NAME = "default";
     public final static String PHOENIX_METADATA = "table";
     public final static String OFFSET_ROW_KEY = "_OFFSET_";
     public final static byte[] OFFSET_ROW_KEY_BYTES = Bytes.toBytes(OFFSET_ROW_KEY);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f6f7b467/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
index fc2e288..5ec5ac3 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
@@ -3980,8 +3980,10 @@ public class MetaDataClient {
                             SQLExceptionCode.CREATE_SCHEMA_NOT_ALLOWED).setSchemaName(create.getSchemaName())
                             .build().buildException(); }
             boolean isIfNotExists = create.isIfNotExists();
-            validateSchema(create.getSchemaName());
             PSchema schema = new PSchema(create.getSchemaName());
+            // Use SchemaName from PSchema object to get the normalized SchemaName
+            // See PHOENIX-4424 for details
+            validateSchema(schema.getSchemaName());
             connection.setAutoCommit(false);
             List<Mutation> schemaMutations;
 
@@ -4016,7 +4018,7 @@ public class MetaDataClient {
 
     private void validateSchema(String schemaName) throws SQLException {
         if (SchemaUtil.NOT_ALLOWED_SCHEMA_LIST.contains(
-                schemaName.toUpperCase())) { throw new SQLExceptionInfo.Builder(SQLExceptionCode.SCHEMA_NOT_ALLOWED)
+                schemaName)) { throw new SQLExceptionInfo.Builder(SQLExceptionCode.SCHEMA_NOT_ALLOWED)
                 .setSchemaName(schemaName).build().buildException(); }
     }
 
@@ -4082,7 +4084,7 @@ public class MetaDataClient {
 
             if (changePermsStatement.getSchemaName() != null) {
                 // SYSTEM.CATALOG doesn't have any entry for "default" HBase namespace, hence we will bypass the check
-                if(!changePermsStatement.getSchemaName().equals(QueryConstants.HBASE_DEFAULT_SCHEMA_NAME)) {
+                if(!changePermsStatement.getSchemaName().equals(SchemaUtil.SCHEMA_FOR_DEFAULT_NAMESPACE)) {
                     FromCompiler.getResolverForSchema(changePermsStatement.getSchemaName(), connection);
                 }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f6f7b467/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
index 5b5c3a5..42c2dcb 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
@@ -133,8 +133,9 @@ public class SchemaUtil {
         
     };
     public static final RowKeySchema VAR_BINARY_SCHEMA = new RowKeySchemaBuilder(1).addField(VAR_BINARY_DATUM, false, SortOrder.getDefault()).build();
-    public static final String SCHEMA_FOR_DEFAULT_NAMESPACE = "DEFAULT";
-    public static final String HBASE_NAMESPACE = "HBASE";
+    // See PHOENIX-4424
+    public static final String SCHEMA_FOR_DEFAULT_NAMESPACE = "default";
+    public static final String HBASE_NAMESPACE = "hbase";
     public static final List<String> NOT_ALLOWED_SCHEMA_LIST = Arrays.asList(SCHEMA_FOR_DEFAULT_NAMESPACE,
             HBASE_NAMESPACE);
     

http://git-wip-us.apache.org/repos/asf/phoenix/blob/f6f7b467/phoenix-core/src/test/java/org/apache/phoenix/parse/QueryParserTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/parse/QueryParserTest.java b/phoenix-core/src/test/java/org/apache/phoenix/parse/QueryParserTest.java
index 25f59c0..24653c6 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/parse/QueryParserTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/parse/QueryParserTest.java
@@ -69,6 +69,19 @@ public class QueryParserTest {
     }
 
     @Test
+    public void testCreateSchema() throws Exception {
+
+        String sql0 = "create schema \"schema1\"";
+        parseQuery(sql0);
+        String sql1 = "create schema schema1";
+        parseQuery(sql1);
+        String sql2 = "create schema \"default\"";
+        parseQuery(sql2);
+        String sql3 = "create schema \"DEFAULT\"";
+        parseQuery(sql3);
+    }
+
+    @Test
     public void testParseGrantQuery() throws Exception {
 
         String sql0 = "GRANT 'RX' ON SYSTEM.\"SEQUENCE\" TO 'user'";


[15/16] phoenix git commit: PHOENIX-4415 Ignore CURRENT_SCN property if set in Pig Storer

Posted by ja...@apache.org.
PHOENIX-4415 Ignore CURRENT_SCN property if set in Pig Storer


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

Branch: refs/heads/4.x-HBase-1.2
Commit: a3db33567b4eb91f9c7b41d0232331da1f7873bb
Parents: ef174bb
Author: James Taylor <jt...@salesforce.com>
Authored: Wed Nov 8 03:13:53 2017 +0000
Committer: James Taylor <jt...@salesforce.com>
Committed: Sat Dec 16 16:42:54 2017 -0800

----------------------------------------------------------------------
 .../phoenix/mapreduce/PhoenixOutputFormat.java  | 13 ++++++++++-
 .../phoenix/mapreduce/PhoenixRecordWriter.java  |  8 ++++++-
 .../phoenix/mapreduce/util/ConnectionUtil.java  | 23 ++++++++++++++++----
 .../org/apache/phoenix/util/PropertiesUtil.java |  9 +++++++-
 .../java/org/apache/phoenix/pig/BasePigIT.java  |  4 ++++
 .../apache/phoenix/pig/PhoenixHBaseStorage.java | 12 ++++++----
 6 files changed, 58 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/a3db3356/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/PhoenixOutputFormat.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/PhoenixOutputFormat.java b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/PhoenixOutputFormat.java
index e55b977..4217e40 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/PhoenixOutputFormat.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/PhoenixOutputFormat.java
@@ -19,6 +19,8 @@ package org.apache.phoenix.mapreduce;
 
 import java.io.IOException;
 import java.sql.SQLException;
+import java.util.Collections;
+import java.util.Set;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -36,6 +38,15 @@ import org.apache.hadoop.mapreduce.lib.db.DBWritable;
  */
 public class PhoenixOutputFormat <T extends DBWritable> extends OutputFormat<NullWritable,T> {
     private static final Log LOG = LogFactory.getLog(PhoenixOutputFormat.class);
+    private final Set<String> propsToIgnore;
+    
+    public PhoenixOutputFormat() {
+        this(Collections.<String>emptySet());
+    }
+    
+    public PhoenixOutputFormat(Set<String> propsToIgnore) {
+        this.propsToIgnore = propsToIgnore;
+    }
     
     @Override
     public void checkOutputSpecs(JobContext jobContext) throws IOException, InterruptedException {      
@@ -52,7 +63,7 @@ public class PhoenixOutputFormat <T extends DBWritable> extends OutputFormat<Nul
     @Override
     public RecordWriter<NullWritable, T> getRecordWriter(TaskAttemptContext context) throws IOException, InterruptedException {
         try {
-            return new PhoenixRecordWriter<T>(context.getConfiguration());
+            return new PhoenixRecordWriter<T>(context.getConfiguration(), propsToIgnore);
         } catch (SQLException e) {
             LOG.error("Error calling PhoenixRecordWriter "  + e.getMessage());
             throw new RuntimeException(e);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a3db3356/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/PhoenixRecordWriter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/PhoenixRecordWriter.java b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/PhoenixRecordWriter.java
index 70ee3f5..52f2fe3 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/PhoenixRecordWriter.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/PhoenixRecordWriter.java
@@ -21,6 +21,8 @@ import java.io.IOException;
 import java.sql.Connection;
 import java.sql.PreparedStatement;
 import java.sql.SQLException;
+import java.util.Collections;
+import java.util.Set;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -46,7 +48,11 @@ public class PhoenixRecordWriter<T extends DBWritable>  extends RecordWriter<Nul
     private long numRecords = 0;
     
     public PhoenixRecordWriter(final Configuration configuration) throws SQLException {
-        this.conn = ConnectionUtil.getOutputConnection(configuration);
+        this(configuration, Collections.<String>emptySet());
+    }
+    
+    public PhoenixRecordWriter(final Configuration configuration, Set<String> propsToIgnore) throws SQLException {
+        this.conn = ConnectionUtil.getOutputConnectionWithoutTheseProps(configuration, propsToIgnore);
         this.batchSize = PhoenixConfigurationUtil.getBatchSize(configuration);
         final String upsertQuery = PhoenixConfigurationUtil.getUpsertStatement(configuration);
         this.statement = this.conn.prepareStatement(upsertQuery);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a3db3356/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/util/ConnectionUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/util/ConnectionUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/util/ConnectionUtil.java
index ada3816..56a5ef5 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/util/ConnectionUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/util/ConnectionUtil.java
@@ -20,15 +20,16 @@ package org.apache.phoenix.mapreduce.util;
 import java.sql.Connection;
 import java.sql.DriverManager;
 import java.sql.SQLException;
-import java.util.Iterator;
-import java.util.Map;
+import java.util.Collections;
 import java.util.Properties;
+import java.util.Set;
 
-import com.google.common.base.Preconditions;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.QueryUtil;
 
+import com.google.common.base.Preconditions;
+
 /**
  * Utility class to return a {@link Connection} .
  */
@@ -74,15 +75,29 @@ public class ConnectionUtil {
      * Create the configured output Connection.
      *
      * @param conf configuration containing the connection information
+     * @return the configured output connection
+     */
+    public static Connection getOutputConnectionWithoutTheseProps(final Configuration conf, Set<String> ignoreTheseProps) throws SQLException {
+        return getOutputConnection(conf, new Properties(), ignoreTheseProps);
+    }
+    
+    /**
+     * Create the configured output Connection.
+     *
+     * @param conf configuration containing the connection information
      * @param props custom connection properties
      * @return the configured output connection
      */
     public static Connection getOutputConnection(final Configuration conf, Properties props) throws SQLException {
+        return getOutputConnection(conf, props, Collections.<String>emptySet());
+    }
+    
+    public static Connection getOutputConnection(final Configuration conf, Properties props, Set<String> withoutTheseProps) throws SQLException {
         Preconditions.checkNotNull(conf);
 		return getConnection(PhoenixConfigurationUtil.getOutputCluster(conf),
 				PhoenixConfigurationUtil.getClientPort(conf),
 				PhoenixConfigurationUtil.getZNodeParent(conf),
-				PropertiesUtil.combineProperties(props, conf));
+				PropertiesUtil.combineProperties(props, conf, withoutTheseProps));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a3db3356/phoenix-core/src/main/java/org/apache/phoenix/util/PropertiesUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/PropertiesUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/PropertiesUtil.java
index f6eb5c5..685b8cb 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/PropertiesUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/PropertiesUtil.java
@@ -17,10 +17,13 @@
  */
 package org.apache.phoenix.util;
 
+import java.util.Collections;
 import java.util.Iterator;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Properties;
+import java.util.Set;
+
 import org.apache.hadoop.conf.Configuration;
 
 public class PropertiesUtil {
@@ -50,13 +53,17 @@ public class PropertiesUtil {
      *         properties contained in conf
      */
     public static Properties combineProperties(Properties props, final Configuration conf) {
+        return combineProperties(props, conf, Collections.<String>emptySet());
+    }
+    
+    public static Properties combineProperties(Properties props, final Configuration conf, Set<String> withoutTheseProps) {
         Iterator<Map.Entry<String, String>> iterator = conf.iterator();
         Properties copy = deepCopy(props);
         if (iterator != null) {
             while (iterator.hasNext()) {
                 Map.Entry<String, String> entry = iterator.next();
                 // set the property from config only if props doesn't have it already
-                if (copy.getProperty(entry.getKey()) == null) {
+                if (copy.getProperty(entry.getKey()) == null && !withoutTheseProps.contains(entry.getKey())) {
                     copy.setProperty(entry.getKey(), entry.getValue());
                 }
             }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a3db3356/phoenix-pig/src/it/java/org/apache/phoenix/pig/BasePigIT.java
----------------------------------------------------------------------
diff --git a/phoenix-pig/src/it/java/org/apache/phoenix/pig/BasePigIT.java b/phoenix-pig/src/it/java/org/apache/phoenix/pig/BasePigIT.java
index 94ccc25..4de9854 100644
--- a/phoenix-pig/src/it/java/org/apache/phoenix/pig/BasePigIT.java
+++ b/phoenix-pig/src/it/java/org/apache/phoenix/pig/BasePigIT.java
@@ -29,8 +29,10 @@ import java.util.Properties;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
 import org.apache.phoenix.end2end.Shadower;
+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.apache.phoenix.util.ReadOnlyProps;
 import org.apache.pig.ExecType;
@@ -62,6 +64,8 @@ public class BasePigIT extends BaseHBaseManagedTimeIT {
     public void setUp() throws Exception {
         conf = getTestClusterConfig();
         conf.set(QueryServices.EXTRA_JDBC_ARGUMENTS_ATTRIB, QueryServicesOptions.DEFAULT_EXTRA_JDBC_ARGUMENTS);
+        // Set CURRENT_SCN to confirm that it's ignored
+        conf.set(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(System.currentTimeMillis()+QueryConstants.MILLIS_IN_DAY));
         pigServer = new PigServer(ExecType.LOCAL, conf);
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         conn = DriverManager.getConnection(getUrl(), props);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a3db3356/phoenix-pig/src/main/java/org/apache/phoenix/pig/PhoenixHBaseStorage.java
----------------------------------------------------------------------
diff --git a/phoenix-pig/src/main/java/org/apache/phoenix/pig/PhoenixHBaseStorage.java b/phoenix-pig/src/main/java/org/apache/phoenix/pig/PhoenixHBaseStorage.java
index a9f0c8f..e061c1c 100644
--- a/phoenix-pig/src/main/java/org/apache/phoenix/pig/PhoenixHBaseStorage.java
+++ b/phoenix-pig/src/main/java/org/apache/phoenix/pig/PhoenixHBaseStorage.java
@@ -21,6 +21,9 @@ import java.io.IOException;
 import java.sql.SQLException;
 import java.util.List;
 import java.util.Properties;
+import java.util.Set;
+import java.util.HashSet;
+import java.util.Arrays;
 
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.CommandLineParser;
@@ -43,6 +46,7 @@ import org.apache.phoenix.pig.util.TableSchemaParserFunction;
 import org.apache.phoenix.pig.util.TypeUtil;
 import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.util.ColumnInfo;
+import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.pig.ResourceSchema;
 import org.apache.pig.ResourceSchema.ResourceFieldSchema;
 import org.apache.pig.StoreFuncInterface;
@@ -87,14 +91,15 @@ import org.slf4j.LoggerFactory;
 public class PhoenixHBaseStorage implements StoreFuncInterface {
 
     private static final Logger LOG = LoggerFactory.getLogger(PhoenixHBaseStorage.class);
-    
+    private static final Set<String> PROPS_TO_IGNORE = new HashSet<>(Arrays.asList(PhoenixRuntime.CURRENT_SCN_ATTRIB));
+
     private Configuration config;
     private RecordWriter<NullWritable, PhoenixRecordWritable> writer;
     private List<ColumnInfo> columnInfo = null;
     private String contextSignature = null;
     private ResourceSchema schema;  
     private long batchSize;
-    private final PhoenixOutputFormat outputFormat = new PhoenixOutputFormat();
+    private final PhoenixOutputFormat outputFormat = new PhoenixOutputFormat<PhoenixRecordWritable>(PROPS_TO_IGNORE);
     // Set of options permitted
     private final static Options validOptions = new Options();
     private final static CommandLineParser parser = new GnuParser();
@@ -228,5 +233,4 @@ public class PhoenixHBaseStorage implements StoreFuncInterface {
         schema = s;
         getUDFProperties().setProperty(contextSignature + SCHEMA, ObjectSerializer.serialize(schema));
     }
-
-}
\ No newline at end of file
+}


[04/16] phoenix git commit: PHOENIX-4342 - Surface QueryPlan in MutationPlan

Posted by ja...@apache.org.
PHOENIX-4342 - Surface QueryPlan in MutationPlan


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

Branch: refs/heads/4.x-HBase-1.2
Commit: a23439cf61a9ba8d4b298b97caf9498c71d29e64
Parents: 0b1f227
Author: Geoffrey Jacoby <gj...@apache.org>
Authored: Thu Nov 2 20:41:02 2017 +0000
Committer: James Taylor <jt...@salesforce.com>
Committed: Sat Dec 16 16:42:54 2017 -0800

----------------------------------------------------------------------
 .../phoenix/compile/BaseMutationPlan.java       |   5 +
 .../phoenix/compile/DelegateMutationPlan.java   |   5 +
 .../apache/phoenix/compile/DeleteCompiler.java  | 545 ++++++++-------
 .../apache/phoenix/compile/MutationPlan.java    |   5 +-
 .../apache/phoenix/compile/UpsertCompiler.java  | 675 +++++++++++--------
 .../apache/phoenix/jdbc/PhoenixStatement.java   |   9 +-
 6 files changed, 733 insertions(+), 511 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/a23439cf/phoenix-core/src/main/java/org/apache/phoenix/compile/BaseMutationPlan.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/BaseMutationPlan.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/BaseMutationPlan.java
index 0e45682..60eb59a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/BaseMutationPlan.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/BaseMutationPlan.java
@@ -79,4 +79,9 @@ public abstract class BaseMutationPlan implements MutationPlan {
         return 0l;
     }
 
+    @Override
+    public QueryPlan getQueryPlan() {
+        return null;
+    }
+
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a23439cf/phoenix-core/src/main/java/org/apache/phoenix/compile/DelegateMutationPlan.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/DelegateMutationPlan.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/DelegateMutationPlan.java
index 343ec32..90eef61 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/DelegateMutationPlan.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/DelegateMutationPlan.java
@@ -42,6 +42,11 @@ public class DelegateMutationPlan implements MutationPlan {
     }
 
     @Override
+    public QueryPlan getQueryPlan() {
+        return plan.getQueryPlan();
+    }
+
+    @Override
     public ParameterMetaData getParameterMetaData() {
         return plan.getParameterMetaData();
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a23439cf/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
----------------------------------------------------------------------
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 f038cda..8d9a5b6 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
@@ -303,14 +303,16 @@ public class DeleteCompiler {
         return Collections.emptyList();
     }
     
-    private class MultiDeleteMutationPlan implements MutationPlan {
+    private class MultiRowDeleteMutationPlan implements MutationPlan {
         private final List<MutationPlan> plans;
         private final MutationPlan firstPlan;
-        
-        public MultiDeleteMutationPlan(@NotNull List<MutationPlan> plans) {
+        private final QueryPlan dataPlan;
+
+        public MultiRowDeleteMutationPlan(QueryPlan dataPlan, @NotNull List<MutationPlan> plans) {
             Preconditions.checkArgument(!plans.isEmpty());
             this.plans = plans;
             this.firstPlan = plans.get(0);
+            this.dataPlan = dataPlan;
         }
         
         @Override
@@ -348,8 +350,8 @@ public class DeleteCompiler {
             return firstPlan.getSourceRefs();
         }
 
-		@Override
-		public Operation getOperation() {
+		    @Override
+		    public Operation getOperation() {
 			return operation;
 		}
 
@@ -401,6 +403,11 @@ public class DeleteCompiler {
             }
             return estInfoTimestamp;
         }
+
+        @Override
+        public QueryPlan getQueryPlan() {
+            return dataPlan;
+        }
     }
 
     public MutationPlan compile(DeleteStatement delete) throws SQLException {
@@ -548,69 +555,9 @@ public class DeleteCompiler {
             List<MutationPlan> mutationPlans = Lists.newArrayListWithExpectedSize(queryPlans.size());
             for (final QueryPlan plan : queryPlans) {
                 final StatementContext context = plan.getContext();
-                mutationPlans.add(new MutationPlan() {
-    
-                    @Override
-                    public ParameterMetaData getParameterMetaData() {
-                        return context.getBindManager().getParameterMetaData();
-                    }
-    
-                    @Override
-                    public MutationState execute() throws SQLException {
-                        // We have a point lookup, so we know we have a simple set of fully qualified
-                        // keys for our ranges
-                        ScanRanges ranges = context.getScanRanges();
-                        Iterator<KeyRange> iterator = ranges.getPointLookupKeyIterator(); 
-                        Map<ImmutableBytesPtr,RowMutationState> mutation = Maps.newHashMapWithExpectedSize(ranges.getPointLookupCount());
-                        while (iterator.hasNext()) {
-                            mutation.put(new ImmutableBytesPtr(iterator.next().getLowerRange()), new RowMutationState(PRow.DELETE_MARKER, statement.getConnection().getStatementExecutionCounter(), NULL_ROWTIMESTAMP_INFO, null));
-                        }
-                        return new MutationState(plan.getTableRef(), mutation, 0, maxSize, maxSizeBytes, connection);
-                    }
-    
-                    @Override
-                    public ExplainPlan getExplainPlan() throws SQLException {
-                        return new ExplainPlan(Collections.singletonList("DELETE SINGLE ROW"));
-                    }
-    
-                    @Override
-                    public StatementContext getContext() {
-                        return context;
-                    }
-
-                    @Override
-                    public TableRef getTargetRef() {
-                        return dataPlan.getTableRef();
-                    }
-
-                    @Override
-                    public Set<TableRef> getSourceRefs() {
-                        // Don't include the target
-                        return Collections.emptySet();
-                    }
-
-            		@Override
-            		public Operation getOperation() {
-            			return operation;
-            		}
-
-                    @Override
-                    public Long getEstimatedRowsToScan() throws SQLException {
-                        return 0l;
-                    }
-
-                    @Override
-                    public Long getEstimatedBytesToScan() throws SQLException {
-                        return 0l;
-                    }
-
-                    @Override
-                    public Long getEstimateInfoTimestamp() throws SQLException {
-                        return 0l;
-                    }
-                });
+                mutationPlans.add(new SingleRowDeleteMutationPlan(plan, connection, maxSize, maxSizeBytes));
             }
-            return new MultiDeleteMutationPlan(mutationPlans);
+            return new MultiRowDeleteMutationPlan(dataPlan, mutationPlans);
         } else if (runOnServer) {
             // TODO: better abstraction
             final StatementContext context = dataPlan.getContext();
@@ -629,91 +576,7 @@ public class DeleteCompiler {
             final RowProjector projector = projectorToBe;
             final QueryPlan aggPlan = new AggregatePlan(context, select, dataPlan.getTableRef(), projector, null, null,
                     OrderBy.EMPTY_ORDER_BY, null, GroupBy.EMPTY_GROUP_BY, null);
-            return new MutationPlan() {
-                        @Override
-                        public ParameterMetaData getParameterMetaData() {
-                            return context.getBindManager().getParameterMetaData();
-                        }
-        
-                        @Override
-                        public StatementContext getContext() {
-                            return context;
-                        }
-        
-                        @Override
-                        public TableRef getTargetRef() {
-                            return dataPlan.getTableRef();
-                        }
-    
-                        @Override
-                        public Set<TableRef> getSourceRefs() {
-                            return dataPlan.getSourceRefs();
-                        }
-    
-                		@Override
-                		public Operation getOperation() {
-                			return operation;
-                		}
-    
-                        @Override
-                        public MutationState execute() throws SQLException {
-                            // TODO: share this block of code with UPSERT SELECT
-                            ImmutableBytesWritable ptr = context.getTempPtr();
-                            PTable table = dataPlan.getTableRef().getTable();
-                            table.getIndexMaintainers(ptr, context.getConnection());
-                            byte[] txState = table.isTransactional() ? connection.getMutationState().encodeTransaction() : ByteUtil.EMPTY_BYTE_ARRAY;
-                            ServerCache cache = null;
-                            try {
-                                if (ptr.getLength() > 0) {
-                                    byte[] uuidValue = ServerCacheClient.generateId();
-                                    context.getScan().setAttribute(PhoenixIndexCodec.INDEX_UUID, uuidValue);
-                                    context.getScan().setAttribute(PhoenixIndexCodec.INDEX_PROTO_MD, ptr.get());
-                                    context.getScan().setAttribute(BaseScannerRegionObserver.TX_STATE, txState);
-                                }
-                                ResultIterator iterator = aggPlan.iterator();
-                                try {
-                                    Tuple row = iterator.next();
-                                    final long mutationCount = (Long)projector.getColumnProjector(0).getValue(row, PLong.INSTANCE, ptr);
-                                    return new MutationState(maxSize, maxSizeBytes, connection) {
-                                        @Override
-                                        public long getUpdateCount() {
-                                            return mutationCount;
-                                        }
-                                    };
-                                } finally {
-                                    iterator.close();
-                                }
-                            } finally {
-                                if (cache != null) {
-                                    cache.close();
-                                }
-                            }
-                        }
-        
-                        @Override
-                        public ExplainPlan getExplainPlan() throws SQLException {
-                            List<String> queryPlanSteps =  aggPlan.getExplainPlan().getPlanSteps();
-                            List<String> planSteps = Lists.newArrayListWithExpectedSize(queryPlanSteps.size()+1);
-                            planSteps.add("DELETE ROWS");
-                            planSteps.addAll(queryPlanSteps);
-                            return new ExplainPlan(planSteps);
-                        }
-    
-                        @Override
-                        public Long getEstimatedRowsToScan() throws SQLException {
-                            return aggPlan.getEstimatedRowsToScan();
-                        }
-    
-                        @Override
-                        public Long getEstimatedBytesToScan() throws SQLException {
-                            return aggPlan.getEstimatedBytesToScan();
-                        }
-    
-                        @Override
-                        public Long getEstimateInfoTimestamp() throws SQLException {
-                            return aggPlan.getEstimateInfoTimestamp();
-                        }
-                    };
+            return new ServerSelectDeleteMutationPlan(dataPlan, connection, aggPlan, projector, maxSize, maxSizeBytes);
         } else {
             final DeletingParallelIteratorFactory parallelIteratorFactory = parallelIteratorFactoryToBe;
             List<PColumn> adjustedProjectedColumns = Lists.newArrayListWithExpectedSize(projectedColumns.size());
@@ -749,90 +612,322 @@ public class DeleteCompiler {
             if (!bestPlan.getTableRef().getTable().equals(targetTableRef.getTable())) {
                 otherTableRefs.add(projectedTableRef);
             }
-            final StatementContext context = bestPlan.getContext();
-            return new MutationPlan() {
-                @Override
-                public ParameterMetaData getParameterMetaData() {
-                    return context.getBindManager().getParameterMetaData();
-                }
+            return new ClientSelectDeleteMutationPlan(targetTableRef, dataPlan, bestPlan, hasPreOrPostProcessing,
+                    parallelIteratorFactory, otherTableRefs, projectedTableRef, maxSize, maxSizeBytes, connection);
+        }
+    }
 
-                @Override
-                public StatementContext getContext() {
-                    return context;
-                }
+    private class SingleRowDeleteMutationPlan implements MutationPlan {
 
-                @Override
-                public TableRef getTargetRef() {
-                    return targetTableRef;
-                }
+        private final QueryPlan dataPlan;
+        private final PhoenixConnection connection;
+        private final int maxSize;
+        private final StatementContext context;
+        private final int maxSizeBytes;
 
-                @Override
-                public Set<TableRef> getSourceRefs() {
-                    return dataPlan.getSourceRefs();
-                }
+        public SingleRowDeleteMutationPlan(QueryPlan dataPlan, PhoenixConnection connection, int maxSize, int maxSizeBytes) {
+            this.dataPlan = dataPlan;
+            this.connection = connection;
+            this.maxSize = maxSize;
+            this.context = dataPlan.getContext();
+            this.maxSizeBytes = maxSizeBytes;
+        }
+
+        @Override
+        public ParameterMetaData getParameterMetaData() {
+            return context.getBindManager().getParameterMetaData();
+        }
 
-        		@Override
-        		public Operation getOperation() {
-        			return operation;
-        		}
-
-                @Override
-                public MutationState execute() throws SQLException {
-                    ResultIterator iterator = bestPlan.iterator();
-                    try {
-                        if (!hasPreOrPostProcessing) {
-                            Tuple tuple;
-                            long totalRowCount = 0;
-                            if (parallelIteratorFactory != null) {
-                                parallelIteratorFactory.setQueryPlan(bestPlan);
-                                parallelIteratorFactory.setOtherTableRefs(otherTableRefs);
-                                parallelIteratorFactory.setProjectedTableRef(projectedTableRef);
-                            }
-                            while ((tuple=iterator.next()) != null) {// Runs query
-                                Cell kv = tuple.getValue(0);
-                                totalRowCount += PLong.INSTANCE.getCodec().decodeLong(kv.getValueArray(), kv.getValueOffset(), SortOrder.getDefault());
-                            }
-                            // Return total number of rows that have been deleted from the table. In the case of auto commit being off
-                            // the mutations will all be in the mutation state of the current connection. We need to divide by the
-                            // total number of tables we updated as otherwise the client will get an unexpected result
-                            MutationState state = new MutationState(maxSize, maxSizeBytes, connection, totalRowCount / ((bestPlan.getTableRef().getTable().getIndexType() == IndexType.LOCAL && !otherTableRefs.isEmpty() ? 0 : 1) + otherTableRefs.size()));
-
-                            // set the read metrics accumulated in the parent context so that it can be published when the mutations are committed.
-                            state.setReadMetricQueue(context.getReadMetricsQueue());
-
-                            return state;
-                        } else {
-                            return deleteRows(context, iterator, bestPlan, projectedTableRef, otherTableRefs);
+        @Override
+        public MutationState execute() throws SQLException {
+            // We have a point lookup, so we know we have a simple set of fully qualified
+            // keys for our ranges
+            ScanRanges ranges = context.getScanRanges();
+            Iterator<KeyRange> iterator = ranges.getPointLookupKeyIterator();
+            Map<ImmutableBytesPtr,RowMutationState> mutation = Maps.newHashMapWithExpectedSize(ranges.getPointLookupCount());
+            while (iterator.hasNext()) {
+                mutation.put(new ImmutableBytesPtr(iterator.next().getLowerRange()),
+                        new RowMutationState(PRow.DELETE_MARKER,
+                                statement.getConnection().getStatementExecutionCounter(), NULL_ROWTIMESTAMP_INFO, null));
+            }
+            return new MutationState(dataPlan.getTableRef(), mutation, 0, maxSize, maxSizeBytes, connection);
+        }
+
+        @Override
+        public ExplainPlan getExplainPlan() throws SQLException {
+            return new ExplainPlan(Collections.singletonList("DELETE SINGLE ROW"));
+        }
+
+        @Override
+        public QueryPlan getQueryPlan() {
+            return dataPlan;
+        }
+
+        @Override
+        public StatementContext getContext() {
+            return context;
+        }
+
+        @Override
+        public TableRef getTargetRef() {
+            return dataPlan.getTableRef();
+        }
+
+        @Override
+        public Set<TableRef> getSourceRefs() {
+            // Don't include the target
+            return Collections.emptySet();
+        }
+
+        @Override
+        public Operation getOperation() {
+          return operation;
+        }
+
+        @Override
+        public Long getEstimatedRowsToScan() throws SQLException {
+            return 0l;
+        }
+
+        @Override
+        public Long getEstimatedBytesToScan() throws SQLException {
+            return 0l;
+        }
+
+        @Override
+        public Long getEstimateInfoTimestamp() throws SQLException {
+            return 0l;
+        }
+    }
+
+    private class ServerSelectDeleteMutationPlan implements MutationPlan {
+        private final StatementContext context;
+        private final QueryPlan dataPlan;
+        private final PhoenixConnection connection;
+        private final QueryPlan aggPlan;
+        private final RowProjector projector;
+        private final int maxSize;
+        private final int maxSizeBytes;
+
+        public ServerSelectDeleteMutationPlan(QueryPlan dataPlan, PhoenixConnection connection, QueryPlan aggPlan,
+                                              RowProjector projector, int maxSize, int maxSizeBytes) {
+            this.context = dataPlan.getContext();
+            this.dataPlan = dataPlan;
+            this.connection = connection;
+            this.aggPlan = aggPlan;
+            this.projector = projector;
+            this.maxSize = maxSize;
+            this.maxSizeBytes = maxSizeBytes;
+        }
+
+        @Override
+        public ParameterMetaData getParameterMetaData() {
+            return context.getBindManager().getParameterMetaData();
+        }
+
+        @Override
+        public StatementContext getContext() {
+            return context;
+        }
+
+        @Override
+        public TableRef getTargetRef() {
+            return dataPlan.getTableRef();
+        }
+
+        @Override
+        public Set<TableRef> getSourceRefs() {
+            return dataPlan.getSourceRefs();
+        }
+
+        @Override
+        public Operation getOperation() {
+          return operation;
+        }
+
+        @Override
+        public MutationState execute() throws SQLException {
+            // TODO: share this block of code with UPSERT SELECT
+            ImmutableBytesWritable ptr = context.getTempPtr();
+            PTable table = dataPlan.getTableRef().getTable();
+            table.getIndexMaintainers(ptr, context.getConnection());
+            byte[] txState = table.isTransactional() ? connection.getMutationState().encodeTransaction() : ByteUtil.EMPTY_BYTE_ARRAY;
+            ServerCache cache = null;
+            try {
+                if (ptr.getLength() > 0) {
+                    byte[] uuidValue = ServerCacheClient.generateId();
+                    context.getScan().setAttribute(PhoenixIndexCodec.INDEX_UUID, uuidValue);
+                    context.getScan().setAttribute(PhoenixIndexCodec.INDEX_PROTO_MD, ptr.get());
+                    context.getScan().setAttribute(BaseScannerRegionObserver.TX_STATE, txState);
+                }
+                ResultIterator iterator = aggPlan.iterator();
+                try {
+                    Tuple row = iterator.next();
+                    final long mutationCount = (Long) projector.getColumnProjector(0).getValue(row, PLong.INSTANCE, ptr);
+                    return new MutationState(maxSize, maxSizeBytes, connection) {
+                        @Override
+                        public long getUpdateCount() {
+                            return mutationCount;
                         }
-                    } finally {
-                        iterator.close();
-                    }
+                    };
+                } finally {
+                    iterator.close();
                 }
-
-                @Override
-                public ExplainPlan getExplainPlan() throws SQLException {
-                    List<String> queryPlanSteps =  bestPlan.getExplainPlan().getPlanSteps();
-                    List<String> planSteps = Lists.newArrayListWithExpectedSize(queryPlanSteps.size()+1);
-                    planSteps.add("DELETE ROWS");
-                    planSteps.addAll(queryPlanSteps);
-                    return new ExplainPlan(planSteps);
+            } finally {
+                if (cache != null) {
+                    cache.close();
                 }
+            }
+        }
 
-                @Override
-                public Long getEstimatedRowsToScan() throws SQLException {
-                    return bestPlan.getEstimatedRowsToScan();
-                }
+        @Override
+        public ExplainPlan getExplainPlan() throws SQLException {
+            List<String> queryPlanSteps =  aggPlan.getExplainPlan().getPlanSteps();
+            List<String> planSteps = Lists.newArrayListWithExpectedSize(queryPlanSteps.size()+1);
+            planSteps.add("DELETE ROWS");
+            planSteps.addAll(queryPlanSteps);
+            return new ExplainPlan(planSteps);
+        }
 
-                @Override
-                public Long getEstimatedBytesToScan() throws SQLException {
-                    return bestPlan.getEstimatedBytesToScan();
-                }
+        @Override
+        public Long getEstimatedRowsToScan() throws SQLException {
+            return aggPlan.getEstimatedRowsToScan();
+        }
 
-                @Override
-                public Long getEstimateInfoTimestamp() throws SQLException {
-                    return bestPlan.getEstimateInfoTimestamp();
+        @Override
+        public Long getEstimatedBytesToScan() throws SQLException {
+            return aggPlan.getEstimatedBytesToScan();
+        }
+
+        @Override
+        public Long getEstimateInfoTimestamp() throws SQLException {
+            return aggPlan.getEstimateInfoTimestamp();
+        }
+
+        @Override
+        public QueryPlan getQueryPlan() {
+            return aggPlan;
+        }
+    }
+
+    private class ClientSelectDeleteMutationPlan implements MutationPlan {
+        private final StatementContext context;
+        private final TableRef targetTableRef;
+        private final QueryPlan dataPlan;
+        private final QueryPlan bestPlan;
+        private final boolean hasPreOrPostProcessing;
+        private final DeletingParallelIteratorFactory parallelIteratorFactory;
+        private final List<TableRef> otherTableRefs;
+        private final TableRef projectedTableRef;
+        private final int maxSize;
+        private final int maxSizeBytes;
+        private final PhoenixConnection connection;
+
+        public ClientSelectDeleteMutationPlan(TableRef targetTableRef, QueryPlan dataPlan, QueryPlan bestPlan,
+                                              boolean hasPreOrPostProcessing,
+                                              DeletingParallelIteratorFactory parallelIteratorFactory,
+                                              List<TableRef> otherTableRefs, TableRef projectedTableRef, int maxSize,
+                                              int maxSizeBytes, PhoenixConnection connection) {
+            this.context = bestPlan.getContext();
+            this.targetTableRef = targetTableRef;
+            this.dataPlan = dataPlan;
+            this.bestPlan = bestPlan;
+            this.hasPreOrPostProcessing = hasPreOrPostProcessing;
+            this.parallelIteratorFactory = parallelIteratorFactory;
+            this.otherTableRefs = otherTableRefs;
+            this.projectedTableRef = projectedTableRef;
+            this.maxSize = maxSize;
+            this.maxSizeBytes = maxSizeBytes;
+            this.connection = connection;
+        }
+
+        @Override
+        public ParameterMetaData getParameterMetaData() {
+            return context.getBindManager().getParameterMetaData();
+        }
+
+        @Override
+        public StatementContext getContext() {
+            return context;
+        }
+
+        @Override
+        public TableRef getTargetRef() {
+            return targetTableRef;
+        }
+
+        @Override
+        public Set<TableRef> getSourceRefs() {
+            return dataPlan.getSourceRefs();
+        }
+
+        @Override
+        public Operation getOperation() {
+          return operation;
+        }
+
+        @Override
+        public MutationState execute() throws SQLException {
+            ResultIterator iterator = bestPlan.iterator();
+            try {
+                if (!hasPreOrPostProcessing) {
+                    Tuple tuple;
+                    long totalRowCount = 0;
+                    if (parallelIteratorFactory != null) {
+                        parallelIteratorFactory.setQueryPlan(bestPlan);
+                        parallelIteratorFactory.setOtherTableRefs(otherTableRefs);
+                        parallelIteratorFactory.setProjectedTableRef(projectedTableRef);
+                    }
+                    while ((tuple=iterator.next()) != null) {// Runs query
+                        Cell kv = tuple.getValue(0);
+                        totalRowCount += PLong.INSTANCE.getCodec().decodeLong(kv.getValueArray(), kv.getValueOffset(), SortOrder.getDefault());
+                    }
+                    // Return total number of rows that have been deleted from the table. In the case of auto commit being off
+                    // the mutations will all be in the mutation state of the current connection. We need to divide by the
+                    // total number of tables we updated as otherwise the client will get an unexpected result
+                    MutationState state = new MutationState(maxSize, maxSizeBytes, connection,
+                            totalRowCount /
+                                    ((bestPlan.getTableRef().getTable().getIndexType() == IndexType.LOCAL && !otherTableRefs.isEmpty() ? 0 : 1) + otherTableRefs.size()));
+
+                    // set the read metrics accumulated in the parent context so that it can be published when the mutations are committed.
+                    state.setReadMetricQueue(context.getReadMetricsQueue());
+
+                    return state;
+                } else {
+                    return deleteRows(context, iterator, bestPlan, projectedTableRef, otherTableRefs);
                 }
-            };
+            } finally {
+                iterator.close();
+            }
+        }
+
+        @Override
+        public ExplainPlan getExplainPlan() throws SQLException {
+            List<String> queryPlanSteps =  bestPlan.getExplainPlan().getPlanSteps();
+            List<String> planSteps = Lists.newArrayListWithExpectedSize(queryPlanSteps.size()+1);
+            planSteps.add("DELETE ROWS");
+            planSteps.addAll(queryPlanSteps);
+            return new ExplainPlan(planSteps);
+        }
+
+        @Override
+        public Long getEstimatedRowsToScan() throws SQLException {
+            return bestPlan.getEstimatedRowsToScan();
+        }
+
+        @Override
+        public Long getEstimatedBytesToScan() throws SQLException {
+            return bestPlan.getEstimatedBytesToScan();
+        }
+
+        @Override
+        public Long getEstimateInfoTimestamp() throws SQLException {
+            return bestPlan.getEstimateInfoTimestamp();
+        }
+
+        @Override
+        public QueryPlan getQueryPlan() {
+            return bestPlan;
         }
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a23439cf/phoenix-core/src/main/java/org/apache/phoenix/compile/MutationPlan.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/MutationPlan.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/MutationPlan.java
index ddc2004..97f3f3d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/MutationPlan.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/MutationPlan.java
@@ -24,6 +24,7 @@ import org.apache.phoenix.schema.TableRef;
 
 
 public interface MutationPlan extends StatementPlan {
-    public MutationState execute() throws SQLException;
-    public TableRef getTargetRef();
+    MutationState execute() throws SQLException;
+    TableRef getTargetRef();
+    QueryPlan getQueryPlan();
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a23439cf/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
index 9eaaf62..bc3e289 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
@@ -695,173 +695,13 @@ public class UpsertCompiler {
                     
                     // Ignore order by - it has no impact
                     final QueryPlan aggPlan = new AggregatePlan(context, select, statementContext.getCurrentTable(), aggProjector, null,null, OrderBy.EMPTY_ORDER_BY, null, GroupBy.EMPTY_GROUP_BY, null);
-                    return new MutationPlan() {
-                        @Override
-                        public ParameterMetaData getParameterMetaData() {
-                            return queryPlan.getContext().getBindManager().getParameterMetaData();
-                        }
-    
-                        @Override
-                        public StatementContext getContext() {
-                            return queryPlan.getContext();
-                        }
-
-                        @Override
-                        public TableRef getTargetRef() {
-                            return tableRef;
-                        }
-
-                        @Override
-                        public Set<TableRef> getSourceRefs() {
-                            return originalQueryPlan.getSourceRefs();
-                        }
-
-                		@Override
-                		public Operation getOperation() {
-                			return operation;
-                		}
-
-                        @Override
-                        public MutationState execute() throws SQLException {
-                            ImmutableBytesWritable ptr = context.getTempPtr();
-                            PTable table = tableRef.getTable();
-                            table.getIndexMaintainers(ptr, context.getConnection());
-                            byte[] txState = table.isTransactional() ? connection.getMutationState().encodeTransaction() : ByteUtil.EMPTY_BYTE_ARRAY;
-
-                            if (ptr.getLength() > 0) {
-                                byte[] uuidValue = ServerCacheClient.generateId();
-                                scan.setAttribute(PhoenixIndexCodec.INDEX_UUID, uuidValue);
-                                scan.setAttribute(PhoenixIndexCodec.INDEX_PROTO_MD, ptr.get());
-                                scan.setAttribute(BaseScannerRegionObserver.TX_STATE, txState);
-                            }
-                            ResultIterator iterator = aggPlan.iterator();
-                            try {
-                                Tuple row = iterator.next();
-                                final long mutationCount = (Long)aggProjector.getColumnProjector(0).getValue(row,
-                                        PLong.INSTANCE, ptr);
-                                return new MutationState(maxSize, maxSizeBytes, connection) {
-                                    @Override
-                                    public long getUpdateCount() {
-                                        return mutationCount;
-                                    }
-                                };
-                            } finally {
-                                iterator.close();
-                            }
-                            
-                        }
-
-                        @Override
-                        public ExplainPlan getExplainPlan() throws SQLException {
-                            List<String> queryPlanSteps =  aggPlan.getExplainPlan().getPlanSteps();
-                            List<String> planSteps = Lists.newArrayListWithExpectedSize(queryPlanSteps.size()+1);
-                            planSteps.add("UPSERT ROWS");
-                            planSteps.addAll(queryPlanSteps);
-                            return new ExplainPlan(planSteps);
-                        }
-
-                        @Override
-                        public Long getEstimatedRowsToScan() throws SQLException {
-                            return aggPlan.getEstimatedRowsToScan();
-                        }
-
-                        @Override
-                        public Long getEstimatedBytesToScan() throws SQLException {
-                            return aggPlan.getEstimatedBytesToScan();
-                        }
-
-                        @Override
-                        public Long getEstimateInfoTimestamp() throws SQLException {
-                            return aggPlan.getEstimateInfoTimestamp();
-                        }
-                    };
+                    return new ServerUpsertSelectMutationPlan(queryPlan, tableRef, originalQueryPlan, context, connection, scan, aggPlan, aggProjector, maxSize, maxSizeBytes);
                 }
             }
             ////////////////////////////////////////////////////////////////////
             // UPSERT SELECT run client-side
             /////////////////////////////////////////////////////////////////////
-            return new MutationPlan() {
-                @Override
-                public ParameterMetaData getParameterMetaData() {
-                    return queryPlan.getContext().getBindManager().getParameterMetaData();
-                }
-
-                @Override
-                public StatementContext getContext() {
-                    return queryPlan.getContext();
-                }
-
-                @Override
-                public TableRef getTargetRef() {
-                    return tableRef;
-                }
-
-                @Override
-                public Set<TableRef> getSourceRefs() {
-                    return originalQueryPlan.getSourceRefs();
-                }
-
-        		@Override
-        		public Operation getOperation() {
-        			return operation;
-        		}
-
-                @Override
-                public MutationState execute() throws SQLException {
-                    ResultIterator iterator = queryPlan.iterator();
-                    if (parallelIteratorFactory == null) {
-                        return upsertSelect(new StatementContext(statement), tableRef, projector, iterator, columnIndexes, pkSlotIndexes, useServerTimestamp, false);
-                    }
-                    try {
-                        parallelIteratorFactory.setRowProjector(projector);
-                        parallelIteratorFactory.setColumnIndexes(columnIndexes);
-                        parallelIteratorFactory.setPkSlotIndexes(pkSlotIndexes);
-                        Tuple tuple;
-                        long totalRowCount = 0;
-                        StatementContext context = queryPlan.getContext();
-                        while ((tuple=iterator.next()) != null) {// Runs query
-                            Cell kv = tuple.getValue(0);
-                            totalRowCount += PLong.INSTANCE.getCodec().decodeLong(kv.getValueArray(), kv.getValueOffset(), SortOrder.getDefault());
-                        }
-                        // Return total number of rows that have been updated. In the case of auto commit being off
-                        // the mutations will all be in the mutation state of the current connection.
-                        MutationState mutationState = new MutationState(maxSize, maxSizeBytes, statement.getConnection(), totalRowCount);
-                        /*
-                         *  All the metrics collected for measuring the reads done by the parallel mutating iterators
-                         *  is included in the ReadMetricHolder of the statement context. Include these metrics in the
-                         *  returned mutation state so they can be published on commit. 
-                         */
-                        mutationState.setReadMetricQueue(context.getReadMetricsQueue());
-                        return mutationState; 
-                    } finally {
-                        iterator.close();
-                    }
-                }
-
-                @Override
-                public ExplainPlan getExplainPlan() throws SQLException {
-                    List<String> queryPlanSteps =  queryPlan.getExplainPlan().getPlanSteps();
-                    List<String> planSteps = Lists.newArrayListWithExpectedSize(queryPlanSteps.size()+1);
-                    planSteps.add("UPSERT SELECT");
-                    planSteps.addAll(queryPlanSteps);
-                    return new ExplainPlan(planSteps);
-                }
-
-                @Override
-                public Long getEstimatedRowsToScan() throws SQLException {
-                    return queryPlan.getEstimatedRowsToScan();
-                }
-
-                @Override
-                public Long getEstimatedBytesToScan() throws SQLException {
-                    return queryPlan.getEstimatedBytesToScan();
-                }
-
-                @Override
-                public Long getEstimateInfoTimestamp() throws SQLException {
-                    return queryPlan.getEstimateInfoTimestamp();
-                }
-            };
+            return new ClientUpsertSelectMutationPlan(queryPlan, tableRef, originalQueryPlan, parallelIteratorFactory, projector, columnIndexes, pkSlotIndexes, useServerTimestamp, maxSize, maxSizeBytes);
         }
 
             
@@ -987,124 +827,9 @@ public class UpsertCompiler {
         }
         final byte[] onDupKeyBytes = onDupKeyBytesToBe;
         
-        return new MutationPlan() {
-            @Override
-            public ParameterMetaData getParameterMetaData() {
-                return context.getBindManager().getParameterMetaData();
-            }
-
-            @Override
-            public StatementContext getContext() {
-                return context;
-            }
-
-            @Override
-            public TableRef getTargetRef() {
-                return tableRef;
-            }
-
-            @Override
-            public Set<TableRef> getSourceRefs() {
-                return Collections.emptySet();
-            }
-
-    		@Override
-    		public Operation getOperation() {
-    			return operation;
-    		}
-
-            @Override
-            public MutationState execute() throws SQLException {
-                ImmutableBytesWritable ptr = context.getTempPtr();
-                final SequenceManager sequenceManager = context.getSequenceManager();
-                // Next evaluate all the expressions
-                int nodeIndex = nodeIndexOffset;
-                PTable table = tableRef.getTable();
-                Tuple tuple = sequenceManager.getSequenceCount() == 0 ? null :
-                    sequenceManager.newSequenceTuple(null);
-                for (Expression constantExpression : constantExpressions) {
-                    PColumn column = allColumns.get(columnIndexes[nodeIndex]);
-                    constantExpression.evaluate(tuple, ptr);
-                    Object value = null;
-                    if (constantExpression.getDataType() != null) {
-                        value = constantExpression.getDataType().toObject(ptr, constantExpression.getSortOrder(), constantExpression.getMaxLength(), constantExpression.getScale());
-                        if (!constantExpression.getDataType().isCoercibleTo(column.getDataType(), value)) { 
-                            throw TypeMismatchException.newException(
-                                constantExpression.getDataType(), column.getDataType(), "expression: "
-                                        + constantExpression.toString() + " in column " + column);
-                        }
-                        if (!column.getDataType().isSizeCompatible(ptr, value, constantExpression.getDataType(),
-                                constantExpression.getSortOrder(), constantExpression.getMaxLength(), 
-                                constantExpression.getScale(), column.getMaxLength(), column.getScale())) { 
-                            throw new SQLExceptionInfo.Builder(
-                                SQLExceptionCode.DATA_EXCEEDS_MAX_CAPACITY).setColumnName(column.getName().getString())
-                                .setMessage("value=" + constantExpression.toString()).build().buildException();
-                        }
-                    }
-                    column.getDataType().coerceBytes(ptr, value, constantExpression.getDataType(), 
-                            constantExpression.getMaxLength(), constantExpression.getScale(), constantExpression.getSortOrder(),
-                            column.getMaxLength(), column.getScale(),column.getSortOrder(),
-                            table.rowKeyOrderOptimizable());
-                    if (overlapViewColumns.contains(column) && Bytes.compareTo(ptr.get(), ptr.getOffset(), ptr.getLength(), column.getViewConstant(), 0, column.getViewConstant().length-1) != 0) {
-                        throw new SQLExceptionInfo.Builder(
-                                SQLExceptionCode.CANNOT_UPDATE_VIEW_COLUMN)
-                                .setColumnName(column.getName().getString())
-                                .setMessage("value=" + constantExpression.toString()).build().buildException();
-                    }
-                    values[nodeIndex] = ByteUtil.copyKeyBytesIfNecessary(ptr);
-                    nodeIndex++;
-                }
-                // Add columns based on view
-                for (PColumn column : addViewColumns) {
-                    if (IndexUtil.getViewConstantValue(column, ptr)) {
-                        values[nodeIndex++] = ByteUtil.copyKeyBytesIfNecessary(ptr);
-                    } else {
-                        throw new IllegalStateException();
-                    }
-                }
-                Map<ImmutableBytesPtr, RowMutationState> mutation = Maps.newHashMapWithExpectedSize(1);
-                IndexMaintainer indexMaintainer = null;
-                byte[][] viewConstants = null;
-                if (table.getIndexType() == IndexType.LOCAL) {
-                    PTable parentTable =
-                            statement
-                                    .getConnection()
-                                    .getMetaDataCache()
-                                    .getTableRef(
-                                        new PTableKey(statement.getConnection().getTenantId(),
-                                                table.getParentName().getString())).getTable();
-                    indexMaintainer = table.getIndexMaintainer(parentTable, connection);
-                    viewConstants = IndexUtil.getViewConstants(parentTable);
-                }
-                setValues(values, pkSlotIndexes, columnIndexes, table, mutation, statement, useServerTimestamp, indexMaintainer, viewConstants, onDupKeyBytes, 0);
-                return new MutationState(tableRef, mutation, 0, maxSize, maxSizeBytes, connection);
-            }
-
-            @Override
-            public ExplainPlan getExplainPlan() throws SQLException {
-                List<String> planSteps = Lists.newArrayListWithExpectedSize(2);
-                if (context.getSequenceManager().getSequenceCount() > 0) {
-                    planSteps.add("CLIENT RESERVE " + context.getSequenceManager().getSequenceCount() + " SEQUENCES");
-                }
-                planSteps.add("PUT SINGLE ROW");
-                return new ExplainPlan(planSteps);
-            }
-
-            @Override
-            public Long getEstimatedRowsToScan() throws SQLException {
-                return 0l;
-            }
-
-            @Override
-            public Long getEstimatedBytesToScan() throws SQLException {
-                return 0l;
-            }
-
-            @Override
-            public Long getEstimateInfoTimestamp() throws SQLException {
-                return 0l;
-            }
-        };
+        return new UpsertValuesMutationPlan(context, tableRef, nodeIndexOffset, constantExpressions,
+                allColumns, columnIndexes, overlapViewColumns, values, addViewColumns,
+                connection, pkSlotIndexes, useServerTimestamp, onDupKeyBytes, maxSize, maxSizeBytes);
     }
     
     private static boolean isRowTimestampSet(int[] pkSlotIndexes, PTable table) {
@@ -1215,4 +940,394 @@ public class UpsertCompiler {
             }
         }
     }
+
+    private class ServerUpsertSelectMutationPlan implements MutationPlan {
+        private final QueryPlan queryPlan;
+        private final TableRef tableRef;
+        private final QueryPlan originalQueryPlan;
+        private final StatementContext context;
+        private final PhoenixConnection connection;
+        private final Scan scan;
+        private final QueryPlan aggPlan;
+        private final RowProjector aggProjector;
+        private final int maxSize;
+        private final int maxSizeBytes;
+
+        public ServerUpsertSelectMutationPlan(QueryPlan queryPlan, TableRef tableRef, QueryPlan originalQueryPlan,
+                                              StatementContext context, PhoenixConnection connection,
+                                              Scan scan, QueryPlan aggPlan, RowProjector aggProjector,
+                                              int maxSize, int maxSizeBytes) {
+            this.queryPlan = queryPlan;
+            this.tableRef = tableRef;
+            this.originalQueryPlan = originalQueryPlan;
+            this.context = context;
+            this.connection = connection;
+            this.scan = scan;
+            this.aggPlan = aggPlan;
+            this.aggProjector = aggProjector;
+            this.maxSize = maxSize;
+            this.maxSizeBytes = maxSizeBytes;
+        }
+
+        @Override
+        public ParameterMetaData getParameterMetaData() {
+            return queryPlan.getContext().getBindManager().getParameterMetaData();
+        }
+
+        @Override
+        public StatementContext getContext() {
+            return queryPlan.getContext();
+        }
+
+        @Override
+        public TableRef getTargetRef() {
+            return tableRef;
+        }
+
+        @Override
+        public QueryPlan getQueryPlan() {
+            return aggPlan;
+        }
+
+        @Override
+        public Set<TableRef> getSourceRefs() {
+            return originalQueryPlan.getSourceRefs();
+        }
+
+        @Override
+        public Operation getOperation() {
+          return operation;
+        }
+
+        @Override
+        public MutationState execute() throws SQLException {
+            ImmutableBytesWritable ptr = context.getTempPtr();
+            PTable table = tableRef.getTable();
+            table.getIndexMaintainers(ptr, context.getConnection());
+            byte[] txState = table.isTransactional() ?
+                    connection.getMutationState().encodeTransaction() : ByteUtil.EMPTY_BYTE_ARRAY;
+
+            if (ptr.getLength() > 0) {
+                byte[] uuidValue = ServerCacheClient.generateId();
+                scan.setAttribute(PhoenixIndexCodec.INDEX_UUID, uuidValue);
+                scan.setAttribute(PhoenixIndexCodec.INDEX_PROTO_MD, ptr.get());
+                scan.setAttribute(BaseScannerRegionObserver.TX_STATE, txState);
+            }
+            ResultIterator iterator = aggPlan.iterator();
+            try {
+                Tuple row = iterator.next();
+                final long mutationCount = (Long) aggProjector.getColumnProjector(0).getValue(row,
+                        PLong.INSTANCE, ptr);
+                return new MutationState(maxSize, maxSizeBytes, connection) {
+                    @Override
+                    public long getUpdateCount() {
+                        return mutationCount;
+                    }
+                };
+            } finally {
+                iterator.close();
+            }
+
+        }
+
+        @Override
+        public ExplainPlan getExplainPlan() throws SQLException {
+            List<String> queryPlanSteps =  aggPlan.getExplainPlan().getPlanSteps();
+            List<String> planSteps = Lists.newArrayListWithExpectedSize(queryPlanSteps.size()+1);
+            planSteps.add("UPSERT ROWS");
+            planSteps.addAll(queryPlanSteps);
+            return new ExplainPlan(planSteps);
+        }
+
+        @Override
+        public Long getEstimatedRowsToScan() throws SQLException {
+            return aggPlan.getEstimatedRowsToScan();
+        }
+
+        @Override
+        public Long getEstimatedBytesToScan() throws SQLException {
+            return aggPlan.getEstimatedBytesToScan();
+        }
+
+        @Override
+        public Long getEstimateInfoTimestamp() throws SQLException {
+            return aggPlan.getEstimateInfoTimestamp();
+        }
+    }
+
+    private class UpsertValuesMutationPlan implements MutationPlan {
+        private final StatementContext context;
+        private final TableRef tableRef;
+        private final int nodeIndexOffset;
+        private final List<Expression> constantExpressions;
+        private final List<PColumn> allColumns;
+        private final int[] columnIndexes;
+        private final Set<PColumn> overlapViewColumns;
+        private final byte[][] values;
+        private final Set<PColumn> addViewColumns;
+        private final PhoenixConnection connection;
+        private final int[] pkSlotIndexes;
+        private final boolean useServerTimestamp;
+        private final byte[] onDupKeyBytes;
+        private final int maxSize;
+        private final int maxSizeBytes;
+
+        public UpsertValuesMutationPlan(StatementContext context, TableRef tableRef, int nodeIndexOffset,
+                                        List<Expression> constantExpressions, List<PColumn> allColumns,
+                                        int[] columnIndexes, Set<PColumn> overlapViewColumns, byte[][] values,
+                                        Set<PColumn> addViewColumns, PhoenixConnection connection,
+                                        int[] pkSlotIndexes, boolean useServerTimestamp, byte[] onDupKeyBytes,
+                                        int maxSize, int maxSizeBytes) {
+            this.context = context;
+            this.tableRef = tableRef;
+            this.nodeIndexOffset = nodeIndexOffset;
+            this.constantExpressions = constantExpressions;
+            this.allColumns = allColumns;
+            this.columnIndexes = columnIndexes;
+            this.overlapViewColumns = overlapViewColumns;
+            this.values = values;
+            this.addViewColumns = addViewColumns;
+            this.connection = connection;
+            this.pkSlotIndexes = pkSlotIndexes;
+            this.useServerTimestamp = useServerTimestamp;
+            this.onDupKeyBytes = onDupKeyBytes;
+            this.maxSize = maxSize;
+            this.maxSizeBytes = maxSizeBytes;
+        }
+
+        @Override
+        public ParameterMetaData getParameterMetaData() {
+            return context.getBindManager().getParameterMetaData();
+        }
+
+        @Override
+        public StatementContext getContext() {
+            return context;
+        }
+
+        @Override
+        public TableRef getTargetRef() {
+            return tableRef;
+        }
+
+        @Override
+        public QueryPlan getQueryPlan() {
+            return null;
+        }
+
+        @Override
+        public Set<TableRef> getSourceRefs() {
+            return Collections.emptySet();
+        }
+
+        @Override
+        public Operation getOperation() {
+          return operation;
+        }
+
+        @Override
+        public MutationState execute() throws SQLException {
+            ImmutableBytesWritable ptr = context.getTempPtr();
+            final SequenceManager sequenceManager = context.getSequenceManager();
+            // Next evaluate all the expressions
+            int nodeIndex = nodeIndexOffset;
+            PTable table = tableRef.getTable();
+            Tuple tuple = sequenceManager.getSequenceCount() == 0 ? null :
+                sequenceManager.newSequenceTuple(null);
+            for (Expression constantExpression : constantExpressions) {
+                PColumn column = allColumns.get(columnIndexes[nodeIndex]);
+                constantExpression.evaluate(tuple, ptr);
+                Object value = null;
+                if (constantExpression.getDataType() != null) {
+                    value = constantExpression.getDataType().toObject(ptr, constantExpression.getSortOrder(),
+                            constantExpression.getMaxLength(), constantExpression.getScale());
+                    if (!constantExpression.getDataType().isCoercibleTo(column.getDataType(), value)) {
+                        throw TypeMismatchException.newException(
+                            constantExpression.getDataType(), column.getDataType(), "expression: "
+                                    + constantExpression.toString() + " in column " + column);
+                    }
+                    if (!column.getDataType().isSizeCompatible(ptr, value, constantExpression.getDataType(),
+                            constantExpression.getSortOrder(), constantExpression.getMaxLength(),
+                            constantExpression.getScale(), column.getMaxLength(), column.getScale())) {
+                        throw new SQLExceptionInfo.Builder(
+                            SQLExceptionCode.DATA_EXCEEDS_MAX_CAPACITY).setColumnName(column.getName().getString())
+                            .setMessage("value=" + constantExpression.toString()).build().buildException();
+                    }
+                }
+                column.getDataType().coerceBytes(ptr, value, constantExpression.getDataType(),
+                        constantExpression.getMaxLength(), constantExpression.getScale(), constantExpression.getSortOrder(),
+                        column.getMaxLength(), column.getScale(),column.getSortOrder(),
+                        table.rowKeyOrderOptimizable());
+                if (overlapViewColumns.contains(column) && Bytes.compareTo(ptr.get(), ptr.getOffset(), ptr.getLength(), column.getViewConstant(), 0, column.getViewConstant().length-1) != 0) {
+                    throw new SQLExceptionInfo.Builder(
+                            SQLExceptionCode.CANNOT_UPDATE_VIEW_COLUMN)
+                            .setColumnName(column.getName().getString())
+                            .setMessage("value=" + constantExpression.toString()).build().buildException();
+                }
+                values[nodeIndex] = ByteUtil.copyKeyBytesIfNecessary(ptr);
+                nodeIndex++;
+            }
+            // Add columns based on view
+            for (PColumn column : addViewColumns) {
+                if (IndexUtil.getViewConstantValue(column, ptr)) {
+                    values[nodeIndex++] = ByteUtil.copyKeyBytesIfNecessary(ptr);
+                } else {
+                    throw new IllegalStateException();
+                }
+            }
+            Map<ImmutableBytesPtr, RowMutationState> mutation = Maps.newHashMapWithExpectedSize(1);
+            IndexMaintainer indexMaintainer = null;
+            byte[][] viewConstants = null;
+            if (table.getIndexType() == IndexType.LOCAL) {
+                PTable parentTable =
+                        statement
+                                .getConnection()
+                                .getMetaDataCache()
+                                .getTableRef(
+                                    new PTableKey(statement.getConnection().getTenantId(),
+                                            table.getParentName().getString())).getTable();
+                indexMaintainer = table.getIndexMaintainer(parentTable, connection);
+                viewConstants = IndexUtil.getViewConstants(parentTable);
+            }
+            setValues(values, pkSlotIndexes, columnIndexes, table, mutation, statement, useServerTimestamp, indexMaintainer, viewConstants, onDupKeyBytes, 0);
+            return new MutationState(tableRef, mutation, 0, maxSize, maxSizeBytes, connection);
+        }
+
+        @Override
+        public ExplainPlan getExplainPlan() throws SQLException {
+            List<String> planSteps = Lists.newArrayListWithExpectedSize(2);
+            if (context.getSequenceManager().getSequenceCount() > 0) {
+                planSteps.add("CLIENT RESERVE " + context.getSequenceManager().getSequenceCount() + " SEQUENCES");
+            }
+            planSteps.add("PUT SINGLE ROW");
+            return new ExplainPlan(planSteps);
+        }
+
+        @Override
+        public Long getEstimatedRowsToScan() throws SQLException {
+            return 0l;
+        }
+
+        @Override
+        public Long getEstimatedBytesToScan() throws SQLException {
+            return 0l;
+        }
+
+        @Override
+        public Long getEstimateInfoTimestamp() throws SQLException {
+            return 0l;
+        }
+    }
+
+    private class ClientUpsertSelectMutationPlan implements MutationPlan {
+        private final QueryPlan queryPlan;
+        private final TableRef tableRef;
+        private final QueryPlan originalQueryPlan;
+        private final UpsertingParallelIteratorFactory parallelIteratorFactory;
+        private final RowProjector projector;
+        private final int[] columnIndexes;
+        private final int[] pkSlotIndexes;
+        private final boolean useServerTimestamp;
+        private final int maxSize;
+        private final int maxSizeBytes;
+
+        public ClientUpsertSelectMutationPlan(QueryPlan queryPlan, TableRef tableRef, QueryPlan originalQueryPlan, UpsertingParallelIteratorFactory parallelIteratorFactory, RowProjector projector, int[] columnIndexes, int[] pkSlotIndexes, boolean useServerTimestamp, int maxSize, int maxSizeBytes) {
+            this.queryPlan = queryPlan;
+            this.tableRef = tableRef;
+            this.originalQueryPlan = originalQueryPlan;
+            this.parallelIteratorFactory = parallelIteratorFactory;
+            this.projector = projector;
+            this.columnIndexes = columnIndexes;
+            this.pkSlotIndexes = pkSlotIndexes;
+            this.useServerTimestamp = useServerTimestamp;
+            this.maxSize = maxSize;
+            this.maxSizeBytes = maxSizeBytes;
+        }
+
+        @Override
+        public ParameterMetaData getParameterMetaData() {
+            return queryPlan.getContext().getBindManager().getParameterMetaData();
+        }
+
+        @Override
+        public StatementContext getContext() {
+            return queryPlan.getContext();
+        }
+
+        @Override
+        public TableRef getTargetRef() {
+            return tableRef;
+        }
+
+        @Override
+        public QueryPlan getQueryPlan() {
+            return queryPlan;
+        }
+
+        @Override
+        public Set<TableRef> getSourceRefs() {
+            return originalQueryPlan.getSourceRefs();
+        }
+
+        @Override
+        public Operation getOperation() {
+          return operation;
+        }
+
+        @Override
+        public MutationState execute() throws SQLException {
+            ResultIterator iterator = queryPlan.iterator();
+            if (parallelIteratorFactory == null) {
+                return upsertSelect(new StatementContext(statement), tableRef, projector, iterator, columnIndexes, pkSlotIndexes, useServerTimestamp, false);
+            }
+            try {
+                parallelIteratorFactory.setRowProjector(projector);
+                parallelIteratorFactory.setColumnIndexes(columnIndexes);
+                parallelIteratorFactory.setPkSlotIndexes(pkSlotIndexes);
+                Tuple tuple;
+                long totalRowCount = 0;
+                StatementContext context = queryPlan.getContext();
+                while ((tuple=iterator.next()) != null) {// Runs query
+                    Cell kv = tuple.getValue(0);
+                    totalRowCount += PLong.INSTANCE.getCodec().decodeLong(kv.getValueArray(), kv.getValueOffset(), SortOrder.getDefault());
+                }
+                // Return total number of rows that have been updated. In the case of auto commit being off
+                // the mutations will all be in the mutation state of the current connection.
+                MutationState mutationState = new MutationState(maxSize, maxSizeBytes, statement.getConnection(), totalRowCount);
+                /*
+                 *  All the metrics collected for measuring the reads done by the parallel mutating iterators
+                 *  is included in the ReadMetricHolder of the statement context. Include these metrics in the
+                 *  returned mutation state so they can be published on commit.
+                 */
+                mutationState.setReadMetricQueue(context.getReadMetricsQueue());
+                return mutationState;
+            } finally {
+                iterator.close();
+            }
+        }
+
+        @Override
+        public ExplainPlan getExplainPlan() throws SQLException {
+            List<String> queryPlanSteps =  queryPlan.getExplainPlan().getPlanSteps();
+            List<String> planSteps = Lists.newArrayListWithExpectedSize(queryPlanSteps.size()+1);
+            planSteps.add("UPSERT SELECT");
+            planSteps.addAll(queryPlanSteps);
+            return new ExplainPlan(planSteps);
+        }
+
+        @Override
+        public Long getEstimatedRowsToScan() throws SQLException {
+            return queryPlan.getEstimatedRowsToScan();
+        }
+
+        @Override
+        public Long getEstimatedBytesToScan() throws SQLException {
+            return queryPlan.getEstimatedBytesToScan();
+        }
+
+        @Override
+        public Long getEstimateInfoTimestamp() throws SQLException {
+            return queryPlan.getEstimateInfoTimestamp();
+        }
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a23439cf/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
index d35cce1..174e643 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
@@ -1313,11 +1313,12 @@ public class PhoenixStatement implements Statement, SQLCloseable {
                 public ExplainPlan getExplainPlan() throws SQLException {
                     return new ExplainPlan(Collections.singletonList("EXECUTE UPGRADE"));
                 }
-                
+
                 @Override
-                public StatementContext getContext() {
-                    return new StatementContext(stmt);
-                }
+                public QueryPlan getQueryPlan() { return null; }
+
+                @Override
+                public StatementContext getContext() { return new StatementContext(stmt); }
                 
                 @Override
                 public TableRef getTargetRef() {


[02/16] phoenix git commit: PHOENIX-3837 Feature enabling to set property on an index with Alter statement

Posted by ja...@apache.org.
PHOENIX-3837 Feature enabling to set property on an index with Alter statement

Signed-off-by: aertoria <ca...@gmail.com>


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

Branch: refs/heads/4.x-HBase-1.2
Commit: ef174bb22ccce806dcdf5435d17319cbfa86c0d5
Parents: 5b49396
Author: aertoria <ca...@gmail.com>
Authored: Mon Nov 27 03:13:53 2017 +0000
Committer: James Taylor <jt...@salesforce.com>
Committed: Sat Dec 16 16:42:54 2017 -0800

----------------------------------------------------------------------
 .../apache/phoenix/end2end/AlterTableIT.java    |   2 +-
 .../phoenix/end2end/index/IndexMetadataIT.java  |  55 ++
 phoenix-core/src/main/antlr3/PhoenixSQL.g       |   5 +-
 .../apache/phoenix/jdbc/PhoenixStatement.java   |  10 +-
 .../phoenix/parse/AddColumnStatement.java       |   2 +-
 .../phoenix/parse/AlterIndexStatement.java      |  14 +
 .../apache/phoenix/parse/ParseNodeFactory.java  |   6 +-
 .../phoenix/query/ConnectionQueryServices.java  |   2 +
 .../query/ConnectionQueryServicesImpl.java      |  20 +
 .../query/ConnectionlessQueryServicesImpl.java  |   7 +
 .../query/DelegateConnectionQueryServices.java  |   8 +-
 .../apache/phoenix/schema/MetaDataClient.java   | 566 +++++++++++++------
 12 files changed, 520 insertions(+), 177 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/ef174bb2/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java
index 5265b09..17f08c4 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java
@@ -1080,7 +1080,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
 			} catch (SQLException e) {
 				assertEquals(SQLExceptionCode.CANNOT_CREATE_TXN_TABLE_IF_TXNS_DISABLED.getErrorCode(), e.getErrorCode());
 			}
-			// altering a table to be transactional  should fail if transactions are disabled
+			// altering a table to be transactional should fail if transactions are disabled
 			conn.createStatement().execute("CREATE TABLE " + dataTableFullName + "(k INTEGER PRIMARY KEY, v VARCHAR)");
 			try {
 				conn.createStatement().execute("ALTER TABLE " + dataTableFullName + " SET TRANSACTIONAL=true");

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ef174bb2/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexMetadataIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexMetadataIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexMetadataIT.java
index 0ce36dd..986c317 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexMetadataIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexMetadataIT.java
@@ -674,4 +674,59 @@ public class IndexMetadataIT extends ParallelStatsDisabledIT {
             conn.close();
         }
     }
+
+
+
+    @Test
+    public void testIndexAlterPhoenixProperty() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        String testTable = generateUniqueName();
+
+
+        String ddl = "create table " + testTable  + " (k varchar primary key, v1 varchar)";
+        Statement stmt = conn.createStatement();
+        stmt.execute(ddl);
+        String indexName = "IDX_" + generateUniqueName();
+
+        ddl = "CREATE INDEX " + indexName + " ON " + testTable  + " (v1) ";
+        stmt.execute(ddl);
+        conn.createStatement().execute("ALTER INDEX "+indexName+" ON " + testTable +" ACTIVE SET GUIDE_POSTS_WIDTH = 10");
+
+        ResultSet rs = conn.createStatement().executeQuery(
+                "select GUIDE_POSTS_WIDTH from SYSTEM.\"CATALOG\" where TABLE_NAME='" + indexName + "'");assertTrue(rs.next());
+        assertEquals(10,rs.getInt(1));
+
+        conn.createStatement().execute("ALTER INDEX "+indexName+" ON " + testTable +" ACTIVE SET GUIDE_POSTS_WIDTH = 20");
+        rs = conn.createStatement().executeQuery(
+                "select GUIDE_POSTS_WIDTH from SYSTEM.\"CATALOG\" where TABLE_NAME='" + indexName + "'");assertTrue(rs.next());
+        assertEquals(20,rs.getInt(1));
+    }
+
+
+    @Test
+    public void testIndexAlterHBaseProperty() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        String testTable = generateUniqueName();
+
+        String ddl = "create table " + testTable  + " (k varchar primary key, v1 varchar)";
+        Statement stmt = conn.createStatement();
+        stmt.execute(ddl);
+        String indexName = "IDX_" + generateUniqueName();
+
+        ddl = "CREATE INDEX " + indexName + " ON " + testTable  + " (v1) ";
+        stmt.execute(ddl);
+
+        conn.createStatement().execute("ALTER INDEX "+indexName+" ON " + testTable +" ACTIVE SET DISABLE_WAL=false");
+        asssertIsWALDisabled(conn,indexName,false);
+        conn.createStatement().execute("ALTER INDEX "+indexName+" ON " + testTable +" ACTIVE SET DISABLE_WAL=true");
+        asssertIsWALDisabled(conn,indexName,true);
+    }
+
+    private static void asssertIsWALDisabled(Connection conn, String fullTableName, boolean expectedValue) throws SQLException {
+        PhoenixConnection pconn = conn.unwrap(PhoenixConnection.class);
+        assertEquals(expectedValue, pconn.getTable(new PTableKey(pconn.getTenantId(), fullTableName)).isWALDisabled());
+    }
+
 }

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

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ef174bb2/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
index 26b4415..c699088 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
@@ -1216,8 +1216,8 @@ public class PhoenixStatement implements Statement, SQLCloseable {
 
     private static class ExecutableAlterIndexStatement extends AlterIndexStatement implements CompilableStatement {
 
-        public ExecutableAlterIndexStatement(NamedTableNode indexTableNode, String dataTableName, boolean ifExists, PIndexState state, boolean async) {
-            super(indexTableNode, dataTableName, ifExists, state, async);
+        public ExecutableAlterIndexStatement(NamedTableNode indexTableNode, String dataTableName, boolean ifExists, PIndexState state, boolean async, ListMultimap<String,Pair<String,Object>> props) {
+            super(indexTableNode, dataTableName, ifExists, state, async, props);
         }
 
         @SuppressWarnings("unchecked")
@@ -1563,10 +1563,10 @@ public class PhoenixStatement implements Statement, SQLCloseable {
         public DropIndexStatement dropIndex(NamedNode indexName, TableName tableName, boolean ifExists) {
             return new ExecutableDropIndexStatement(indexName, tableName, ifExists);
         }
-        
+
         @Override
-        public AlterIndexStatement alterIndex(NamedTableNode indexTableNode, String dataTableName, boolean ifExists, PIndexState state, boolean async) {
-            return new ExecutableAlterIndexStatement(indexTableNode, dataTableName, ifExists, state, async);
+        public AlterIndexStatement alterIndex(NamedTableNode indexTableNode, String dataTableName, boolean ifExists, PIndexState state, boolean async, ListMultimap<String,Pair<String,Object>> props) {
+            return new ExecutableAlterIndexStatement(indexTableNode, dataTableName, ifExists, state, async, props);
         }
 
         @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ef174bb2/phoenix-core/src/main/java/org/apache/phoenix/parse/AddColumnStatement.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/AddColumnStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/AddColumnStatement.java
index 1890d31..678e560 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/AddColumnStatement.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/AddColumnStatement.java
@@ -48,4 +48,4 @@ public class AddColumnStatement extends AlterTableStatement {
     public ListMultimap<String,Pair<String,Object>> getProps() {
         return props;
     }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ef174bb2/phoenix-core/src/main/java/org/apache/phoenix/parse/AlterIndexStatement.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/AlterIndexStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/AlterIndexStatement.java
index 11328c2..de04505 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/AlterIndexStatement.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/AlterIndexStatement.java
@@ -17,20 +17,31 @@
  */
 package org.apache.phoenix.parse;
 
+import com.google.common.collect.ImmutableListMultimap;
+import com.google.common.collect.ListMultimap;
+import org.apache.hadoop.hbase.util.Pair;
 import org.apache.phoenix.schema.PIndexState;
+import org.apache.phoenix.schema.PTableType;
 
 public class AlterIndexStatement extends SingleTableStatement {
     private final String dataTableName;
     private final boolean ifExists;
     private final PIndexState indexState;
     private boolean async;
+    private ListMultimap<String,Pair<String,Object>> props;
+    private static final PTableType tableType=PTableType.INDEX;
 
     public AlterIndexStatement(NamedTableNode indexTableNode, String dataTableName, boolean ifExists, PIndexState indexState, boolean async) {
+        this(indexTableNode,dataTableName,ifExists,indexState,async,null);
+    }
+
+    public AlterIndexStatement(NamedTableNode indexTableNode, String dataTableName, boolean ifExists, PIndexState indexState, boolean async, ListMultimap<String,Pair<String,Object>> props) {
         super(indexTableNode,0);
         this.dataTableName = dataTableName;
         this.ifExists = ifExists;
         this.indexState = indexState;
         this.async = async;
+        this.props= props==null ? ImmutableListMultimap.<String,Pair<String,Object>>of() : props;
     }
 
     public String getTableName() {
@@ -54,4 +65,7 @@ public class AlterIndexStatement extends SingleTableStatement {
         return async;
     }
 
+    public ListMultimap<String,Pair<String,Object>> getProps() { return props; }
+
+    public PTableType getTableType(){ return tableType; }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ef174bb2/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java
index 32c3d8d..9be59f3 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java
@@ -373,10 +373,10 @@ public class ParseNodeFactory {
         return new DropIndexStatement(indexName, tableName, ifExists);
     }
 
-    public AlterIndexStatement alterIndex(NamedTableNode indexTableNode, String dataTableName, boolean ifExists, PIndexState state, boolean async) {
-        return new AlterIndexStatement(indexTableNode, dataTableName, ifExists, state, async);
+    public AlterIndexStatement alterIndex(NamedTableNode indexTableNode, String dataTableName, boolean ifExists, PIndexState state, boolean async, ListMultimap<String,Pair<String,Object>> props) {
+        return new AlterIndexStatement(indexTableNode, dataTableName, ifExists, state, async, props);
     }
-    
+
     public AlterIndexStatement alterIndex(NamedTableNode indexTableNode, String dataTableName, boolean ifExists, PIndexState state) {
         return new AlterIndexStatement(indexTableNode, dataTableName, ifExists, state, false);
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ef174bb2/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java
index 45ab5fa..90f8089 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java
@@ -86,6 +86,8 @@ public interface ConnectionQueryServices extends QueryServices, MetaDataMutated
     public MetaDataMutationResult addColumn(List<Mutation> tableMetaData, PTable table, Map<String, List<Pair<String,Object>>> properties, Set<String> colFamiliesForPColumnsToBeAdded, List<PColumn> columns) throws SQLException;
     public MetaDataMutationResult dropColumn(List<Mutation> tableMetadata, PTableType tableType) throws SQLException;
     public MetaDataMutationResult updateIndexState(List<Mutation> tableMetadata, String parentTableName) throws SQLException;
+    public MetaDataMutationResult updateIndexState(List<Mutation> tableMetadata, String parentTableName,  Map<String, List<Pair<String,Object>>> stmtProperties,  PTable table) throws SQLException;
+
     public MutationState updateData(MutationPlan plan) throws SQLException;
 
     public void init(String url, Properties props) throws SQLException;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ef174bb2/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
index 08aadfb..072bf28 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
@@ -3601,6 +3601,26 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     }
 
     @Override
+    public MetaDataMutationResult updateIndexState(final List<Mutation> tableMetaData, String parentTableName, Map<String, List<Pair<String,Object>>> stmtProperties,  PTable table) throws SQLException {
+        if(stmtProperties==null) return updateIndexState(tableMetaData,parentTableName);
+
+        Map<String, Object> tableProps = new HashMap<String, Object>();
+        Pair<HTableDescriptor,HTableDescriptor> tableDescriptorPair = separateAndValidateProperties(table, stmtProperties, new HashSet<String>(), tableProps);
+        HTableDescriptor tableDescriptor = tableDescriptorPair.getSecond();
+        HTableDescriptor origTableDescriptor = tableDescriptorPair.getFirst();
+        Set<HTableDescriptor> tableDescriptors = Collections.emptySet();
+        Set<HTableDescriptor> origTableDescriptors = Collections.emptySet();
+        if (tableDescriptor != null) {
+            tableDescriptors = Sets.newHashSetWithExpectedSize(3 + table.getIndexes().size());
+            origTableDescriptors = Sets.newHashSetWithExpectedSize(3 + table.getIndexes().size());
+            tableDescriptors.add(tableDescriptor);
+            origTableDescriptors.add(origTableDescriptor);
+        }
+        sendHBaseMetaData(tableDescriptors, true);
+        return updateIndexState(tableMetaData,parentTableName);
+    }
+
+    @Override
     public long createSequence(String tenantId, String schemaName, String sequenceName,
             long startWith, long incrementBy, long cacheSize, long minValue, long maxValue,
             boolean cycle, long timestamp) throws SQLException {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ef174bb2/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
index f15e0b1..3154f86 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
@@ -383,6 +383,13 @@ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices imple
     }
 
     @Override
+    public MetaDataMutationResult updateIndexState(List<Mutation> tableMetadata,
+            String parentTableName, Map<String, List<Pair<String, Object>>> stmtProperties,
+            PTable table) throws SQLException {
+        return updateIndexState(tableMetadata,parentTableName);
+    }
+
+    @Override
     public HTableDescriptor getTableDescriptor(byte[] tableName) throws SQLException {
         return null;
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ef174bb2/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java b/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java
index 6c464eb..05d1af6 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java
@@ -135,7 +135,13 @@ public class DelegateConnectionQueryServices extends DelegateQueryServices imple
     public MetaDataMutationResult updateIndexState(List<Mutation> tableMetadata, String parentTableName) throws SQLException {
         return getDelegate().updateIndexState(tableMetadata, parentTableName);
     }
-    
+
+    @Override public MetaDataMutationResult updateIndexState(List<Mutation> tableMetadata,
+            String parentTableName, Map<String, List<Pair<String, Object>>> stmtProperties,
+            PTable table) throws SQLException {
+        return getDelegate().updateIndexState(tableMetadata, parentTableName, stmtProperties,table);
+    }
+
     @Override
     public void init(String url, Properties props) throws SQLException {
         getDelegate().init(url, props);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ef174bb2/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
index 1f76e90..fc2e288 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
@@ -3007,6 +3007,21 @@ public class MetaDataClient {
         return mutationCode;
     }
 
+    private long incrementTableSeqNum(PTable table, PTableType expectedType, int columnCountDelta, MetaPropertiesEvaluated metaPropertiesEvaluated)
+            throws SQLException {
+        return incrementTableSeqNum(table, expectedType, columnCountDelta,
+                metaPropertiesEvaluated.getIsTransactional(),
+                metaPropertiesEvaluated.getUpdateCacheFrequency(),
+                metaPropertiesEvaluated.getIsImmutableRows(),
+                metaPropertiesEvaluated.getDisableWAL(),
+                metaPropertiesEvaluated.getMultiTenant(),
+                metaPropertiesEvaluated.getStoreNulls(),
+                metaPropertiesEvaluated.getGuidePostWidth(),
+                metaPropertiesEvaluated.getAppendOnlySchema(),
+                metaPropertiesEvaluated.getImmutableStorageScheme(),
+                metaPropertiesEvaluated.getUseStatsForParallelization());
+    }
+
     private  long incrementTableSeqNum(PTable table, PTableType expectedType, int columnCountDelta, Boolean isTransactional, Long updateCacheFrequency) throws SQLException {
         return incrementTableSeqNum(table, expectedType, columnCountDelta, isTransactional, updateCacheFrequency, null, null, null, null, -1L, null, null, null);
     }
@@ -3138,18 +3153,7 @@ public class MetaDataClient {
             PName tenantId = connection.getTenantId();
             String schemaName = table.getSchemaName().getString();
             String tableName = table.getTableName().getString();
-            Boolean isImmutableRowsProp = null;
-            Boolean multiTenantProp = null;
-            Boolean disableWALProp = null;
-            Boolean storeNullsProp = null;
-            Boolean isTransactionalProp = null;
-            Long updateCacheFrequencyProp = null;
-            Boolean appendOnlySchemaProp = null;
-            Long guidePostWidth = -1L;
-            ImmutableStorageScheme immutableStorageSchemeProp = null;
-            Boolean useStatsForParallelizationProp = null;
-
-            Map<String, List<Pair<String, Object>>> properties = new HashMap<>(stmtProperties.size());
+
             List<ColumnDef> columnDefs = null;
             if (table.isAppendOnlySchema()) {
                 // only make the rpc if we are adding new columns
@@ -3185,48 +3189,14 @@ public class MetaDataClient {
             else {
                 columnDefs = origColumnDefs == null ? Collections.<ColumnDef>emptyList() : origColumnDefs;
             }
-            for (String family : stmtProperties.keySet()) {
-                List<Pair<String, Object>> origPropsList = stmtProperties.get(family);
-                List<Pair<String, Object>> propsList = Lists.newArrayListWithExpectedSize(origPropsList.size());
-                for (Pair<String, Object> prop : origPropsList) {
-                    String propName = prop.getFirst();
-                    if (TableProperty.isPhoenixTableProperty(propName)) {
-                        TableProperty tableProp = TableProperty.valueOf(propName);
-                        tableProp.validate(true, !family.equals(QueryConstants.ALL_FAMILY_PROPERTIES_KEY), table.getType());
-                        Object value = tableProp.getValue(prop.getSecond());
-                        if (propName.equals(PTable.IS_IMMUTABLE_ROWS_PROP_NAME)) {
-                            isImmutableRowsProp = (Boolean)value;
-                        } else if (propName.equals(PhoenixDatabaseMetaData.MULTI_TENANT)) {
-                            multiTenantProp = (Boolean)value;
-                        } else if (propName.equals(DISABLE_WAL)) {
-                            disableWALProp = (Boolean)value;
-                        } else if (propName.equals(STORE_NULLS)) {
-                            storeNullsProp = (Boolean)value;
-                        } else if (propName.equals(TRANSACTIONAL)) {
-                            isTransactionalProp = (Boolean)value;
-                        } else if (propName.equals(UPDATE_CACHE_FREQUENCY)) {
-                            updateCacheFrequencyProp = (Long)value;
-                        } else if (propName.equals(GUIDE_POSTS_WIDTH)) {
-                            guidePostWidth = (Long)value;
-                        } else if (propName.equals(APPEND_ONLY_SCHEMA)) {
-                            appendOnlySchemaProp = (Boolean) value;
-                        } else if (propName.equalsIgnoreCase(IMMUTABLE_STORAGE_SCHEME)) {
-                            immutableStorageSchemeProp = (ImmutableStorageScheme)value;
-                        } else if (propName.equalsIgnoreCase(USE_STATS_FOR_PARALLELIZATION)) {
-                            useStatsForParallelizationProp = (Boolean)value;
-                        }
-                    }
-                    // if removeTableProps is true only add the property if it is not a HTable or Phoenix Table property
-                    if (!removeTableProps || (!TableProperty.isPhoenixTableProperty(propName) && !MetaDataUtil.isHTableProperty(propName))) {
-                        propsList.add(prop);
-                    }
-                }
-                properties.put(family, propsList);
-            }
+
             boolean retried = false;
             boolean changingPhoenixTableProperty = false;
-            boolean nonTxToTx = false;
+            MetaProperties metaProperties = new MetaProperties();
             while (true) {
+                Map<String, List<Pair<String, Object>>> properties=new HashMap<>(stmtProperties.size());;
+                metaProperties = loadStmtProperties(stmtProperties,properties,table,removeTableProps);
+
                 ColumnResolver resolver = FromCompiler.getResolver(namedTableNode, connection);
                 table = resolver.getTables().get(0).getTable();
                 int nIndexes = table.getIndexes().size();
@@ -3253,108 +3223,11 @@ public class MetaDataClient {
                     .setColumnName(lastPK.getName().getString()).build().buildException();
                 }
 
-                Boolean isImmutableRows = null;
-                if (isImmutableRowsProp != null) {
-                    if (isImmutableRowsProp.booleanValue() != table.isImmutableRows()) {
-                    	if (table.getImmutableStorageScheme() != ImmutableStorageScheme.ONE_CELL_PER_COLUMN) {
-                    		throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_ALTER_IMMUTABLE_ROWS_PROPERTY)
-                    		.setSchemaName(schemaName).setTableName(tableName).build().buildException();
-                    	}
-                        isImmutableRows = isImmutableRowsProp;
-                        changingPhoenixTableProperty = true;
-                    }
-                }
-                Boolean multiTenant = null;
-                if (multiTenantProp != null) {
-                    if (multiTenantProp.booleanValue() != table.isMultiTenant()) {
-                        multiTenant = multiTenantProp;
-                        changingPhoenixTableProperty = true;
-                    }
-                }
-                Boolean disableWAL = null;
-                if (disableWALProp != null) {
-                    if (disableWALProp.booleanValue() != table.isWALDisabled()) {
-                        disableWAL = disableWALProp;
-                        changingPhoenixTableProperty = true;
-                    }
-                }
-                Long updateCacheFrequency = null;
-                if (updateCacheFrequencyProp != null) {
-                    if (updateCacheFrequencyProp.longValue() != table.getUpdateCacheFrequency()) {
-                        updateCacheFrequency = updateCacheFrequencyProp;
-                        changingPhoenixTableProperty = true;
-                    }
-                }
-                Boolean appendOnlySchema = null;
-                if (appendOnlySchemaProp !=null) {
-                    if (appendOnlySchemaProp != table.isAppendOnlySchema()) {
-                        appendOnlySchema  = appendOnlySchemaProp;
-                        changingPhoenixTableProperty = true;
-                    }
-                }
-                ImmutableStorageScheme immutableStorageScheme = null;
-                if (immutableStorageSchemeProp!=null) {
-                    if (table.getImmutableStorageScheme() == ONE_CELL_PER_COLUMN || 
-                            immutableStorageSchemeProp == ONE_CELL_PER_COLUMN) {
-                        throw new SQLExceptionInfo.Builder(SQLExceptionCode.INVALID_IMMUTABLE_STORAGE_SCHEME_CHANGE)
-                        .setSchemaName(schemaName).setTableName(tableName).build().buildException();
-                    }
-                    else if (immutableStorageSchemeProp != table.getImmutableStorageScheme()) {
-                        immutableStorageScheme = immutableStorageSchemeProp;
-                        changingPhoenixTableProperty = true;
-                    }
-                }
-            
-                if (guidePostWidth == null || guidePostWidth >= 0) {
-                    changingPhoenixTableProperty = true;
-                }
-                Boolean storeNulls = null;
-                if (storeNullsProp != null) {
-                    if (storeNullsProp.booleanValue() != table.getStoreNulls()) {
-                        storeNulls = storeNullsProp;
-                        changingPhoenixTableProperty = true;
-                    }
-                }
-                Boolean useStatsForParallelization = null;
-                if (useStatsForParallelizationProp != null
-                        && (table.useStatsForParallelization() == null
-                                || (useStatsForParallelizationProp.booleanValue() != table
-                                        .useStatsForParallelization()))) {
-                    useStatsForParallelization = useStatsForParallelizationProp;
-                    changingPhoenixTableProperty = true;
-                }
-                Boolean isTransactional = null;
-                if (isTransactionalProp != null) {
-                    if (isTransactionalProp.booleanValue() != table.isTransactional()) {
-                        isTransactional = isTransactionalProp;
-                        // We can only go one way: from non transactional to transactional
-                        // Going the other way would require rewriting the cell timestamps
-                        // and doing a major compaction to get rid of any Tephra specific
-                        // delete markers.
-                        if (!isTransactional) {
-                            throw new SQLExceptionInfo.Builder(SQLExceptionCode.TX_MAY_NOT_SWITCH_TO_NON_TX)
-                            .setSchemaName(schemaName).setTableName(tableName).build().buildException();
-                        }
-                        // cannot create a transactional table if transactions are disabled
-                        boolean transactionsEnabled = connection.getQueryServices().getProps().getBoolean(
-                                QueryServices.TRANSACTIONS_ENABLED,
-                                QueryServicesOptions.DEFAULT_TRANSACTIONS_ENABLED);
-                        if (!transactionsEnabled) {
-                            throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_ALTER_TO_BE_TXN_IF_TXNS_DISABLED)
-                            .setSchemaName(schemaName).setTableName(tableName).build().buildException();
-                        }
-                        // cannot make a table transactional if it has a row timestamp column
-                        if (SchemaUtil.hasRowTimestampColumn(table)) {
-                            throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_ALTER_TO_BE_TXN_WITH_ROW_TIMESTAMP)
-                            .setSchemaName(schemaName).setTableName(tableName)
-                            .build().buildException();
-                        }
-                        changingPhoenixTableProperty = true;
-                        nonTxToTx = true;
-                    }
-                }
-                Long timeStamp = TransactionUtil.getTableTimestamp(connection, table.isTransactional() || nonTxToTx);
 
+                MetaPropertiesEvaluated metaPropertiesEvaluated = new MetaPropertiesEvaluated();
+                changingPhoenixTableProperty = evaluateStmtProperties(metaProperties,metaPropertiesEvaluated,table,schemaName,tableName);
+
+                Long timeStamp = TransactionUtil.getTableTimestamp(connection, table.isTransactional() || metaProperties.getNonTxToTx());
                 int numPkColumnsAdded = 0;
                 List<PColumn> columns = Lists.newArrayListWithExpectedSize(numCols);
                 Set<String> colFamiliesForPColumnsToBeAdded = new LinkedHashSet<>();
@@ -3471,7 +3344,7 @@ public class MetaDataClient {
                     // Check that HBase configured properly for mutable secondary indexing
                     // if we're changing from an immutable table to a mutable table and we
                     // have existing indexes.
-                    if (Boolean.FALSE.equals(isImmutableRows) && !table.getIndexes().isEmpty()) {
+                    if (Boolean.FALSE.equals(metaPropertiesEvaluated.getIsImmutableRows()) && !table.getIndexes().isEmpty()) {
                         int hbaseVersion = connection.getQueryServices().getLowestClusterHBaseVersion();
                         if (hbaseVersion < PhoenixDatabaseMetaData.MUTABLE_SI_VERSION_THRESHOLD) {
                             throw new SQLExceptionInfo.Builder(SQLExceptionCode.NO_MUTABLE_INDEXES)
@@ -3482,22 +3355,22 @@ public class MetaDataClient {
                             .setSchemaName(schemaName).setTableName(tableName).build().buildException();
                         }
                     }
-                    if (Boolean.TRUE.equals(multiTenant)) {
-                        throwIfInsufficientColumns(schemaName, tableName, table.getPKColumns(), table.getBucketNum()!=null, multiTenant);
+                    if (Boolean.TRUE.equals(metaPropertiesEvaluated.getMultiTenant())) {
+                        throwIfInsufficientColumns(schemaName, tableName, table.getPKColumns(), table.getBucketNum()!=null, metaPropertiesEvaluated.getMultiTenant());
                     }
                 }
 
-                if (!table.getIndexes().isEmpty() && (numPkColumnsAdded>0 || nonTxToTx)) {
+                if (!table.getIndexes().isEmpty() && (numPkColumnsAdded>0 || metaProperties.getNonTxToTx())) {
                     for (PTable index : table.getIndexes()) {
-                        incrementTableSeqNum(index, index.getType(), numPkColumnsAdded, nonTxToTx ? Boolean.TRUE : null, updateCacheFrequency);
+                        incrementTableSeqNum(index, index.getType(), numPkColumnsAdded, metaProperties.getNonTxToTx() ? Boolean.TRUE : null, metaPropertiesEvaluated.getUpdateCacheFrequency());
                     }
                     tableMetaData.addAll(connection.getMutationState().toMutations(timeStamp).next().getSecond());
                     connection.rollback();
                 }
                 
                 if (changingPhoenixTableProperty || columnDefs.size() > 0) {
-                    incrementTableSeqNum(table, tableType, columnDefs.size(), isTransactional, updateCacheFrequency, isImmutableRows,
-                            disableWAL, multiTenant, storeNulls, guidePostWidth, appendOnlySchema, immutableStorageScheme, useStatsForParallelization);
+                    incrementTableSeqNum(table, tableType, columnDefs.size(), metaPropertiesEvaluated);
+
                     tableMetaData.addAll(connection.getMutationState().toMutations(timeStamp).next().getSecond());
                     connection.rollback();
                 }
@@ -3567,10 +3440,10 @@ public class MetaDataClient {
                     // We could update the cache manually then too, it'd just be a pain.
                     String fullTableName = SchemaUtil.getTableName(schemaName, tableName);
                     long resolvedTimeStamp = TransactionUtil.getResolvedTime(connection, result);
-                    if (table.getIndexes().isEmpty() || (numPkColumnsAdded==0 && !nonTxToTx)) {
+                    if (table.getIndexes().isEmpty() || (numPkColumnsAdded==0 && ! metaProperties.getNonTxToTx())) {
                         connection.addTable(result.getTable(), resolvedTimeStamp);
                         table = result.getTable();
-                    } else if (updateCacheFrequency != null) {
+                    } else if (metaPropertiesEvaluated.getUpdateCacheFrequency() != null) {
                         // Force removal from cache as the update cache frequency has changed
                         // Note that clients outside this JVM won't be affected.
                         connection.removeTable(tenantId, fullTableName, null, resolvedTimeStamp);
@@ -3578,7 +3451,7 @@ public class MetaDataClient {
                     // Delete rows in view index if we haven't dropped it already
                     // We only need to do this if the multiTenant transitioned to false
                     if (table.getType() == PTableType.TABLE
-                            && Boolean.FALSE.equals(multiTenant)
+                            && Boolean.FALSE.equals(metaPropertiesEvaluated.getMultiTenant())
                             && MetaDataUtil.hasViewIndexTable(connection, table.getPhysicalName())) {
                         connection.setAutoCommit(true);
                         MetaDataUtil.deleteViewIndexSequences(connection, table.getPhysicalName(), table.isNamespaceMapped());
@@ -3940,9 +3813,19 @@ public class MetaDataClient {
         boolean wasAutoCommit = connection.getAutoCommit();
         try {
             String dataTableName = statement.getTableName();
-            String schemaName = statement.getTable().getName().getSchemaName();
             String indexName = statement.getTable().getName().getTableName();
             boolean isAsync = statement.isAsync();
+            String tenantId = connection.getTenantId() == null ? null : connection.getTenantId().getString();
+            PTable table = FromCompiler.getResolver(statement, connection).getTables().get(0).getTable();
+            String schemaName = statement.getTable().getName().getSchemaName();
+            String tableName = table.getTableName().getString();
+
+            Map<String, List<Pair<String, Object>>> properties=new HashMap<>(statement.getProps().size());;
+            MetaProperties metaProperties = loadStmtProperties(statement.getProps(),properties,table,false);
+
+            MetaPropertiesEvaluated metaPropertiesEvaluated = new MetaPropertiesEvaluated();
+            boolean changingPhoenixTableProperty= evaluateStmtProperties(metaProperties,metaPropertiesEvaluated,table,schemaName,tableName);
+
             PIndexState newIndexState = statement.getIndexState();
             if (isAsync && newIndexState != PIndexState.REBUILD) { throw new SQLExceptionInfo.Builder(
                     SQLExceptionCode.ASYNC_NOT_ALLOWED)
@@ -3980,7 +3863,15 @@ public class MetaDataClient {
             List<Mutation> tableMetadata = connection.getMutationState().toMutations(timeStamp).next().getSecond();
             connection.rollback();
 
-            MetaDataMutationResult result = connection.getQueryServices().updateIndexState(tableMetadata, dataTableName);
+
+            if (changingPhoenixTableProperty) {
+                incrementTableSeqNum(table,statement.getTableType(), 0, metaPropertiesEvaluated);
+                tableMetadata.addAll(connection.getMutationState().toMutations(timeStamp).next().getSecond());
+                connection.rollback();
+            }
+
+            MetaDataMutationResult result = connection.getQueryServices().updateIndexState(tableMetadata, dataTableName, properties, table);
+
             MutationCode code = result.getMutationCode();
             if (code == MutationCode.TABLE_NOT_FOUND) {
                 throw new TableNotFoundException(schemaName,indexName);
@@ -4306,4 +4197,351 @@ public class MetaDataClient {
             AccessControlClient.revoke(clusterConnection, changePermsStatement.getName(), Permission.Action.values());
         }
     }
+
+    private MetaProperties loadStmtProperties(ListMultimap<String, Pair<String, Object>> stmtProperties, Map<String, List<Pair<String, Object>>> properties, PTable table, boolean removeTableProps)
+            throws SQLException {
+        MetaProperties metaProperties = new MetaProperties();
+        for (String family : stmtProperties.keySet()) {
+            List<Pair<String, Object>> origPropsList = stmtProperties.get(family);
+            List<Pair<String, Object>> propsList = Lists.newArrayListWithExpectedSize(origPropsList.size());
+            for (Pair<String, Object> prop : origPropsList) {
+                String propName = prop.getFirst();
+                if (TableProperty.isPhoenixTableProperty(propName)) {
+                    TableProperty tableProp = TableProperty.valueOf(propName);
+                    tableProp.validate(true, !family.equals(QueryConstants.ALL_FAMILY_PROPERTIES_KEY), table.getType());
+                    Object value = tableProp.getValue(prop.getSecond());
+                    if (propName.equals(PTable.IS_IMMUTABLE_ROWS_PROP_NAME)) {
+                        metaProperties.setImmutableRowsProp((Boolean)value);
+                    } else if (propName.equals(PhoenixDatabaseMetaData.MULTI_TENANT)) {
+                        metaProperties.setMultiTenantProp((Boolean)value);
+                    } else if (propName.equals(DISABLE_WAL)) {
+                        metaProperties.setDisableWALProp((Boolean)value);
+                    } else if (propName.equals(STORE_NULLS)) {
+                        metaProperties.setStoreNullsProp((Boolean)value);
+                    } else if (propName.equals(TRANSACTIONAL)) {
+                        metaProperties.setIsTransactionalProp((Boolean)value);
+                    } else if (propName.equals(UPDATE_CACHE_FREQUENCY)) {
+                        metaProperties.setUpdateCacheFrequencyProp((Long)value);
+                    } else if (propName.equals(GUIDE_POSTS_WIDTH)) {
+                        metaProperties.setGuidePostWidth((Long)value);
+                    } else if (propName.equals(APPEND_ONLY_SCHEMA)) {
+                        metaProperties.setAppendOnlySchemaProp((Boolean) value);
+                    } else if (propName.equalsIgnoreCase(IMMUTABLE_STORAGE_SCHEME)) {
+                        metaProperties.setImmutableStorageSchemeProp((ImmutableStorageScheme)value);
+                    } else if (propName.equalsIgnoreCase(USE_STATS_FOR_PARALLELIZATION)) {
+                        metaProperties.setUseStatsForParallelizationProp((Boolean)value);
+                    }
+                }
+                // if removeTableProps is true only add the property if it is not a HTable or Phoenix Table property
+                if (!removeTableProps || (!TableProperty.isPhoenixTableProperty(propName) && !MetaDataUtil.isHTableProperty(propName))) {
+                    propsList.add(prop);
+                }
+            }
+            properties.put(family, propsList);
+        }
+        return metaProperties;
+    }
+
+    private boolean evaluateStmtProperties(MetaProperties metaProperties, MetaPropertiesEvaluated metaPropertiesEvaluated, PTable table, String schemaName, String tableName)
+            throws SQLException {
+        boolean changingPhoenixTableProperty = false;
+
+        if (metaProperties.getImmutableRowsProp() != null) {
+            if (metaProperties.getImmutableRowsProp().booleanValue() != table.isImmutableRows()) {
+                if (table.getImmutableStorageScheme() != ImmutableStorageScheme.ONE_CELL_PER_COLUMN) {
+                    throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_ALTER_IMMUTABLE_ROWS_PROPERTY)
+                            .setSchemaName(schemaName).setTableName(tableName).build().buildException();
+                }
+                metaPropertiesEvaluated.setIsImmutableRows(metaProperties.getImmutableRowsProp());
+                changingPhoenixTableProperty = true;
+            }
+        }
+
+        if (metaProperties.getMultiTenantProp() != null) {
+            if (metaProperties.getMultiTenantProp().booleanValue() != table.isMultiTenant()) {
+                metaPropertiesEvaluated.setMultiTenant(metaProperties.getMultiTenantProp());
+                changingPhoenixTableProperty = true;
+            }
+        }
+
+        if (metaProperties.getDisableWALProp() != null) {
+            if (metaProperties.getDisableWALProp().booleanValue() != table.isWALDisabled()) {
+                metaPropertiesEvaluated.setDisableWAL(metaProperties.getDisableWALProp());
+                changingPhoenixTableProperty = true;
+            }
+        }
+
+        if (metaProperties.getUpdateCacheFrequencyProp() != null) {
+            if (metaProperties.getUpdateCacheFrequencyProp().longValue() != table.getUpdateCacheFrequency()) {
+                metaPropertiesEvaluated.setUpdateCacheFrequency(metaProperties.getUpdateCacheFrequencyProp());
+                changingPhoenixTableProperty = true;
+            }
+        }
+
+        if (metaProperties.getAppendOnlySchemaProp() !=null) {
+            if (metaProperties.getAppendOnlySchemaProp() != table.isAppendOnlySchema()) {
+                metaPropertiesEvaluated.setAppendOnlySchema(metaProperties.getAppendOnlySchemaProp());
+                changingPhoenixTableProperty = true;
+            }
+        }
+
+        if (metaProperties.getImmutableStorageSchemeProp()!=null) {
+            if (table.getImmutableStorageScheme() == ONE_CELL_PER_COLUMN ||
+                    metaProperties.getImmutableStorageSchemeProp() == ONE_CELL_PER_COLUMN) {
+                throw new SQLExceptionInfo.Builder(SQLExceptionCode.INVALID_IMMUTABLE_STORAGE_SCHEME_CHANGE)
+                        .setSchemaName(schemaName).setTableName(tableName).build().buildException();
+            }
+            else if (metaProperties.getImmutableStorageSchemeProp() != table.getImmutableStorageScheme()) {
+                metaPropertiesEvaluated.setImmutableStorageScheme(metaProperties.getImmutableStorageSchemeProp());
+                changingPhoenixTableProperty = true;
+            }
+        }
+
+        if (metaProperties.getGuidePostWidth() == null || metaProperties.getGuidePostWidth() >= 0) {
+            metaPropertiesEvaluated.setGuidePostWidth(metaProperties.getGuidePostWidth());
+            changingPhoenixTableProperty = true;
+        }
+
+        if (metaProperties.getStoreNullsProp() != null) {
+            if (metaProperties.getStoreNullsProp().booleanValue() != table.getStoreNulls()) {
+                metaPropertiesEvaluated.setStoreNulls(metaProperties.getStoreNullsProp());
+                changingPhoenixTableProperty = true;
+            }
+        }
+
+        if (metaProperties.getUseStatsForParallelizationProp() != null
+                && (table.useStatsForParallelization() == null
+                || (metaProperties.getUseStatsForParallelizationProp().booleanValue() != table
+                .useStatsForParallelization()))) {
+            metaPropertiesEvaluated.setUseStatsForParallelization(metaProperties.getUseStatsForParallelizationProp());
+            changingPhoenixTableProperty = true;
+        }
+
+        if (metaProperties.getIsTransactionalProp() != null) {
+            if (metaProperties.getIsTransactionalProp().booleanValue() != table.isTransactional()) {
+                metaPropertiesEvaluated.setIsTransactional(metaProperties.getIsTransactionalProp());
+                // We can only go one way: from non transactional to transactional
+                // Going the other way would require rewriting the cell timestamps
+                // and doing a major compaction to get rid of any Tephra specific
+                // delete markers.
+                if (!metaPropertiesEvaluated.getIsTransactional()) {
+                    throw new SQLExceptionInfo.Builder(SQLExceptionCode.TX_MAY_NOT_SWITCH_TO_NON_TX)
+                            .setSchemaName(schemaName).setTableName(tableName).build().buildException();
+                }
+                // cannot create a transactional table if transactions are disabled
+                boolean transactionsEnabled = connection.getQueryServices().getProps().getBoolean(
+                        QueryServices.TRANSACTIONS_ENABLED,
+                        QueryServicesOptions.DEFAULT_TRANSACTIONS_ENABLED);
+                if (!transactionsEnabled) {
+                    throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_ALTER_TO_BE_TXN_IF_TXNS_DISABLED)
+                            .setSchemaName(schemaName).setTableName(tableName).build().buildException();
+                }
+                // cannot make a table transactional if it has a row timestamp column
+                if (SchemaUtil.hasRowTimestampColumn(table)) {
+                    throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_ALTER_TO_BE_TXN_WITH_ROW_TIMESTAMP)
+                            .setSchemaName(schemaName).setTableName(tableName)
+                            .build().buildException();
+                }
+                changingPhoenixTableProperty = true;
+                metaProperties.setNonTxToTx(true);
+            }
+        }
+        return changingPhoenixTableProperty;
+    }
+
+    class MetaProperties{
+        private Boolean isImmutableRowsProp = null;
+        private Boolean multiTenantProp = null;
+        private Boolean disableWALProp = null;
+        private Boolean storeNullsProp = null;
+        private Boolean isTransactionalProp = null;
+        private Long updateCacheFrequencyProp = null;
+        private Boolean appendOnlySchemaProp = null;
+        private Long guidePostWidth = -1L;
+        private ImmutableStorageScheme immutableStorageSchemeProp = null;
+        private Boolean useStatsForParallelizationProp = null;
+        private boolean nonTxToTx = false;
+
+        public Boolean getImmutableRowsProp() {
+            return isImmutableRowsProp;
+        }
+
+        public void setImmutableRowsProp(Boolean isImmutableRowsProp) {
+            this.isImmutableRowsProp = isImmutableRowsProp;
+        }
+
+        public Boolean getMultiTenantProp() {
+            return multiTenantProp;
+        }
+
+        public void setMultiTenantProp(Boolean multiTenantProp) {
+            this.multiTenantProp = multiTenantProp;
+        }
+
+        public Boolean getDisableWALProp() {
+            return disableWALProp;
+        }
+
+        public void setDisableWALProp(Boolean disableWALProp) {
+            this.disableWALProp = disableWALProp;
+        }
+
+        public Boolean getStoreNullsProp() {
+            return storeNullsProp;
+        }
+
+        public void setStoreNullsProp(Boolean storeNullsProp) {
+            this.storeNullsProp = storeNullsProp;
+        }
+
+        public Boolean getIsTransactionalProp() {
+            return isTransactionalProp;
+        }
+
+        public void setIsTransactionalProp(Boolean isTransactionalProp) {
+            this.isTransactionalProp = isTransactionalProp;
+        }
+
+        public Long getUpdateCacheFrequencyProp() {
+            return updateCacheFrequencyProp;
+        }
+
+        public void setUpdateCacheFrequencyProp(Long updateCacheFrequencyProp) {
+            this.updateCacheFrequencyProp = updateCacheFrequencyProp;
+        }
+
+        public Boolean getAppendOnlySchemaProp() {
+            return appendOnlySchemaProp;
+        }
+
+        public void setAppendOnlySchemaProp(Boolean appendOnlySchemaProp) {
+            this.appendOnlySchemaProp = appendOnlySchemaProp;
+        }
+
+        public Long getGuidePostWidth() {
+            return guidePostWidth;
+        }
+
+        public void setGuidePostWidth(Long guidePostWidth) {
+            this.guidePostWidth = guidePostWidth;
+        }
+
+        public ImmutableStorageScheme getImmutableStorageSchemeProp() {
+            return immutableStorageSchemeProp;
+        }
+
+        public void setImmutableStorageSchemeProp(
+                ImmutableStorageScheme immutableStorageSchemeProp) {
+            this.immutableStorageSchemeProp = immutableStorageSchemeProp;
+        }
+
+        public Boolean getUseStatsForParallelizationProp() {
+            return useStatsForParallelizationProp;
+        }
+
+        public void setUseStatsForParallelizationProp(Boolean useStatsForParallelizationProp) {
+            this.useStatsForParallelizationProp = useStatsForParallelizationProp;
+        }
+
+        public boolean getNonTxToTx() {
+            return nonTxToTx;
+        }
+
+        public void setNonTxToTx(boolean nonTxToTx) {
+            this.nonTxToTx = nonTxToTx;
+        }
+    }
+
+    class MetaPropertiesEvaluated{
+        private Boolean isImmutableRows;
+        private Boolean multiTenant = null;
+        private Boolean disableWAL = null;
+        private Long updateCacheFrequency = null;
+        private Boolean appendOnlySchema = null;
+        private Long guidePostWidth = -1L;
+        private ImmutableStorageScheme immutableStorageScheme = null;
+        private Boolean storeNulls = null;
+        private Boolean useStatsForParallelization = null;
+        private Boolean isTransactional = null;
+
+        public Boolean getIsImmutableRows() {
+            return isImmutableRows;
+        }
+
+        public void setIsImmutableRows(Boolean isImmutableRows) {
+            this.isImmutableRows = isImmutableRows;
+        }
+
+        public Boolean getMultiTenant() {
+            return multiTenant;
+        }
+
+        public void setMultiTenant(Boolean multiTenant) {
+            this.multiTenant = multiTenant;
+        }
+
+        public Boolean getDisableWAL() {
+            return disableWAL;
+        }
+
+        public void setDisableWAL(Boolean disableWAL) {
+            this.disableWAL = disableWAL;
+        }
+
+        public Long getUpdateCacheFrequency() {
+            return updateCacheFrequency;
+        }
+
+        public void setUpdateCacheFrequency(Long updateCacheFrequency) {
+            this.updateCacheFrequency = updateCacheFrequency;
+        }
+
+        public Boolean getAppendOnlySchema() {
+            return appendOnlySchema;
+        }
+
+        public void setAppendOnlySchema(Boolean appendOnlySchema) {
+            this.appendOnlySchema = appendOnlySchema;
+        }
+
+        public Long getGuidePostWidth() {
+            return guidePostWidth;
+        }
+
+        public void setGuidePostWidth(Long guidePostWidth) {
+            this.guidePostWidth = guidePostWidth;
+        }
+
+        public ImmutableStorageScheme getImmutableStorageScheme() {
+            return immutableStorageScheme;
+        }
+
+        public void setImmutableStorageScheme(ImmutableStorageScheme immutableStorageScheme) {
+            this.immutableStorageScheme = immutableStorageScheme;
+        }
+
+        public Boolean getStoreNulls() {
+            return storeNulls;
+        }
+
+        public void setStoreNulls(Boolean storeNulls) {
+            this.storeNulls = storeNulls;
+        }
+
+        public Boolean getUseStatsForParallelization() {
+            return useStatsForParallelization;
+        }
+
+        public void setUseStatsForParallelization(Boolean useStatsForParallelization) {
+            this.useStatsForParallelization = useStatsForParallelization;
+        }
+
+        public Boolean getIsTransactional() {
+            return isTransactional;
+        }
+
+        public void setIsTransactional(Boolean isTransactional) {
+            this.isTransactional = isTransactional;
+        }
+    }
 }


[08/16] phoenix git commit: Revert "PHOENIX-4386 Calculate the estimatedSize of MutationState using Map> mutations (addendum)"

Posted by ja...@apache.org.
Revert "PHOENIX-4386 Calculate the estimatedSize of MutationState using Map<TableRef, Map<ImmutableBytesPtr,RowMutationState>> mutations (addendum)"

This reverts commit 4e0c0a33ed8b401f7785dde8979041dd5ab9a1f4.


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

Branch: refs/heads/4.x-HBase-1.2
Commit: d523350ef9251dba06e2b9f76bab794896e18862
Parents: eb0f744
Author: Thomas D'Silva <td...@apache.org>
Authored: Wed Nov 29 02:37:55 2017 +0000
Committer: James Taylor <jt...@salesforce.com>
Committed: Sat Dec 16 16:42:54 2017 -0800

----------------------------------------------------------------------
 .../apache/phoenix/execute/PartialCommitIT.java |   5 +-
 .../apache/phoenix/compile/DeleteCompiler.java  |  11 +-
 .../apache/phoenix/compile/UpsertCompiler.java  |   7 +-
 .../apache/phoenix/execute/MutationState.java   | 127 +++++++------------
 .../java/org/apache/phoenix/util/IndexUtil.java |   4 +-
 .../org/apache/phoenix/util/KeyValueUtil.java   |   5 +-
 6 files changed, 61 insertions(+), 98 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/d523350e/phoenix-core/src/it/java/org/apache/phoenix/execute/PartialCommitIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/execute/PartialCommitIT.java b/phoenix-core/src/it/java/org/apache/phoenix/execute/PartialCommitIT.java
index e5b57e3..10fd7f8 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/execute/PartialCommitIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/execute/PartialCommitIT.java
@@ -33,6 +33,7 @@ import java.sql.SQLException;
 import java.sql.Statement;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.Comparator;
 import java.util.List;
 import java.util.Map;
@@ -51,8 +52,8 @@ import org.apache.hadoop.hbase.coprocessor.SimpleRegionObserver;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.end2end.BaseOwnClusterIT;
-import org.apache.phoenix.execute.MutationState.MultiRowMutationState;
 import org.apache.phoenix.hbase.index.Indexer;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.monitoring.GlobalMetric;
 import org.apache.phoenix.monitoring.MetricType;
@@ -284,7 +285,7 @@ public class PartialCommitIT extends BaseOwnClusterIT {
     private PhoenixConnection getConnectionWithTableOrderPreservingMutationState() throws SQLException {
         Connection con = driver.connect(url, new Properties());
         PhoenixConnection phxCon = new PhoenixConnection(con.unwrap(PhoenixConnection.class));
-        final Map<TableRef, MultiRowMutationState> mutations = Maps.newTreeMap(new TableRefComparator());
+        final Map<TableRef,Map<ImmutableBytesPtr,MutationState.RowMutationState>> mutations = Maps.newTreeMap(new TableRefComparator());
         // passing a null mutation state forces the connection.newMutationState() to be used to create the MutationState
         return new PhoenixConnection(phxCon, null) {
             @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d523350e/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
----------------------------------------------------------------------
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 a06e2ca..f9ca300 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
@@ -26,6 +26,7 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 
 import org.apache.hadoop.hbase.Cell;
@@ -42,7 +43,6 @@ import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.exception.SQLExceptionInfo;
 import org.apache.phoenix.execute.AggregatePlan;
 import org.apache.phoenix.execute.MutationState;
-import org.apache.phoenix.execute.MutationState.MultiRowMutationState;
 import org.apache.phoenix.execute.MutationState.RowMutationState;
 import org.apache.phoenix.filter.SkipScanFilter;
 import org.apache.phoenix.hbase.index.ValueGetter;
@@ -91,6 +91,7 @@ import org.apache.phoenix.util.ScanUtil;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
 import com.sun.istack.NotNull;
 
 public class DeleteCompiler {
@@ -120,14 +121,14 @@ public class DeleteCompiler {
         final int maxSize = services.getProps().getInt(QueryServices.MAX_MUTATION_SIZE_ATTRIB,QueryServicesOptions.DEFAULT_MAX_MUTATION_SIZE);
         final int maxSizeBytes = services.getProps().getInt(QueryServices.MAX_MUTATION_SIZE_BYTES_ATTRIB,QueryServicesOptions.DEFAULT_MAX_MUTATION_SIZE_BYTES);
         final int batchSize = Math.min(connection.getMutateBatchSize(), maxSize);
-        MultiRowMutationState mutations = new MultiRowMutationState(batchSize);
-        List<MultiRowMutationState> indexMutations = null;
+        Map<ImmutableBytesPtr,RowMutationState> mutations = Maps.newHashMapWithExpectedSize(batchSize);
+        List<Map<ImmutableBytesPtr,RowMutationState>> indexMutations = null;
         // If indexTableRef is set, we're deleting the rows from both the index table and
         // the data table through a single query to save executing an additional one.
         if (!otherTableRefs.isEmpty()) {
             indexMutations = Lists.newArrayListWithExpectedSize(otherTableRefs.size());
             for (int i = 0; i < otherTableRefs.size(); i++) {
-                indexMutations.add(new MultiRowMutationState(batchSize));
+                indexMutations.add(Maps.<ImmutableBytesPtr,RowMutationState>newHashMapWithExpectedSize(batchSize));
             }
         }
         List<PColumn> pkColumns = table.getPKColumns();
@@ -643,7 +644,7 @@ public class DeleteCompiler {
             // keys for our ranges
             ScanRanges ranges = context.getScanRanges();
             Iterator<KeyRange> iterator = ranges.getPointLookupKeyIterator();
-            MultiRowMutationState mutation = new MultiRowMutationState(ranges.getPointLookupCount());
+            Map<ImmutableBytesPtr,RowMutationState> mutation = Maps.newHashMapWithExpectedSize(ranges.getPointLookupCount());
             while (iterator.hasNext()) {
                 mutation.put(new ImmutableBytesPtr(iterator.next().getLowerRange()),
                         new RowMutationState(PRow.DELETE_MARKER, 0,

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d523350e/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
index a81a427..a51fd4c 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
@@ -47,7 +47,6 @@ import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.exception.SQLExceptionInfo;
 import org.apache.phoenix.execute.AggregatePlan;
 import org.apache.phoenix.execute.MutationState;
-import org.apache.phoenix.execute.MutationState.MultiRowMutationState;
 import org.apache.phoenix.execute.MutationState.RowMutationState;
 import org.apache.phoenix.execute.MutationState.RowTimestampColInfo;
 import org.apache.phoenix.expression.Determinism;
@@ -117,7 +116,7 @@ import com.google.common.collect.Sets;
 public class UpsertCompiler {
 
     private static void setValues(byte[][] values, int[] pkSlotIndex, int[] columnIndexes,
-            PTable table, MultiRowMutationState mutation,
+            PTable table, Map<ImmutableBytesPtr, RowMutationState> mutation,
             PhoenixStatement statement, boolean useServerTimestamp, IndexMaintainer maintainer,
             byte[][] viewConstants, byte[] onDupKeyBytes, int numSplColumns) throws SQLException {
         long columnValueSize = 0;
@@ -198,7 +197,7 @@ public class UpsertCompiler {
             }
         }
         int rowCount = 0;
-        MultiRowMutationState mutation = new MultiRowMutationState(batchSize);
+        Map<ImmutableBytesPtr, RowMutationState> mutation = Maps.newHashMapWithExpectedSize(batchSize);
         PTable table = tableRef.getTable();
         IndexMaintainer indexMaintainer = null;
         byte[][] viewConstants = null;
@@ -1178,7 +1177,7 @@ public class UpsertCompiler {
                     throw new IllegalStateException();
                 }
             }
-            MultiRowMutationState mutation = new MultiRowMutationState(1);
+            Map<ImmutableBytesPtr, RowMutationState> mutation = Maps.newHashMapWithExpectedSize(1);
             IndexMaintainer indexMaintainer = null;
             byte[][] viewConstants = null;
             if (table.getIndexType() == IndexType.LOCAL) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d523350e/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
index 993438e..15e905a 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
@@ -123,7 +123,7 @@ public class MutationState implements SQLCloseable {
     private final long batchSize;
     private final long batchSizeBytes;
     private long batchCount = 0L;
-    private final Map<TableRef, MultiRowMutationState> mutations;
+    private final Map<TableRef, Map<ImmutableBytesPtr,RowMutationState>> mutations;
     private final Set<String> uncommittedPhysicalNames = Sets.newHashSetWithExpectedSize(10);
 
     private long sizeOffset;
@@ -131,7 +131,7 @@ public class MutationState implements SQLCloseable {
     private long estimatedSize = 0;
     private int[] uncommittedStatementIndexes = EMPTY_STATEMENT_INDEX_ARRAY;
     private boolean isExternalTxContext = false;
-    private Map<TableRef, MultiRowMutationState> txMutations = Collections.emptyMap();
+    private Map<TableRef, Map<ImmutableBytesPtr,RowMutationState>> txMutations = Collections.emptyMap();
 
     final PhoenixTransactionContext phoenixTransactionContext;
 
@@ -159,12 +159,12 @@ public class MutationState implements SQLCloseable {
     }
 
     private MutationState(long maxSize, long maxSizeBytes, PhoenixConnection connection, boolean subTask, PhoenixTransactionContext txContext, long sizeOffset) {
-        this(maxSize, maxSizeBytes, connection, Maps.<TableRef, MultiRowMutationState>newHashMapWithExpectedSize(5), subTask, txContext);
+        this(maxSize, maxSizeBytes, connection, Maps.<TableRef, Map<ImmutableBytesPtr,RowMutationState>>newHashMapWithExpectedSize(5), subTask, txContext);
         this.sizeOffset = sizeOffset;
     }
 
     MutationState(long maxSize, long maxSizeBytes, PhoenixConnection connection,
-            Map<TableRef, MultiRowMutationState> mutations,
+            Map<TableRef, Map<ImmutableBytesPtr, RowMutationState>> mutations,
             boolean subTask, PhoenixTransactionContext txContext) {
         this.maxSize = maxSize;
         this.maxSizeBytes = maxSizeBytes;
@@ -189,7 +189,7 @@ public class MutationState implements SQLCloseable {
         }
     }
 
-    public MutationState(TableRef table, MultiRowMutationState mutations, long sizeOffset, long maxSize, long maxSizeBytes, PhoenixConnection connection)  throws SQLException {
+    public MutationState(TableRef table, Map<ImmutableBytesPtr,RowMutationState> mutations, long sizeOffset, long maxSize, long maxSizeBytes, PhoenixConnection connection) throws SQLException {
         this(maxSize, maxSizeBytes, connection, false, null, sizeOffset);
         if (!mutations.isEmpty()) {
             this.mutations.put(table, mutations);
@@ -350,7 +350,7 @@ public class MutationState implements SQLCloseable {
     }
 
     public static MutationState emptyMutationState(long maxSize, long maxSizeBytes, PhoenixConnection connection) {
-        MutationState state = new MutationState(maxSize, maxSizeBytes, connection, Collections.<TableRef, MultiRowMutationState>emptyMap(), false, null);
+        MutationState state = new MutationState(maxSize, maxSizeBytes, connection, Collections.<TableRef, Map<ImmutableBytesPtr,RowMutationState>>emptyMap(), false, null);
         state.sizeOffset = 0;
         return state;
     }
@@ -372,12 +372,12 @@ public class MutationState implements SQLCloseable {
         return sizeOffset + numRows;
     }
     
-    private void joinMutationState(TableRef tableRef, MultiRowMutationState srcRows,
-            Map<TableRef, MultiRowMutationState> dstMutations) {
+    private void joinMutationState(TableRef tableRef, Map<ImmutableBytesPtr,RowMutationState> srcRows,
+            Map<TableRef, Map<ImmutableBytesPtr, RowMutationState>> dstMutations) {
         PTable table = tableRef.getTable();
         boolean isIndex = table.getType() == PTableType.INDEX;
         boolean incrementRowCount = dstMutations == this.mutations;
-        MultiRowMutationState existingRows = dstMutations.put(tableRef, srcRows);
+        Map<ImmutableBytesPtr,RowMutationState> existingRows = dstMutations.put(tableRef, srcRows);
         if (existingRows != null) { // Rows for that table already exist
             // Loop through new rows and replace existing with new
             for (Map.Entry<ImmutableBytesPtr,RowMutationState> rowEntry : srcRows.entrySet()) {
@@ -389,12 +389,8 @@ public class MutationState implements SQLCloseable {
                         Map<PColumn,byte[]> newRow = rowEntry.getValue().getColumnValues();
                         // if new row is PRow.DELETE_MARKER, it means delete, and we don't need to merge it with existing row. 
                         if (newRow != PRow.DELETE_MARKER) {
-                            // decrement estimated size by the size of the old row
-                            estimatedSize-=existingRowMutationState.calculateEstimatedSize();
                             // Merge existing column values with new column values
                             existingRowMutationState.join(rowEntry.getValue());
-                            // increment estimated size by the size of the new row
-                            estimatedSize+=existingRowMutationState.calculateEstimatedSize();
                             // Now that the existing row has been merged with the new row, replace it back
                             // again (since it was merged with the new one above).
                             existingRows.put(rowEntry.getKey(), existingRowMutationState);
@@ -403,8 +399,6 @@ public class MutationState implements SQLCloseable {
                 } else {
                     if (incrementRowCount && !isIndex) { // Don't count index rows in row count
                         numRows++;
-                        // increment estimated size by the size of the new row
-                        estimatedSize += rowEntry.getValue().calculateEstimatedSize();
                     }
                 }
             }
@@ -412,25 +406,22 @@ public class MutationState implements SQLCloseable {
             dstMutations.put(tableRef, existingRows);
         } else {
             // Size new map at batch size as that's what it'll likely grow to.
-            MultiRowMutationState newRows = new MultiRowMutationState(connection.getMutateBatchSize());
+            Map<ImmutableBytesPtr,RowMutationState> newRows = Maps.newHashMapWithExpectedSize(connection.getMutateBatchSize());
             newRows.putAll(srcRows);
             dstMutations.put(tableRef, newRows);
             if (incrementRowCount && !isIndex) {
                 numRows += srcRows.size();
-                // if we added all the rows from newMutationState we can just increment the
-                // estimatedSize by newMutationState.estimatedSize
-                estimatedSize +=  srcRows.estimatedSize;
             }
         }
     }
     
-    private void joinMutationState(Map<TableRef, MultiRowMutationState> srcMutations, 
-            Map<TableRef, MultiRowMutationState> dstMutations) {
+    private void joinMutationState(Map<TableRef, Map<ImmutableBytesPtr, RowMutationState>> srcMutations, 
+            Map<TableRef, Map<ImmutableBytesPtr, RowMutationState>> dstMutations) {
         // Merge newMutation with this one, keeping state from newMutation for any overlaps
-        for (Map.Entry<TableRef, MultiRowMutationState> entry : srcMutations.entrySet()) {
+        for (Map.Entry<TableRef, Map<ImmutableBytesPtr,RowMutationState>> entry : srcMutations.entrySet()) {
             // Replace existing entries for the table with new entries
             TableRef tableRef = entry.getKey();
-            MultiRowMutationState srcRows = entry.getValue();
+            Map<ImmutableBytesPtr,RowMutationState> srcRows = entry.getValue();
             joinMutationState(tableRef, srcRows, dstMutations);
         }
     }
@@ -448,7 +439,19 @@ public class MutationState implements SQLCloseable {
         phoenixTransactionContext.join(newMutationState.getPhoenixTransactionContext());
 
         this.sizeOffset += newMutationState.sizeOffset;
+        int oldNumRows = this.numRows;
         joinMutationState(newMutationState.mutations, this.mutations);
+        if (newMutationState.numRows>0) {
+            // if we added all the rows from newMutationState we can just increment the
+            // estimatedSize by newMutationState.estimatedSize
+            if (newMutationState.numRows == this.numRows-oldNumRows) {
+                this.estimatedSize +=  newMutationState.estimatedSize;
+            }
+            // we merged the two mutation states so we need to recalculate the size
+            else {
+                this.estimatedSize = KeyValueUtil.getEstimatedRowMutationSize(this.mutations);
+            }
+        }
         if (!newMutationState.txMutations.isEmpty()) {
             if (txMutations.isEmpty()) {
                 txMutations = Maps.newHashMapWithExpectedSize(mutations.size());
@@ -486,7 +489,7 @@ public class MutationState implements SQLCloseable {
         return ptr;
     }
     
-    private Iterator<Pair<PName,List<Mutation>>> addRowMutations(final TableRef tableRef, final MultiRowMutationState values,
+    private Iterator<Pair<PName,List<Mutation>>> addRowMutations(final TableRef tableRef, final Map<ImmutableBytesPtr, RowMutationState> values,
             final long mutationTimestamp, final long serverTimestamp, boolean includeAllIndexes, final boolean sendAll) { 
         final PTable table = tableRef.getTable();
         final Iterator<PTable> indexes = // Only maintain tables with immutable rows through this client-side mechanism
@@ -521,10 +524,10 @@ public class MutationState implements SQLCloseable {
                     // we may also have to include delete mutations for immutable tables if we are not processing all the tables in the mutations map
                     if (!sendAll) {
                         TableRef key = new TableRef(index);
-                        MultiRowMutationState multiRowMutationState = mutations.remove(key);
-                        if (multiRowMutationState!=null) {
+                        Map<ImmutableBytesPtr, RowMutationState> rowToColumnMap = mutations.remove(key);
+                        if (rowToColumnMap!=null) {
                             final List<Mutation> deleteMutations = Lists.newArrayList();
-                            generateMutations(tableRef, mutationTimestamp, serverTimestamp, multiRowMutationState, deleteMutations, null);
+                            generateMutations(tableRef, mutationTimestamp, serverTimestamp, rowToColumnMap, deleteMutations, null);
                             indexMutations.addAll(deleteMutations);
                         }
                     }
@@ -543,14 +546,14 @@ public class MutationState implements SQLCloseable {
     }
 
     private void generateMutations(final TableRef tableRef, final long mutationTimestamp,
-            final long serverTimestamp, final MultiRowMutationState values,
+            final long serverTimestamp, final Map<ImmutableBytesPtr, RowMutationState> values,
             final List<Mutation> mutationList, final List<Mutation> mutationsPertainingToIndex) {
         final PTable table = tableRef.getTable();
         boolean tableWithRowTimestampCol = table.getRowTimestampColPos() != -1;
         Iterator<Map.Entry<ImmutableBytesPtr, RowMutationState>> iterator =
                 values.entrySet().iterator();
         long timestampToUse = mutationTimestamp;
-        MultiRowMutationState modifiedValues = new MultiRowMutationState(16);
+        Map<ImmutableBytesPtr, RowMutationState> modifiedValues = Maps.newHashMap();
         while (iterator.hasNext()) {
             Map.Entry<ImmutableBytesPtr, RowMutationState> rowEntry = iterator.next();
             byte[] onDupKeyBytes = rowEntry.getValue().getOnDupKeyBytes();
@@ -625,7 +628,7 @@ public class MutationState implements SQLCloseable {
     }
     
     public Iterator<Pair<byte[],List<Mutation>>> toMutations(final boolean includeMutableIndexes, final Long tableTimestamp) {
-        final Iterator<Map.Entry<TableRef, MultiRowMutationState>> iterator = this.mutations.entrySet().iterator();
+        final Iterator<Map.Entry<TableRef, Map<ImmutableBytesPtr,RowMutationState>>> iterator = this.mutations.entrySet().iterator();
         if (!iterator.hasNext()) {
             return Collections.emptyIterator();
         }
@@ -633,7 +636,7 @@ public class MutationState implements SQLCloseable {
         final long serverTimestamp = getTableTimestamp(tableTimestamp, scn);
         final long mutationTimestamp = getMutationTimestamp(scn);
         return new Iterator<Pair<byte[],List<Mutation>>>() {
-            private Map.Entry<TableRef, MultiRowMutationState> current = iterator.next();
+            private Map.Entry<TableRef, Map<ImmutableBytesPtr,RowMutationState>> current = iterator.next();
             private Iterator<Pair<byte[],List<Mutation>>> innerIterator = init();
                     
             private Iterator<Pair<byte[],List<Mutation>>> init() {
@@ -697,14 +700,14 @@ public class MutationState implements SQLCloseable {
     private long[] validateAll() throws SQLException {
         int i = 0;
         long[] timeStamps = new long[this.mutations.size()];
-        for (Map.Entry<TableRef, MultiRowMutationState> entry : mutations.entrySet()) {
+        for (Map.Entry<TableRef, Map<ImmutableBytesPtr,RowMutationState>> entry : mutations.entrySet()) {
             TableRef tableRef = entry.getKey();
             timeStamps[i++] = validateAndGetServerTimestamp(tableRef, entry.getValue());
         }
         return timeStamps;
     }
     
-    private long validateAndGetServerTimestamp(TableRef tableRef, MultiRowMutationState rowKeyToColumnMap) throws SQLException {
+    private long validateAndGetServerTimestamp(TableRef tableRef, Map<ImmutableBytesPtr, RowMutationState> rowKeyToColumnMap) throws SQLException {
         Long scn = connection.getSCN();
         MetaDataClient client = new MetaDataClient(connection);
         long serverTimeStamp = tableRef.getTimeStamp();
@@ -916,7 +919,7 @@ public class MutationState implements SQLCloseable {
             sendAll = true;
         }
 
-        MultiRowMutationState multiRowMutationState;
+        Map<ImmutableBytesPtr, RowMutationState> valuesMap;
         Map<TableInfo,List<Mutation>> physicalTableMutationMap = Maps.newLinkedHashMap(); 
         // add tracing for this operation
         try (TraceScope trace = Tracing.startNewSpan(connection, "Committing mutations to tables")) {
@@ -925,16 +928,16 @@ public class MutationState implements SQLCloseable {
             while (tableRefIterator.hasNext()) {
                 // at this point we are going through mutations for each table
                 final TableRef tableRef = tableRefIterator.next();
-                multiRowMutationState = mutations.get(tableRef);
-                if (multiRowMutationState == null || multiRowMutationState.isEmpty()) {
+                valuesMap = mutations.get(tableRef);
+                if (valuesMap == null || valuesMap.isEmpty()) {
                     continue;
                 }
                 // Validate as we go if transactional since we can undo if a problem occurs (which is unlikely)
-                long serverTimestamp = serverTimeStamps == null ? validateAndGetServerTimestamp(tableRef, multiRowMutationState) : serverTimeStamps[i++];
+                long serverTimestamp = serverTimeStamps == null ? validateAndGetServerTimestamp(tableRef, valuesMap) : serverTimeStamps[i++];
                 Long scn = connection.getSCN();
                 long mutationTimestamp = scn == null ? HConstants.LATEST_TIMESTAMP : scn;
                 final PTable table = tableRef.getTable();
-                Iterator<Pair<PName,List<Mutation>>> mutationsIterator = addRowMutations(tableRef, multiRowMutationState, mutationTimestamp, serverTimestamp, false, sendAll);
+                Iterator<Pair<PName,List<Mutation>>> mutationsIterator = addRowMutations(tableRef, valuesMap, mutationTimestamp, serverTimestamp, false, sendAll);
                 // build map from physical table to mutation list
                 boolean isDataTable = true;
                 while (mutationsIterator.hasNext()) {
@@ -952,7 +955,7 @@ public class MutationState implements SQLCloseable {
                 // involved in the transaction since none of them would have been
                 // committed in the event of a failure.
                 if (table.isTransactional()) {
-                    addUncommittedStatementIndexes(multiRowMutationState.values());
+                    addUncommittedStatementIndexes(valuesMap.values());
                     if (txMutations.isEmpty()) {
                         txMutations = Maps.newHashMapWithExpectedSize(mutations.size());
                     }
@@ -961,7 +964,7 @@ public class MutationState implements SQLCloseable {
                     // in the event that we need to replay the commit.
                     // Copy TableRef so we have the original PTable and know when the
                     // indexes have changed.
-                    joinMutationState(new TableRef(tableRef), multiRowMutationState, txMutations);
+                    joinMutationState(new TableRef(tableRef), valuesMap, txMutations);
                 }
             }
             long serverTimestamp = HConstants.LATEST_TIMESTAMP;
@@ -1185,7 +1188,7 @@ public class MutationState implements SQLCloseable {
     }
     
     private int[] getUncommittedStatementIndexes() {
-        for (MultiRowMutationState rowMutationMap : mutations.values()) {
+        for (Map<ImmutableBytesPtr, RowMutationState> rowMutationMap : mutations.values()) {
             addUncommittedStatementIndexes(rowMutationMap.values());
         }
         return uncommittedStatementIndexes;
@@ -1218,7 +1221,7 @@ public class MutationState implements SQLCloseable {
     }
 
     public void commit() throws SQLException {
-        Map<TableRef, MultiRowMutationState> txMutations = Collections.emptyMap();
+        Map<TableRef, Map<ImmutableBytesPtr,RowMutationState>> txMutations = Collections.emptyMap();
         int retryCount = 0;
         do {
             boolean sendSuccessful=false;
@@ -1428,46 +1431,6 @@ public class MutationState implements SQLCloseable {
         }
     }
     
-    public static class MultiRowMutationState {
-        private Map<ImmutableBytesPtr,RowMutationState> rowKeyToRowMutationState;
-        private long estimatedSize;
-        
-        public MultiRowMutationState(int size) {
-            this.rowKeyToRowMutationState = Maps.newHashMapWithExpectedSize(size);
-            this.estimatedSize = 0;
-        }
-        
-        public RowMutationState put(ImmutableBytesPtr ptr, RowMutationState rowMutationState) { 
-            estimatedSize += rowMutationState.calculateEstimatedSize();
-            return rowKeyToRowMutationState.put(ptr, rowMutationState);
-        }
-        
-        public void putAll(MultiRowMutationState other) {
-            estimatedSize += other.estimatedSize;
-            rowKeyToRowMutationState.putAll(other.rowKeyToRowMutationState);
-        }
-        
-        public boolean isEmpty() {
-            return rowKeyToRowMutationState.isEmpty();
-        }
-        
-        public int size() {
-            return rowKeyToRowMutationState.size();
-        }
-        
-        public Set<Entry<ImmutableBytesPtr, RowMutationState>> entrySet() {
-            return rowKeyToRowMutationState.entrySet();
-        }
-        
-        public void clear(){
-            rowKeyToRowMutationState.clear();
-        }
-        
-        public Collection<RowMutationState> values() {
-            return rowKeyToRowMutationState.values();
-        }
-    }
-    
     public static class RowMutationState {
         @Nonnull private Map<PColumn,byte[]> columnValues;
         private int[] statementIndexes;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d523350e/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
index 74f91b4..b23ea1b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
@@ -73,7 +73,7 @@ import org.apache.phoenix.coprocessor.generated.MetaDataProtos.MetaDataService;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.UpdateIndexStateRequest;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.exception.SQLExceptionInfo;
-import org.apache.phoenix.execute.MutationState.MultiRowMutationState;
+import org.apache.phoenix.execute.MutationState.RowMutationState;
 import org.apache.phoenix.execute.TupleProjector;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.expression.KeyValueColumnExpression;
@@ -296,7 +296,7 @@ public class IndexUtil {
     }
     
     public static List<Mutation> generateIndexData(final PTable table, PTable index,
-            final MultiRowMutationState multiRowMutationState, List<Mutation> dataMutations, final KeyValueBuilder kvBuilder, PhoenixConnection connection)
+            final Map<ImmutableBytesPtr, RowMutationState> valuesMap, List<Mutation> dataMutations, final KeyValueBuilder kvBuilder, PhoenixConnection connection)
             throws SQLException {
         try {
         	final ImmutableBytesPtr ptr = new ImmutableBytesPtr();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d523350e/phoenix-core/src/main/java/org/apache/phoenix/util/KeyValueUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/KeyValueUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/KeyValueUtil.java
index df6a349..318c9d6 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/KeyValueUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/KeyValueUtil.java
@@ -30,7 +30,6 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValue.Type;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.phoenix.execute.MutationState.MultiRowMutationState;
 import org.apache.phoenix.execute.MutationState.RowMutationState;
 import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.hbase.index.util.KeyValueBuilder;
@@ -189,10 +188,10 @@ public class KeyValueUtil {
      * @return estimated row size
      */
     public static long
-            getEstimatedRowMutationSize(Map<TableRef, MultiRowMutationState> tableMutationMap) {
+            getEstimatedRowMutationSize(Map<TableRef, Map<ImmutableBytesPtr, RowMutationState>> tableMutationMap) {
         long size = 0;
         // iterate over table
-        for (Entry<TableRef, MultiRowMutationState> tableEntry : tableMutationMap.entrySet()) {
+        for (Entry<TableRef, Map<ImmutableBytesPtr, RowMutationState>> tableEntry : tableMutationMap.entrySet()) {
             // iterate over rows
             for (Entry<ImmutableBytesPtr, RowMutationState> rowEntry : tableEntry.getValue().entrySet()) {
                 size += calculateRowMutationSize(rowEntry);


[06/16] phoenix git commit: PHOENIX-4386 Calculate the estimatedSize of MutationState using Map> mutations

Posted by ja...@apache.org.
PHOENIX-4386 Calculate the estimatedSize of MutationState using Map<TableRef, Map<ImmutableBytesPtr,RowMutationState>> mutations


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

Branch: refs/heads/4.x-HBase-1.2
Commit: a6db039268964534715eab31fcc985861de4d0a9
Parents: bab1e46
Author: Thomas D'Silva <td...@apache.org>
Authored: Fri Nov 17 19:11:43 2017 +0000
Committer: James Taylor <jt...@salesforce.com>
Committed: Sat Dec 16 16:42:54 2017 -0800

----------------------------------------------------------------------
 .../apache/phoenix/end2end/MutationStateIT.java | 144 +++++++++++++++++++
 .../org/apache/phoenix/end2end/QueryMoreIT.java |  42 ------
 .../apache/phoenix/compile/DeleteCompiler.java  |   6 +-
 .../apache/phoenix/compile/UpsertCompiler.java  |   4 +-
 .../apache/phoenix/execute/MutationState.java   |  50 +++++--
 .../org/apache/phoenix/util/KeyValueUtil.java   |  51 ++-----
 6 files changed, 201 insertions(+), 96 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/a6db0392/phoenix-core/src/it/java/org/apache/phoenix/end2end/MutationStateIT.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..2d5f360
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MutationStateIT.java
@@ -0,0 +1,144 @@
+package org.apache.phoenix.end2end;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Properties;
+
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.execute.MutationState;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.query.QueryServices;
+import org.junit.Test;
+
+public class MutationStateIT extends ParallelStatsDisabledIT {
+
+    private static final String DDL =
+            " (ORGANIZATION_ID CHAR(15) NOT NULL, SCORE DOUBLE, "
+            + "ENTITY_ID CHAR(15) NOT NULL, TAGS VARCHAR, CONSTRAINT PAGE_SNAPSHOT_PK "
+            + "PRIMARY KEY (ORGANIZATION_ID, ENTITY_ID DESC)) MULTI_TENANT=TRUE";
+
+    private void upsertRows(PhoenixConnection conn, String fullTableName) throws SQLException {
+        PreparedStatement stmt =
+                conn.prepareStatement("upsert into " + fullTableName
+                        + " (organization_id, entity_id, score) values (?,?,?)");
+        for (int i = 0; i < 10000; i++) {
+            stmt.setString(1, "AAAA" + i);
+            stmt.setString(2, "BBBB" + i);
+            stmt.setInt(3, 1);
+            stmt.execute();
+        }
+    }
+
+    @Test
+    public void testMaxMutationSize() throws Exception {
+        Properties connectionProperties = new Properties();
+        connectionProperties.setProperty(QueryServices.MAX_MUTATION_SIZE_ATTRIB, "3");
+        connectionProperties.setProperty(QueryServices.MAX_MUTATION_SIZE_BYTES_ATTRIB, "1000000");
+        PhoenixConnection connection =
+                (PhoenixConnection) DriverManager.getConnection(getUrl(), connectionProperties);
+        String fullTableName = generateUniqueName();
+        try (Statement stmt = connection.createStatement()) {
+            stmt.execute(
+                "CREATE TABLE " + fullTableName + DDL);
+        }
+        try {
+            upsertRows(connection, fullTableName);
+            fail();
+        } catch (SQLException e) {
+            assertEquals(SQLExceptionCode.MAX_MUTATION_SIZE_EXCEEDED.getErrorCode(),
+                e.getErrorCode());
+        }
+
+        // set the max mutation size (bytes) to a low value
+        connectionProperties.setProperty(QueryServices.MAX_MUTATION_SIZE_ATTRIB, "1000");
+        connectionProperties.setProperty(QueryServices.MAX_MUTATION_SIZE_BYTES_ATTRIB, "4");
+        connection =
+                (PhoenixConnection) DriverManager.getConnection(getUrl(), connectionProperties);
+        try {
+            upsertRows(connection, fullTableName);
+            fail();
+        } catch (SQLException e) {
+            assertEquals(SQLExceptionCode.MAX_MUTATION_SIZE_BYTES_EXCEEDED.getErrorCode(),
+                e.getErrorCode());
+        }
+    }
+
+    @Test
+    public void testMutationEstimatedSize() throws Exception {
+        PhoenixConnection conn = (PhoenixConnection) DriverManager.getConnection(getUrl());
+        conn.setAutoCommit(false);
+        String fullTableName = generateUniqueName();
+        try (Statement stmt = conn.createStatement()) {
+            stmt.execute(
+                "CREATE TABLE " + fullTableName + DDL);
+        }
+
+        // upserting rows should increase the mutation state size
+        MutationState state = conn.unwrap(PhoenixConnection.class).getMutationState();
+        long prevEstimatedSize = state.getEstimatedSize();
+        upsertRows(conn, fullTableName);
+        assertTrue("Mutation state size should have increased",
+            state.getEstimatedSize() > prevEstimatedSize);
+        
+        
+        // after commit or rollback the size should be zero
+        conn.commit();
+        assertEquals("Mutation state size should be zero after commit", 0,
+            state.getEstimatedSize());
+        upsertRows(conn, fullTableName);
+        conn.rollback();
+        assertEquals("Mutation state size should be zero after rollback", 0,
+            state.getEstimatedSize());
+
+        // upsert one row
+        PreparedStatement stmt =
+                conn.prepareStatement("upsert into " + fullTableName
+                        + " (organization_id, entity_id, score) values (?,?,?)");
+        stmt.setString(1, "ZZZZ");
+        stmt.setString(2, "YYYY");
+        stmt.setInt(3, 1);
+        stmt.execute();
+        assertTrue("Mutation state size should be greater than zero ", state.getEstimatedSize()>0);
+
+        prevEstimatedSize = state.getEstimatedSize();
+        // upserting the same row twice should not increase the size
+        stmt.setString(1, "ZZZZ");
+        stmt.setString(2, "YYYY");
+        stmt.setInt(3, 1);
+        stmt.execute();
+        assertEquals(
+            "Mutation state size should only increase 4 bytes (size of the new statement index)",
+            prevEstimatedSize + 4, state.getEstimatedSize());
+        
+        prevEstimatedSize = state.getEstimatedSize();
+        // changing the value of one column of a row to a larger value should increase the estimated size 
+        stmt =
+                conn.prepareStatement("upsert into " + fullTableName
+                        + " (organization_id, entity_id, score, tags) values (?,?,?,?)");
+        stmt.setString(1, "ZZZZ");
+        stmt.setString(2, "YYYY");
+        stmt.setInt(3, 1);
+        stmt.setString(4, "random text string random text string random text string");
+        stmt.execute();
+        assertTrue("Mutation state size should increase", prevEstimatedSize+4 < state.getEstimatedSize());
+        
+        prevEstimatedSize = state.getEstimatedSize();
+        // changing the value of one column of a row to a smaller value should decrease the estimated size 
+        stmt =
+                conn.prepareStatement("upsert into " + fullTableName
+                        + " (organization_id, entity_id, score, tags) values (?,?,?,?)");
+        stmt.setString(1, "ZZZZ");
+        stmt.setString(2, "YYYY");
+        stmt.setInt(3, 1);
+        stmt.setString(4, "");
+        stmt.execute();
+        assertTrue("Mutation state size should decrease", prevEstimatedSize+4 > state.getEstimatedSize());
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a6db0392/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryMoreIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryMoreIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryMoreIT.java
index 77cb19f..9109c12 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
@@ -22,7 +22,6 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
 
 import java.sql.Connection;
 import java.sql.Date;
@@ -39,7 +38,6 @@ import java.util.Properties;
 
 import org.apache.hadoop.hbase.util.Base64;
 import org.apache.hadoop.hbase.util.Pair;
-import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.util.PhoenixRuntime;
@@ -510,46 +508,6 @@ public class QueryMoreIT extends ParallelStatsDisabledIT {
         assertEquals(4L, connection.getMutationState().getBatchCount());
     }
     
-    @Test
-    public void testMaxMutationSize() throws Exception {
-        Properties connectionProperties = new Properties();
-        connectionProperties.setProperty(QueryServices.MAX_MUTATION_SIZE_ATTRIB, "3");
-        connectionProperties.setProperty(QueryServices.MAX_MUTATION_SIZE_BYTES_ATTRIB, "1000000");
-        PhoenixConnection connection = (PhoenixConnection) DriverManager.getConnection(getUrl(), connectionProperties);
-        String fullTableName = generateUniqueName();
-        try (Statement stmt = connection.createStatement()) {
-            stmt.execute("CREATE TABLE " + fullTableName + "(\n" +
-                "    ORGANIZATION_ID CHAR(15) NOT NULL,\n" +
-                "    SCORE DOUBLE NOT NULL,\n" +
-                "    ENTITY_ID CHAR(15) NOT NULL\n" +
-                "    CONSTRAINT PAGE_SNAPSHOT_PK PRIMARY KEY (\n" +
-                "        ORGANIZATION_ID,\n" +
-                "        SCORE DESC,\n" +
-                "        ENTITY_ID DESC\n" +
-                "    )\n" +
-                ") MULTI_TENANT=TRUE");
-        }
-        try {
-            upsertRows(connection, fullTableName);
-            fail();
-        }
-        catch(SQLException e) {
-            assertEquals(SQLExceptionCode.MAX_MUTATION_SIZE_EXCEEDED.getErrorCode(), e.getErrorCode());
-        }
-        
-        // set the max mutation size (bytes) to a low value
-        connectionProperties.setProperty(QueryServices.MAX_MUTATION_SIZE_ATTRIB, "1000");
-        connectionProperties.setProperty(QueryServices.MAX_MUTATION_SIZE_BYTES_ATTRIB, "4");
-        connection = (PhoenixConnection) DriverManager.getConnection(getUrl(), connectionProperties);
-        try {
-            upsertRows(connection, fullTableName);
-            fail();
-        }
-        catch(SQLException e) {
-            assertEquals(SQLExceptionCode.MAX_MUTATION_SIZE_BYTES_EXCEEDED.getErrorCode(), e.getErrorCode());
-        }
-    }
-
     private void upsertRows(PhoenixConnection conn, String fullTableName) throws SQLException {
         PreparedStatement stmt = conn.prepareStatement("upsert into " + fullTableName +
                 " (organization_id, entity_id, score) values (?,?,?)");

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a6db0392/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
----------------------------------------------------------------------
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 8d9a5b6..f9ca300 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
@@ -207,7 +207,7 @@ public class DeleteCompiler {
                 // row key will already have its value.
                 // Check for otherTableRefs being empty required when deleting directly from the index
                 if (otherTableRefs.isEmpty() || table.getIndexType() != IndexType.LOCAL) {
-                    mutations.put(rowKeyPtr, new RowMutationState(PRow.DELETE_MARKER, statement.getConnection().getStatementExecutionCounter(), NULL_ROWTIMESTAMP_INFO, null));
+                    mutations.put(rowKeyPtr, new RowMutationState(PRow.DELETE_MARKER, 0, statement.getConnection().getStatementExecutionCounter(), NULL_ROWTIMESTAMP_INFO, null));
                 }
                 for (int i = 0; i < otherTableRefs.size(); i++) {
                     PTable otherTable = otherTableRefs.get(i).getTable();
@@ -221,7 +221,7 @@ public class DeleteCompiler {
                     } else {
                         indexPtr.set(maintainers[i].buildRowKey(getter, rowKeyPtr, null, null, HConstants.LATEST_TIMESTAMP));
                     }
-                    indexMutations.get(i).put(indexPtr, new RowMutationState(PRow.DELETE_MARKER, statement.getConnection().getStatementExecutionCounter(), NULL_ROWTIMESTAMP_INFO, null));
+                    indexMutations.get(i).put(indexPtr, new RowMutationState(PRow.DELETE_MARKER, 0, statement.getConnection().getStatementExecutionCounter(), NULL_ROWTIMESTAMP_INFO, null));
                 }
                 if (mutations.size() > maxSize) {
                     throw new IllegalArgumentException("MutationState size of " + mutations.size() + " is bigger than max allowed size of " + maxSize);
@@ -647,7 +647,7 @@ public class DeleteCompiler {
             Map<ImmutableBytesPtr,RowMutationState> mutation = Maps.newHashMapWithExpectedSize(ranges.getPointLookupCount());
             while (iterator.hasNext()) {
                 mutation.put(new ImmutableBytesPtr(iterator.next().getLowerRange()),
-                        new RowMutationState(PRow.DELETE_MARKER,
+                        new RowMutationState(PRow.DELETE_MARKER, 0,
                                 statement.getConnection().getStatementExecutionCounter(), NULL_ROWTIMESTAMP_INFO, null));
             }
             return new MutationState(dataPlan.getTableRef(), mutation, 0, maxSize, maxSizeBytes, connection);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a6db0392/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
index bc3e289..a51fd4c 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
@@ -119,6 +119,7 @@ public class UpsertCompiler {
             PTable table, Map<ImmutableBytesPtr, RowMutationState> mutation,
             PhoenixStatement statement, boolean useServerTimestamp, IndexMaintainer maintainer,
             byte[][] viewConstants, byte[] onDupKeyBytes, int numSplColumns) throws SQLException {
+        long columnValueSize = 0;
         Map<PColumn,byte[]> columnValues = Maps.newHashMapWithExpectedSize(columnIndexes.length);
         byte[][] pkValues = new byte[table.getPKColumns().size()][];
         // If the table uses salting, the first byte is the salting byte, set to an empty array
@@ -148,6 +149,7 @@ public class UpsertCompiler {
                 }
             } else {
                 columnValues.put(column, value);
+                columnValueSize += (column.getEstimatedSize() + value.length);
             }
         }
         ImmutableBytesPtr ptr = new ImmutableBytesPtr();
@@ -166,7 +168,7 @@ public class UpsertCompiler {
                     regionPrefix.length));
             }
         } 
-        mutation.put(ptr, new RowMutationState(columnValues, statement.getConnection().getStatementExecutionCounter(), rowTsColInfo, onDupKeyBytes));
+        mutation.put(ptr, new RowMutationState(columnValues, columnValueSize, statement.getConnection().getStatementExecutionCounter(), rowTsColInfo, onDupKeyBytes));
     }
     
     public static MutationState upsertSelect(StatementContext childContext, TableRef tableRef, RowProjector projector,

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a6db0392/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
index 0faa20c..15e905a 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
@@ -96,6 +96,7 @@ import org.apache.phoenix.util.SQLCloseable;
 import org.apache.phoenix.util.SQLCloseables;
 import org.apache.phoenix.util.ScanUtil;
 import org.apache.phoenix.util.ServerUtil;
+import org.apache.phoenix.util.SizedUtil;
 import org.apache.phoenix.util.TransactionUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -194,9 +195,13 @@ public class MutationState implements SQLCloseable {
             this.mutations.put(table, mutations);
         }
         this.numRows = mutations.size();
-        this.estimatedSize = KeyValueUtil.getEstimatedRowSize(table, mutations);
+        this.estimatedSize = KeyValueUtil.getEstimatedRowMutationSize(this.mutations);
         throwIfTooBig();
     }
+    
+    public long getEstimatedSize() {
+        return estimatedSize;
+    }
 
     public long getMaxSize() {
         return maxSize;
@@ -436,9 +441,16 @@ public class MutationState implements SQLCloseable {
         this.sizeOffset += newMutationState.sizeOffset;
         int oldNumRows = this.numRows;
         joinMutationState(newMutationState.mutations, this.mutations);
-        // here we increment the estimated size by the fraction of new rows we added from the newMutationState 
         if (newMutationState.numRows>0) {
-            this.estimatedSize += ((double)(this.numRows-oldNumRows)/newMutationState.numRows) * newMutationState.estimatedSize;
+            // if we added all the rows from newMutationState we can just increment the
+            // estimatedSize by newMutationState.estimatedSize
+            if (newMutationState.numRows == this.numRows-oldNumRows) {
+                this.estimatedSize +=  newMutationState.estimatedSize;
+            }
+            // we merged the two mutation states so we need to recalculate the size
+            else {
+                this.estimatedSize = KeyValueUtil.getEstimatedRowMutationSize(this.mutations);
+            }
         }
         if (!newMutationState.txMutations.isEmpty()) {
             if (txMutations.isEmpty()) {
@@ -974,8 +986,6 @@ public class MutationState implements SQLCloseable {
                 long mutationCommitTime = 0;
                 long numFailedMutations = 0;;
                 long startTime = 0;
-                long startNumRows = numRows;
-                long startEstimatedSize = estimatedSize;
                 do {
                     TableRef origTableRef = tableInfo.getOrigTableRef();
                     PTable table = origTableRef.getTable();
@@ -1021,13 +1031,13 @@ public class MutationState implements SQLCloseable {
                         GLOBAL_MUTATION_COMMIT_TIME.update(mutationCommitTime);
                         numFailedMutations = 0;
                         
+                        // Remove batches as we process them
+                        mutations.remove(origTableRef);
                         if (tableInfo.isDataTable()) {
                             numRows -= numMutations;
-                            // decrement estimated size by the fraction of rows we sent to hbase
-                            estimatedSize -= ((double)numMutations/startNumRows)*startEstimatedSize;
+                            // recalculate the estimated size
+                            estimatedSize = KeyValueUtil.getEstimatedRowMutationSize(mutations);
                         }
-                        // Remove batches as we process them
-                        mutations.remove(origTableRef);
                     } catch (Exception e) {
                     	mutationCommitTime = System.currentTimeMillis() - startTime;
                         serverTimestamp = ServerUtil.parseServerTimestamp(e);
@@ -1426,8 +1436,9 @@ public class MutationState implements SQLCloseable {
         private int[] statementIndexes;
         @Nonnull private final RowTimestampColInfo rowTsColInfo;
         private byte[] onDupKeyBytes;
+        private long colValuesSize;
         
-        public RowMutationState(@Nonnull Map<PColumn,byte[]> columnValues, int statementIndex, @Nonnull RowTimestampColInfo rowTsColInfo,
+        public RowMutationState(@Nonnull Map<PColumn,byte[]> columnValues, long colValuesSize, int statementIndex, @Nonnull RowTimestampColInfo rowTsColInfo,
                 byte[] onDupKeyBytes) {
             checkNotNull(columnValues);
             checkNotNull(rowTsColInfo);
@@ -1435,6 +1446,12 @@ public class MutationState implements SQLCloseable {
             this.statementIndexes = new int[] {statementIndex};
             this.rowTsColInfo = rowTsColInfo;
             this.onDupKeyBytes = onDupKeyBytes;
+            this.colValuesSize = colValuesSize;
+        }
+
+        public long calculateEstimatedSize() {
+            return colValuesSize + statementIndexes.length * SizedUtil.INT_SIZE + SizedUtil.LONG_SIZE
+                    + (onDupKeyBytes != null ? onDupKeyBytes.length : 0);
         }
 
         byte[] getOnDupKeyBytes() {
@@ -1453,7 +1470,16 @@ public class MutationState implements SQLCloseable {
             // If we already have a row and the new row has an ON DUPLICATE KEY clause
             // ignore the new values (as that's what the server will do).
             if (newRow.onDupKeyBytes == null) {
-                getColumnValues().putAll(newRow.getColumnValues());
+                // increment the column value size by the new row column value size
+                colValuesSize+=newRow.colValuesSize;
+                for (Map.Entry<PColumn,byte[]> entry : newRow.columnValues.entrySet()) {
+                    PColumn col = entry.getKey();
+                    byte[] oldValue = columnValues.put(col, entry.getValue());
+                    if (oldValue!=null) {
+                        // decrement column value size by the size of all column values that were replaced
+                        colValuesSize-=(col.getEstimatedSize() + oldValue.length);
+                    }
+                }
             }
             // Concatenate ON DUPLICATE KEY bytes to allow multiple
             // increments of the same row in the same commit batch.
@@ -1465,7 +1491,7 @@ public class MutationState implements SQLCloseable {
         RowTimestampColInfo getRowTimestampColInfo() {
             return rowTsColInfo;
         }
-       
+
     }
     
     public ReadMetricQueue getReadMetricQueue() {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/a6db0392/phoenix-core/src/main/java/org/apache/phoenix/util/KeyValueUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/KeyValueUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/KeyValueUtil.java
index 2dfe1b9..318c9d6 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/KeyValueUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/KeyValueUtil.java
@@ -30,14 +30,10 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValue.Type;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.hadoop.hbase.util.Pair;
 import org.apache.phoenix.execute.MutationState.RowMutationState;
 import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.hbase.index.util.KeyValueBuilder;
-import org.apache.phoenix.schema.PColumn;
-import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.TableRef;
-import org.apache.phoenix.schema.types.PArrayDataTypeEncoder;
 
 /**
  * 
@@ -187,47 +183,26 @@ public class KeyValueUtil {
     }
 
     /**
-     * Estimates the storage size of a row
+     * Estimates the size of rows stored in RowMutationState (in memory)
      * @param mutations map from table to row to RowMutationState
      * @return estimated row size
      */
     public static long
-            getEstimatedRowSize(TableRef tableRef, Map<ImmutableBytesPtr, RowMutationState> mutations) {
+            getEstimatedRowMutationSize(Map<TableRef, Map<ImmutableBytesPtr, RowMutationState>> tableMutationMap) {
         long size = 0;
-        PTable table = tableRef.getTable();
-        // iterate over rows
-        for (Entry<ImmutableBytesPtr, RowMutationState> rowEntry : mutations.entrySet()) {
-            int rowLength = rowEntry.getKey().getLength();
-            Map<PColumn, byte[]> colValueMap = rowEntry.getValue().getColumnValues();
-            switch (table.getImmutableStorageScheme()) {
-            case ONE_CELL_PER_COLUMN:
-                // iterate over columns
-                for (Entry<PColumn, byte[]> colValueEntry : colValueMap.entrySet()) {
-                    PColumn pColumn = colValueEntry.getKey();
-                    size +=
-                            KeyValue.getKeyValueDataStructureSize(rowLength,
-                                pColumn.getFamilyName().getBytes().length,
-                                pColumn.getColumnQualifierBytes().length,
-                                colValueEntry.getValue().length);
-                }
-                break;
-            case SINGLE_CELL_ARRAY_WITH_OFFSETS:
-                // we store all the column values in a single key value that contains all the
-                // column values followed by an offset array
-                size +=
-                        PArrayDataTypeEncoder.getEstimatedByteSize(table, rowLength,
-                            colValueMap);
-                break;
+        // iterate over table
+        for (Entry<TableRef, Map<ImmutableBytesPtr, RowMutationState>> tableEntry : tableMutationMap.entrySet()) {
+            // iterate over rows
+            for (Entry<ImmutableBytesPtr, RowMutationState> rowEntry : tableEntry.getValue().entrySet()) {
+                size += calculateRowMutationSize(rowEntry);
             }
-            // count the empty key value
-            Pair<byte[], byte[]> emptyKeyValueInfo =
-                    EncodedColumnsUtil.getEmptyKeyValueInfo(table);
-            size +=
-                    KeyValue.getKeyValueDataStructureSize(rowLength,
-                        SchemaUtil.getEmptyColumnFamilyPtr(table).getLength(),
-                        emptyKeyValueInfo.getFirst().length,
-                        emptyKeyValueInfo.getSecond().length);
         }
         return size;
     }
+
+    private static long calculateRowMutationSize(Entry<ImmutableBytesPtr, RowMutationState> rowEntry) {
+        int rowLength = rowEntry.getKey().getLength();
+        long colValuesLength = rowEntry.getValue().calculateEstimatedSize();
+        return (rowLength + colValuesLength);
+    }
 }


[13/16] phoenix git commit: PHOENIX-672 Add GRANT and REVOKE commands using HBase AccessController

Posted by ja...@apache.org.
PHOENIX-672 Add GRANT and REVOKE commands using HBase AccessController


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

Branch: refs/heads/4.x-HBase-1.2
Commit: bd72ce3d4bb8943a46b1431380d7db9a0dcbf339
Parents: 493a54f
Author: Karan Mehta <ka...@gmail.com>
Authored: Wed Nov 29 02:37:55 2017 +0000
Committer: James Taylor <jt...@salesforce.com>
Committed: Sat Dec 16 16:42:54 2017 -0800

----------------------------------------------------------------------
 .../phoenix/end2end/BasePermissionsIT.java      | 754 +++++++++++++++++++
 .../phoenix/end2end/ChangePermissionsIT.java    | 269 +++++++
 .../end2end/SystemTablePermissionsIT.java       | 226 +-----
 .../phoenix/end2end/TableDDLPermissionsIT.java  | 583 ++------------
 phoenix-core/src/main/antlr3/PhoenixSQL.g       |  30 +-
 .../coprocessor/PhoenixAccessController.java    |  29 +-
 .../phoenix/exception/SQLExceptionCode.java     |   1 +
 .../apache/phoenix/jdbc/PhoenixStatement.java   |  40 +-
 .../phoenix/parse/ChangePermsStatement.java     | 102 +++
 .../apache/phoenix/parse/ParseNodeFactory.java  |   7 +-
 .../query/ConnectionQueryServicesImpl.java      |  24 +-
 .../apache/phoenix/query/QueryConstants.java    |   1 +
 .../org/apache/phoenix/query/QueryServices.java |   2 -
 .../phoenix/query/QueryServicesOptions.java     |   8 +-
 .../apache/phoenix/schema/MetaDataClient.java   | 138 ++++
 .../schema/TablesNotInSyncException.java        |  22 +
 .../org/apache/phoenix/util/SchemaUtil.java     |  25 +-
 .../apache/phoenix/parse/QueryParserTest.java   |  46 +-
 18 files changed, 1544 insertions(+), 763 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/bd72ce3d/phoenix-core/src/it/java/org/apache/phoenix/end2end/BasePermissionsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BasePermissionsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BasePermissionsIT.java
new file mode 100644
index 0000000..9d7ef1b
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BasePermissionsIT.java
@@ -0,0 +1,754 @@
+/*
+ * 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 com.google.common.base.Joiner;
+import com.google.common.base.Throwables;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.AuthUtil;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.security.AccessDeniedException;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.security.access.AccessControlClient;
+import org.apache.hadoop.hbase.security.access.Permission;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
+import org.apache.phoenix.jdbc.PhoenixStatement;
+import org.apache.phoenix.query.BaseTest;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.apache.phoenix.util.QueryUtil;
+import org.junit.After;
+import org.junit.BeforeClass;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.io.IOException;
+import java.lang.reflect.UndeclaredThrowableException;
+import java.security.PrivilegedExceptionAction;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+@RunWith(Parameterized.class)
+public class BasePermissionsIT extends BaseTest {
+
+    private static final Log LOG = LogFactory.getLog(BasePermissionsIT.class);
+
+    static String SUPERUSER;
+
+    static HBaseTestingUtility testUtil;
+    static final Set<String> PHOENIX_SYSTEM_TABLES = new HashSet<>(Arrays.asList(
+            "SYSTEM.CATALOG", "SYSTEM.SEQUENCE", "SYSTEM.STATS", "SYSTEM.FUNCTION"));
+
+    static final Set<String> PHOENIX_SYSTEM_TABLES_IDENTIFIERS = new HashSet<>(Arrays.asList(
+            "SYSTEM.\"CATALOG\"", "SYSTEM.\"SEQUENCE\"", "SYSTEM.\"STATS\"", "SYSTEM.\"FUNCTION\""));
+
+    static final String SYSTEM_SEQUENCE_IDENTIFIER =
+            QueryConstants.SYSTEM_SCHEMA_NAME + "." + "\"" + PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_TABLE+ "\"";
+
+    static final Set<String> PHOENIX_NAMESPACE_MAPPED_SYSTEM_TABLES = new HashSet<>(Arrays.asList(
+            "SYSTEM:CATALOG", "SYSTEM:SEQUENCE", "SYSTEM:STATS", "SYSTEM:FUNCTION"));
+
+    // Create Multiple users so that we can use Hadoop UGI to run tasks as various users
+    // Permissions can be granted or revoke by superusers and admins only
+    // DON'T USE HADOOP UserGroupInformation class to create testing users since HBase misses some of its functionality
+    // Instead use org.apache.hadoop.hbase.security.User class for testing purposes.
+
+    // Super User has all the access
+    User superUser1 = null;
+    User superUser2 = null;
+
+    // Regular users are granted and revoked permissions as needed
+    User regularUser1 = null;
+    User regularUser2 = null;
+    User regularUser3 = null;
+    User regularUser4 = null;
+
+    // Group User is equivalent of regular user but inside a group
+    // Permissions can be granted to group should affect this user
+    static final String GROUP_SYSTEM_ACCESS = "group_system_access";
+    User groupUser = null;
+
+    // Unpriviledged User doesn't have any access and is denied for every action
+    User unprivilegedUser = null;
+
+    static final int NUM_RECORDS = 5;
+
+    boolean isNamespaceMapped;
+
+    public BasePermissionsIT(final boolean isNamespaceMapped) throws Exception {
+        this.isNamespaceMapped = isNamespaceMapped;
+    }
+
+    @BeforeClass
+    public static void doSetup() throws Exception {
+        SUPERUSER = System.getProperty("user.name");
+    }
+
+    void startNewMiniCluster() throws Exception {
+        startNewMiniCluster(new Configuration());
+    }
+    
+    void startNewMiniCluster(Configuration overrideConf) throws Exception{
+        if (null != testUtil) {
+            testUtil.shutdownMiniCluster();
+            testUtil = null;
+        }
+
+        testUtil = new HBaseTestingUtility();
+
+        Configuration config = testUtil.getConfiguration();
+        enablePhoenixHBaseAuthorization(config);
+        configureNamespacesOnServer(config);
+        configureRandomHMasterPort(config);
+        if (overrideConf != null) {
+            config.addResource(overrideConf);
+        }
+
+        testUtil.startMiniCluster(1);
+        initializeUsers(testUtil.getConfiguration());
+    }
+
+    private void initializeUsers(Configuration configuration) {
+
+        superUser1 = User.createUserForTesting(configuration, SUPERUSER, new String[0]);
+        superUser2 = User.createUserForTesting(configuration, "superUser2", new String[0]);
+
+        regularUser1 = User.createUserForTesting(configuration, "regularUser1", new String[0]);
+        regularUser2 = User.createUserForTesting(configuration, "regularUser2", new String[0]);
+        regularUser3 = User.createUserForTesting(configuration, "regularUser3", new String[0]);
+        regularUser4 = User.createUserForTesting(configuration, "regularUser4", new String[0]);
+
+        groupUser = User.createUserForTesting(testUtil.getConfiguration(), "groupUser", new String[] {GROUP_SYSTEM_ACCESS});
+
+        unprivilegedUser = User.createUserForTesting(configuration, "unprivilegedUser", new String[0]);
+    }
+
+    private void configureRandomHMasterPort(Configuration config) {
+        // Avoid multiple clusters trying to bind the master's info port (16010)
+        config.setInt(HConstants.MASTER_INFO_PORT, -1);
+    }
+
+    void enablePhoenixHBaseAuthorization(Configuration config) {
+        config.set("hbase.superuser", SUPERUSER + "," + "superUser2");
+        config.set("hbase.security.authorization", Boolean.TRUE.toString());
+        config.set("hbase.security.exec.permission.checks", Boolean.TRUE.toString());
+        config.set("hbase.coprocessor.master.classes",
+                "org.apache.hadoop.hbase.security.access.AccessController");
+        config.set("hbase.coprocessor.region.classes",
+                "org.apache.hadoop.hbase.security.access.AccessController");
+        config.set("hbase.coprocessor.regionserver.classes",
+                "org.apache.hadoop.hbase.security.access.AccessController");
+
+        config.set(QueryServices.PHOENIX_ACLS_ENABLED,"true");
+
+        config.set("hbase.regionserver.wal.codec", "org.apache.hadoop.hbase.regionserver.wal.IndexedWALEditCodec");
+    }
+
+    void configureNamespacesOnServer(Configuration conf) {
+        conf.set(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, Boolean.toString(isNamespaceMapped));
+    }
+
+    @Parameterized.Parameters(name = "isNamespaceMapped={0}") // name is used by failsafe as file name in reports
+    public static Collection<Boolean> data() {
+        return Arrays.asList(false, true);
+    }
+
+    @After
+    public void cleanup() throws Exception {
+        if (testUtil != null) {
+            testUtil.shutdownMiniCluster();
+            testUtil = null;
+        }
+    }
+
+    public static HBaseTestingUtility getUtility(){
+        return testUtil;
+    }
+
+    // Utility functions to grant permissions with HBase API
+    void grantPermissions(String toUser, Set<String> tablesToGrant, Permission.Action... actions) throws Throwable {
+        for (String table : tablesToGrant) {
+            AccessControlClient.grant(getUtility().getConnection(), TableName.valueOf(table), toUser, null, null,
+                    actions);
+        }
+    }
+
+    void grantPermissions(String toUser, String namespace, Permission.Action... actions) throws Throwable {
+        AccessControlClient.grant(getUtility().getConnection(), namespace, toUser, actions);
+    }
+
+    void grantPermissions(String groupEntry, Permission.Action... actions) throws IOException, Throwable {
+        AccessControlClient.grant(getUtility().getConnection(), groupEntry, actions);
+    }
+
+    // Utility functions to revoke permissions with HBase API
+    void revokeAll() throws Throwable {
+        AccessControlClient.revoke(getUtility().getConnection(), AuthUtil.toGroupEntry(GROUP_SYSTEM_ACCESS), Permission.Action.values() );
+        AccessControlClient.revoke(getUtility().getConnection(), regularUser1.getShortName(), Permission.Action.values() );
+        AccessControlClient.revoke(getUtility().getConnection(), unprivilegedUser.getShortName(), Permission.Action.values() );
+    }
+
+    Properties getClientProperties(String tenantId) {
+        Properties props = new Properties();
+        if(tenantId != null) {
+            props.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, tenantId);
+        }
+        props.setProperty(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, Boolean.toString(isNamespaceMapped));
+        return props;
+    }
+
+    public Connection getConnection() throws SQLException {
+        return getConnection(null);
+    }
+
+    public Connection getConnection(String tenantId) throws SQLException {
+        return DriverManager.getConnection(getUrl(), getClientProperties(tenantId));
+    }
+
+    protected static String getUrl() {
+        return "jdbc:phoenix:localhost:" + testUtil.getZkCluster().getClientPort() + ":/hbase";
+    }
+
+    static Set<String> getHBaseTables() throws IOException {
+        Set<String> tables = new HashSet<>();
+        for (TableName tn : testUtil.getHBaseAdmin().listTableNames()) {
+            tables.add(tn.getNameAsString());
+        }
+        return tables;
+    }
+
+    // UG Object
+    // 1. Instance of String --> represents GROUP name
+    // 2. Instance of User --> represents HBase user
+    AccessTestAction grantPermissions(final String actions, final Object ug,
+                                      final String tableOrSchemaList, final boolean isSchema) throws SQLException {
+        return grantPermissions(actions, ug, Collections.singleton(tableOrSchemaList), isSchema);
+    }
+
+    AccessTestAction grantPermissions(final String actions, final Object ug,
+                                      final Set<String> tableOrSchemaList, final boolean isSchema) throws SQLException {
+        return new AccessTestAction() {
+            @Override
+            public Object run() throws Exception {
+                try (Connection conn = getConnection(); Statement stmt = conn.createStatement();) {
+                    for(String tableOrSchema : tableOrSchemaList) {
+                        String grantStmtSQL = "GRANT '" + actions + "' ON " + (isSchema ? " SCHEMA " : " TABLE ") + tableOrSchema + " TO "
+                                + ((ug instanceof String) ? (" GROUP " + "'" + ug + "'") : ("'" + ((User)ug).getShortName() + "'"));
+                        LOG.info("Grant Permissions SQL: " + grantStmtSQL);
+                        assertFalse(stmt.execute(grantStmtSQL));
+                    }
+                }
+                return null;
+            }
+        };
+    }
+
+    AccessTestAction grantPermissions(final String actions, final User user) throws SQLException {
+        return new AccessTestAction() {
+            @Override
+            public Object run() throws Exception {
+                try (Connection conn = getConnection(); Statement stmt = conn.createStatement();) {
+                    String grantStmtSQL = "GRANT '" + actions + "' TO " + " '" + user.getShortName() + "'";
+                    LOG.info("Grant Permissions SQL: " + grantStmtSQL);
+                    assertFalse(stmt.execute(grantStmtSQL));
+                }
+                return null;
+            }
+        };
+    }
+
+    AccessTestAction revokePermissions(final Object ug,
+                                       final String tableOrSchemaList, final boolean isSchema) throws SQLException {
+        return revokePermissions(ug, Collections.singleton(tableOrSchemaList), isSchema);
+    }
+
+    AccessTestAction revokePermissions(final Object ug,
+                                       final Set<String> tableOrSchemaList, final boolean isSchema) throws SQLException {
+        return new AccessTestAction() {
+            @Override
+            public Object run() throws Exception {
+                try (Connection conn = getConnection(); Statement stmt = conn.createStatement();) {
+                    for(String tableOrSchema : tableOrSchemaList) {
+                        String revokeStmtSQL = "REVOKE ON " + (isSchema ? " SCHEMA " : " TABLE ") + tableOrSchema + " FROM "
+                                + ((ug instanceof String) ? (" GROUP " + "'" + ug + "'") : ("'" + ((User)ug).getShortName() + "'"));
+                        LOG.info("Revoke Permissions SQL: " + revokeStmtSQL);
+                        assertFalse(stmt.execute(revokeStmtSQL));
+                    }
+                }
+                return null;
+            }
+        };
+    }
+
+    AccessTestAction revokePermissions(final Object ug) throws SQLException {
+        return new AccessTestAction() {
+            @Override
+            public Object run() throws Exception {
+                try (Connection conn = getConnection(); Statement stmt = conn.createStatement();) {
+                    String revokeStmtSQL = "REVOKE FROM " +
+                            ((ug instanceof String) ? (" GROUP " + "'" + ug + "'") : ("'" + ((User)ug).getShortName() + "'"));
+                    LOG.info("Revoke Permissions SQL: " + revokeStmtSQL);
+                    assertFalse(stmt.execute(revokeStmtSQL));
+                }
+                return null;
+            }
+        };
+    }
+
+    // Attempts to get a Phoenix Connection
+    // New connections could create SYSTEM tables if appropriate perms are granted
+    AccessTestAction getConnectionAction() throws SQLException {
+        return new AccessTestAction() {
+            @Override
+            public Object run() throws Exception {
+                try (Connection conn = getConnection();) {
+                }
+                return null;
+            }
+        };
+    }
+
+    AccessTestAction createSchema(final String schemaName) throws SQLException {
+        return new AccessTestAction() {
+            @Override
+            public Object run() throws Exception {
+                if (isNamespaceMapped) {
+                    try (Connection conn = getConnection(); Statement stmt = conn.createStatement();) {
+                        assertFalse(stmt.execute("CREATE SCHEMA " + schemaName));
+                    }
+                }
+                return null;
+            }
+        };
+    }
+
+    AccessTestAction dropSchema(final String schemaName) throws SQLException {
+        return new AccessTestAction() {
+            @Override
+            public Object run() throws Exception {
+                if (isNamespaceMapped) {
+                    try (Connection conn = getConnection(); Statement stmt = conn.createStatement();) {
+                        assertFalse(stmt.execute("DROP SCHEMA " + schemaName));
+                    }
+                }
+                return null;
+            }
+        };
+    }
+
+    AccessTestAction createTable(final String tableName) throws SQLException {
+        return new AccessTestAction() {
+            @Override
+            public Object run() throws Exception {
+                try (Connection conn = getConnection(); Statement stmt = conn.createStatement();) {
+                    assertFalse(stmt.execute("CREATE TABLE " + tableName + "(pk INTEGER not null primary key, data VARCHAR, val integer)"));
+                    try (PreparedStatement pstmt = conn.prepareStatement("UPSERT INTO " + tableName + " values(?, ?, ?)")) {
+                        for (int i = 0; i < NUM_RECORDS; i++) {
+                            pstmt.setInt(1, i);
+                            pstmt.setString(2, Integer.toString(i));
+                            pstmt.setInt(3, i);
+                            assertEquals(1, pstmt.executeUpdate());
+                        }
+                    }
+                    conn.commit();
+                }
+                return null;
+            }
+        };
+    }
+
+    AccessTestAction createMultiTenantTable(final String tableName) throws SQLException {
+        return new AccessTestAction() {
+            @Override
+            public Object run() throws Exception {
+                try (Connection conn = getConnection(); Statement stmt = conn.createStatement();) {
+                    assertFalse(stmt.execute("CREATE TABLE " + tableName
+                            + "(ORG_ID VARCHAR NOT NULL, PREFIX CHAR(3) NOT NULL, DATA VARCHAR, VAL INTEGER CONSTRAINT PK PRIMARY KEY (ORG_ID, PREFIX))  MULTI_TENANT=TRUE"));
+                    try (PreparedStatement pstmt = conn.prepareStatement("UPSERT INTO " + tableName + " values(?, ?, ?, ?)")) {
+                        for (int i = 0; i < NUM_RECORDS; i++) {
+                            pstmt.setString(1, "o" + i);
+                            pstmt.setString(2, "pr" + i);
+                            pstmt.setString(3, Integer.toString(i));
+                            pstmt.setInt(4, i);
+                            assertEquals(1, pstmt.executeUpdate());
+                        }
+                    }
+                    conn.commit();
+                }
+                return null;
+            }
+        };
+    }
+
+    AccessTestAction dropTable(final String tableName) throws SQLException {
+        return new AccessTestAction() {
+            @Override
+            public Object run() throws Exception {
+                try (Connection conn = getConnection(); Statement stmt = conn.createStatement();) {
+                    assertFalse(stmt.execute("DROP TABLE IF EXISTS " + tableName));
+                }
+                return null;
+            }
+        };
+
+    }
+
+    // Attempts to read given table without verifying data
+    // AccessDeniedException is only triggered when ResultSet#next() method is called
+    // The first call triggers HBase Scan object
+    // The Statement#executeQuery() method returns an iterator and doesn't interact with HBase API at all
+    AccessTestAction readTableWithoutVerification(final String tableName) throws SQLException {
+        return new AccessTestAction() {
+            @Override
+            public Object run() throws Exception {
+                try (Connection conn = getConnection(); Statement stmt = conn.createStatement()) {
+                    ResultSet rs = stmt.executeQuery("SELECT * FROM " + tableName);
+                    assertNotNull(rs);
+                    while (rs.next()) {
+                    }
+                }
+                return null;
+            }
+        };
+    }
+
+    AccessTestAction readTable(final String tableName) throws SQLException {
+        return readTable(tableName,null);
+    }
+
+    AccessTestAction readTable(final String tableName, final String indexName) throws SQLException {
+        return new AccessTestAction() {
+            @Override
+            public Object run() throws Exception {
+                try (Connection conn = getConnection(); Statement stmt = conn.createStatement()) {
+                    String readTableSQL = "SELECT "+(indexName!=null?"/*+ INDEX("+tableName+" "+indexName+")*/":"")+" pk, data, val FROM " + tableName +" where data >= '0'";
+                    ResultSet rs = stmt.executeQuery(readTableSQL);
+                    assertNotNull(rs);
+                    int i = 0;
+                    while (rs.next()) {
+                        assertEquals(i, rs.getInt(1));
+                        assertEquals(Integer.toString(i), rs.getString(2));
+                        assertEquals(i, rs.getInt(3));
+                        i++;
+                    }
+                    assertEquals(NUM_RECORDS, i);
+                }
+                return null;
+            }
+        };
+    }
+
+    AccessTestAction readMultiTenantTableWithoutIndex(final String tableName) throws SQLException {
+        return readMultiTenantTableWithoutIndex(tableName, null);
+    }
+
+    AccessTestAction readMultiTenantTableWithoutIndex(final String tableName, final String tenantId) throws SQLException {
+        return new AccessTestAction() {
+            @Override
+            public Object run() throws Exception {
+                try (Connection conn = getConnection(tenantId); Statement stmt = conn.createStatement()) {
+                    // Accessing all the data from the table avoids the use of index
+                    String readTableSQL = "SELECT data, val FROM " + tableName;
+                    ResultSet rs = stmt.executeQuery(readTableSQL);
+                    assertNotNull(rs);
+                    int i = 0;
+                    String explainPlan = Joiner.on(" ").join(((PhoenixStatement)stmt).getQueryPlan().getExplainPlan().getPlanSteps());
+                    rs = stmt.executeQuery(readTableSQL);
+                    if(tenantId != null) {
+                        rs.next();
+                        assertFalse(explainPlan.contains("_IDX_"));
+                        assertEquals(((PhoenixConnection)conn).getTenantId().toString(), tenantId);
+                        // For tenant ID "o3", the value in table will be 3
+                        assertEquals(Character.toString(tenantId.charAt(1)), rs.getString(1));
+                        // Only 1 record is inserted per Tenant
+                        assertFalse(rs.next());
+                    } else {
+                        while(rs.next()) {
+                            assertEquals(Integer.toString(i), rs.getString(1));
+                            assertEquals(i, rs.getInt(2));
+                            i++;
+                        }
+                        assertEquals(NUM_RECORDS, i);
+                    }
+                }
+                return null;
+            }
+        };
+    }
+
+    AccessTestAction readMultiTenantTableWithIndex(final String tableName) throws SQLException {
+        return readMultiTenantTableWithIndex(tableName, null);
+    }
+
+    AccessTestAction readMultiTenantTableWithIndex(final String tableName, final String tenantId) throws SQLException {
+        return new AccessTestAction() {
+            @Override
+            public Object run() throws Exception {
+                try (Connection conn = getConnection(tenantId); Statement stmt = conn.createStatement()) {
+                    // Accessing only the 'data' from the table uses index since index tables are built on 'data' column
+                    String readTableSQL = "SELECT data FROM " + tableName;
+                    ResultSet rs = stmt.executeQuery(readTableSQL);
+                    assertNotNull(rs);
+                    int i = 0;
+                    String explainPlan = Joiner.on(" ").join(((PhoenixStatement) stmt).getQueryPlan().getExplainPlan().getPlanSteps());
+                    assertTrue(explainPlan.contains("_IDX_"));
+                    rs = stmt.executeQuery(readTableSQL);
+                    if (tenantId != null) {
+                        rs.next();
+                        assertEquals(((PhoenixConnection) conn).getTenantId().toString(), tenantId);
+                        // For tenant ID "o3", the value in table will be 3
+                        assertEquals(Character.toString(tenantId.charAt(1)), rs.getString(1));
+                        // Only 1 record is inserted per Tenant
+                        assertFalse(rs.next());
+                    } else {
+                        while (rs.next()) {
+                            assertEquals(Integer.toString(i), rs.getString(1));
+                            i++;
+                        }
+                        assertEquals(NUM_RECORDS, i);
+                    }
+                }
+                return null;
+            }
+        };
+    }
+
+    AccessTestAction addProperties(final String tableName, final String property, final String value)
+            throws SQLException {
+        return new AccessTestAction() {
+            @Override
+            public Object run() throws Exception {
+                try (Connection conn = getConnection(); Statement stmt = conn.createStatement();) {
+                    assertFalse(stmt.execute("ALTER TABLE " + tableName + " SET " + property + "=" + value));
+                }
+                return null;
+            }
+        };
+    }
+
+    AccessTestAction addColumn(final String tableName, final String columnName) throws SQLException {
+        return new AccessTestAction() {
+            @Override
+            public Object run() throws Exception {
+                try (Connection conn = getConnection(); Statement stmt = conn.createStatement();) {
+                    assertFalse(stmt.execute("ALTER TABLE " + tableName + " ADD "+columnName+" varchar"));
+                }
+                return null;
+            }
+        };
+    }
+
+    AccessTestAction dropColumn(final String tableName, final String columnName) throws SQLException {
+        return new AccessTestAction() {
+            @Override
+            public Object run() throws Exception {
+                try (Connection conn = getConnection(); Statement stmt = conn.createStatement();) {
+                    assertFalse(stmt.execute("ALTER TABLE " + tableName + " DROP COLUMN "+columnName));
+                }
+                return null;
+            }
+        };
+    }
+
+    AccessTestAction createIndex(final String indexName, final String dataTable) throws SQLException {
+        return createIndex(indexName, dataTable, null);
+    }
+
+    AccessTestAction createIndex(final String indexName, final String dataTable, final String tenantId) throws SQLException {
+        return new AccessTestAction() {
+            @Override
+            public Object run() throws Exception {
+
+                try (Connection conn = getConnection(tenantId); Statement stmt = conn.createStatement();) {
+                    assertFalse(stmt.execute("CREATE INDEX " + indexName + " on " + dataTable + "(data)"));
+                }
+                return null;
+            }
+        };
+    }
+
+    AccessTestAction createLocalIndex(final String indexName, final String dataTable) throws SQLException {
+        return new AccessTestAction() {
+            @Override
+            public Object run() throws Exception {
+
+                try (Connection conn = getConnection(); Statement stmt = conn.createStatement();) {
+                    assertFalse(stmt.execute("CREATE LOCAL INDEX " + indexName + " on " + dataTable + "(data)"));
+                }
+                return null;
+            }
+        };
+    }
+
+    AccessTestAction dropIndex(final String indexName, final String dataTable) throws SQLException {
+        return new AccessTestAction() {
+            @Override
+            public Object run() throws Exception {
+                try (Connection conn = getConnection(); Statement stmt = conn.createStatement();) {
+                    assertFalse(stmt.execute("DROP INDEX " + indexName + " on " + dataTable));
+                }
+                return null;
+            }
+        };
+    }
+
+    AccessTestAction rebuildIndex(final String indexName, final String dataTable) throws SQLException {
+        return new AccessTestAction() {
+            @Override
+            public Object run() throws Exception {
+                try (Connection conn = getConnection(); Statement stmt = conn.createStatement();) {
+                    assertFalse(stmt.execute("ALTER INDEX " + indexName + " on " + dataTable + " DISABLE"));
+                    assertFalse(stmt.execute("ALTER INDEX " + indexName + " on " + dataTable + " REBUILD"));
+                }
+                return null;
+            }
+        };
+    }
+
+    AccessTestAction dropView(final String viewName) throws SQLException {
+        return new AccessTestAction() {
+            @Override
+            public Object run() throws Exception {
+                try (Connection conn = getConnection(); Statement stmt = conn.createStatement();) {
+                    assertFalse(stmt.execute("DROP VIEW " + viewName));
+                }
+                return null;
+            }
+        };
+    }
+
+    AccessTestAction createView(final String viewName, final String dataTable) throws SQLException {
+        return createView(viewName, dataTable, null);
+    }
+
+    AccessTestAction createView(final String viewName, final String dataTable, final String tenantId) throws SQLException {
+        return new AccessTestAction() {
+            @Override
+            public Object run() throws Exception {
+                try (Connection conn = getConnection(tenantId); Statement stmt = conn.createStatement();) {
+                    String viewStmtSQL = "CREATE VIEW " + viewName + " AS SELECT * FROM " + dataTable;
+                    assertFalse(stmt.execute(viewStmtSQL));
+                }
+                return null;
+            }
+        };
+    }
+
+    static interface AccessTestAction extends PrivilegedExceptionAction<Object> { }
+
+    /** This fails only in case of ADE or empty list for any of the users. */
+    void verifyAllowed(AccessTestAction action, User... users) throws Exception {
+        if(users.length == 0) {
+            throw new Exception("Action needs at least one user to run");
+        }
+        for (User user : users) {
+            verifyAllowed(user, action);
+        }
+    }
+
+    void verifyAllowed(User user, TableDDLPermissionsIT.AccessTestAction... actions) throws Exception {
+        for (TableDDLPermissionsIT.AccessTestAction action : actions) {
+            try {
+                Object obj = user.runAs(action);
+                if (obj != null && obj instanceof List<?>) {
+                    List<?> results = (List<?>) obj;
+                    if (results != null && results.isEmpty()) {
+                        fail("Empty non null results from action for user '" + user.getShortName() + "'");
+                    }
+                }
+            } catch (AccessDeniedException ade) {
+                fail("Expected action to pass for user '" + user.getShortName() + "' but was denied");
+            }
+        }
+    }
+
+    /** This passes only if desired exception is caught for all users. */
+    <T> void verifyDenied(AccessTestAction action, Class<T> exception, User... users) throws Exception {
+        if(users.length == 0) {
+            throw new Exception("Action needs at least one user to run");
+        }
+        for (User user : users) {
+            verifyDenied(user, exception, action);
+        }
+    }
+
+    /** This passes only if desired exception is caught for all users. */
+    <T> void verifyDenied(User user, Class<T> exception, TableDDLPermissionsIT.AccessTestAction... actions) throws Exception {
+        for (TableDDLPermissionsIT.AccessTestAction action : actions) {
+            try {
+                user.runAs(action);
+                fail("Expected exception was not thrown for user '" + user.getShortName() + "'");
+            } catch (IOException e) {
+                fail("Expected exception was not thrown for user '" + user.getShortName() + "'");
+            } catch (UndeclaredThrowableException ute) {
+                Throwable ex = ute.getUndeclaredThrowable();
+
+                // HBase AccessDeniedException(ADE) is handled in different ways in different parts of code
+                // 1. Wrap HBase ADE in PhoenixIOException (Mostly for create, delete statements)
+                // 2. Wrap HBase ADE in ExecutionException (Mostly for scans)
+                // 3. Directly throwing HBase ADE or custom msg with HBase ADE
+                // Thus we iterate over the chain of throwables and find ADE
+                for(Throwable throwable : Throwables.getCausalChain(ex)) {
+                    if(exception.equals(throwable.getClass())) {
+                        if(throwable instanceof AccessDeniedException) {
+                            validateAccessDeniedException((AccessDeniedException) throwable);
+                        }
+                        return;
+                    }
+                }
+
+            } catch(RuntimeException ex) {
+                // This can occur while accessing tabledescriptors from client by the unprivileged user
+                if (ex.getCause() instanceof AccessDeniedException) {
+                    // expected result
+                    validateAccessDeniedException((AccessDeniedException) ex.getCause());
+                    return;
+                }
+            }
+            fail("Expected exception was not thrown for user '" + user.getShortName() + "'");
+        }
+    }
+
+    void validateAccessDeniedException(AccessDeniedException ade) {
+        String msg = ade.getMessage();
+        assertTrue("Exception contained unexpected message: '" + msg + "'",
+                !msg.contains("is not the scanner owner"));
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bd72ce3d/phoenix-core/src/it/java/org/apache/phoenix/end2end/ChangePermissionsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ChangePermissionsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ChangePermissionsIT.java
new file mode 100644
index 0000000..c023440
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ChangePermissionsIT.java
@@ -0,0 +1,269 @@
+/*
+ * 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.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.security.AccessDeniedException;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.schema.TableNotFoundException;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test that verifies a user can read Phoenix tables with a minimal set of permissions.
+ */
+@Category(NeedsOwnMiniClusterTest.class)
+public class ChangePermissionsIT extends BasePermissionsIT {
+
+    private static final Log LOG = LogFactory.getLog(ChangePermissionsIT.class);
+
+    private static final String SCHEMA_NAME = "CHANGEPERMSSCHEMA";
+    private static final String TABLE_NAME =
+            ChangePermissionsIT.class.getSimpleName().toUpperCase();
+    private static final String FULL_TABLE_NAME = SCHEMA_NAME + "." + TABLE_NAME;
+    private static final String IDX1_TABLE_NAME = TABLE_NAME + "_IDX1";
+    private static final String IDX2_TABLE_NAME = TABLE_NAME + "_IDX2";
+    private static final String IDX3_TABLE_NAME = TABLE_NAME + "_IDX3";
+    private static final String LOCAL_IDX1_TABLE_NAME = TABLE_NAME + "_LIDX1";
+    private static final String VIEW1_TABLE_NAME = TABLE_NAME + "_V1";
+    private static final String VIEW2_TABLE_NAME = TABLE_NAME + "_V2";
+
+    public ChangePermissionsIT(boolean isNamespaceMapped) throws Exception {
+        super(isNamespaceMapped);
+    }
+
+    private void grantSystemTableAccess(User superUser, User... users) throws Exception {
+        for(User user : users) {
+            if(isNamespaceMapped) {
+                verifyAllowed(grantPermissions("RX", user, QueryConstants.SYSTEM_SCHEMA_NAME, true), superUser);
+            } else {
+                verifyAllowed(grantPermissions("RX", user, PHOENIX_SYSTEM_TABLES_IDENTIFIERS, false), superUser);
+            }
+            verifyAllowed(grantPermissions("W", user, SYSTEM_SEQUENCE_IDENTIFIER, false), superUser);
+        }
+    }
+
+    private void revokeSystemTableAccess(User superUser, User... users) throws Exception {
+        for(User user : users) {
+            if(isNamespaceMapped) {
+                verifyAllowed(revokePermissions(user, QueryConstants.SYSTEM_SCHEMA_NAME, true), superUser);
+            } else {
+                verifyAllowed(revokePermissions(user, PHOENIX_SYSTEM_TABLES_IDENTIFIERS, false), superUser);
+            }
+            verifyAllowed(revokePermissions(user, SYSTEM_SEQUENCE_IDENTIFIER, false), superUser);
+        }
+    }
+
+    /**
+     * Verify that READ and EXECUTE permissions are required on SYSTEM tables to get a Phoenix Connection
+     * Tests grant revoke permissions per user 1. if NS enabled -> on namespace 2. If NS disabled -> on tables
+     */
+    @Test
+    public void testRXPermsReqdForPhoenixConn() throws Exception {
+
+        startNewMiniCluster();
+
+        if(isNamespaceMapped) {
+            // NS is enabled, CQSI tries creating SYSCAT, we get NamespaceNotFoundException exception for "SYSTEM" NS
+            // We create custom ADE and throw it (and ignore NamespaceNotFoundException)
+            // This is because we didn't had CREATE perms to create "SYSTEM" NS
+            verifyDenied(getConnectionAction(), AccessDeniedException.class, regularUser1);
+        } else {
+            // NS is disabled, CQSI tries creating SYSCAT, Two cases here
+            // 1. First client ever --> Gets ADE, runs client server compatibility check again and gets TableNotFoundException since SYSCAT doesn't exist
+            // 2. Any other client --> Gets ADE, runs client server compatibility check again and gets AccessDeniedException since it doesn't have EXEC perms
+            verifyDenied(getConnectionAction(), TableNotFoundException.class, regularUser1);
+        }
+
+        // Phoenix Client caches connection per user
+        // If we grant permissions, get a connection and then revoke it, we can still get the cached connection
+        // However it will fail for other read queries
+        // Thus this test grants and revokes for 2 users, so that both functionality can be tested.
+        grantSystemTableAccess(superUser1, regularUser1, regularUser2);
+        verifyAllowed(getConnectionAction(), regularUser1);
+        revokeSystemTableAccess(superUser1, regularUser2);
+        verifyDenied(getConnectionAction(), AccessDeniedException.class, regularUser2);
+    }
+
+    /**
+     * Superuser grants admin perms to user1, who will in-turn grant admin perms to user2
+     * Not affected with namespace props
+     * Tests grant revoke permissions on per user global level
+     */
+    @Test
+    public void testSuperUserCanChangePerms() throws Exception {
+
+        startNewMiniCluster();
+
+        // Grant System Table access to all users, else they can't create a Phoenix connection
+        grantSystemTableAccess(superUser1, regularUser1, regularUser2, unprivilegedUser);
+
+        verifyAllowed(grantPermissions("A", regularUser1), superUser1);
+
+        verifyAllowed(readTableWithoutVerification(PhoenixDatabaseMetaData.SYSTEM_CATALOG), regularUser1);
+        verifyAllowed(grantPermissions("A", regularUser2), regularUser1);
+
+        verifyAllowed(revokePermissions(regularUser1), superUser1);
+        verifyDenied(grantPermissions("A", regularUser3), AccessDeniedException.class, regularUser1);
+
+        // Don't grant ADMIN perms to unprivilegedUser, thus unprivilegedUser is unable to control other permissions.
+        verifyAllowed(getConnectionAction(), unprivilegedUser);
+        verifyDenied(grantPermissions("ARX", regularUser4), AccessDeniedException.class, unprivilegedUser);
+    }
+
+    /**
+     * Test to verify READ permissions on table, indexes and views
+     * Tests automatic grant revoke of permissions per user on a table
+     */
+    @Test
+    public void testReadPermsOnTableIndexAndView() throws Exception {
+
+        startNewMiniCluster();
+
+        grantSystemTableAccess(superUser1, regularUser1, regularUser2, unprivilegedUser);
+
+        // Create new schema and grant CREATE permissions to a user
+        if(isNamespaceMapped) {
+            verifyAllowed(createSchema(SCHEMA_NAME), superUser1);
+            verifyAllowed(grantPermissions("C", regularUser1, SCHEMA_NAME, true), superUser1);
+        } else {
+            verifyAllowed(grantPermissions("C", regularUser1, "\"" + QueryConstants.HBASE_DEFAULT_SCHEMA_NAME + "\"", true), superUser1);
+        }
+
+        // Create new table. Create indexes, views and view indexes on top of it. Verify the contents by querying it
+        verifyAllowed(createTable(FULL_TABLE_NAME), regularUser1);
+        verifyAllowed(readTable(FULL_TABLE_NAME), regularUser1);
+        verifyAllowed(createIndex(IDX1_TABLE_NAME, FULL_TABLE_NAME), regularUser1);
+        verifyAllowed(createIndex(IDX2_TABLE_NAME, FULL_TABLE_NAME), regularUser1);
+        verifyAllowed(createLocalIndex(LOCAL_IDX1_TABLE_NAME, FULL_TABLE_NAME), regularUser1);
+        verifyAllowed(createView(VIEW1_TABLE_NAME, FULL_TABLE_NAME), regularUser1);
+        verifyAllowed(createIndex(IDX3_TABLE_NAME, VIEW1_TABLE_NAME), regularUser1);
+
+        // RegularUser2 doesn't have any permissions. It can get a PhoenixConnection
+        // However it cannot query table, indexes or views without READ perms
+        verifyAllowed(getConnectionAction(), regularUser2);
+        verifyDenied(readTable(FULL_TABLE_NAME), AccessDeniedException.class, regularUser2);
+        verifyDenied(readTable(FULL_TABLE_NAME, IDX1_TABLE_NAME), AccessDeniedException.class, regularUser2);
+        verifyDenied(readTable(VIEW1_TABLE_NAME), AccessDeniedException.class, regularUser2);
+        verifyDenied(readTableWithoutVerification(SCHEMA_NAME + "." + IDX1_TABLE_NAME), AccessDeniedException.class, regularUser2);
+
+        // Grant READ permissions to RegularUser2 on the table
+        // Permissions should propagate automatically to relevant physical tables such as global index and view index.
+        verifyAllowed(grantPermissions("R", regularUser2, FULL_TABLE_NAME, false), regularUser1);
+        // Granting permissions directly to index tables should fail
+        verifyDenied(grantPermissions("W", regularUser2, SCHEMA_NAME + "." + IDX1_TABLE_NAME, false), AccessDeniedException.class, regularUser1);
+        // Granting permissions directly to views should fail. We expect TableNotFoundException since VIEWS are not physical tables
+        verifyDenied(grantPermissions("W", regularUser2, SCHEMA_NAME + "." + VIEW1_TABLE_NAME, false), TableNotFoundException.class, regularUser1);
+
+        // Verify that all other access are successful now
+        verifyAllowed(readTable(FULL_TABLE_NAME), regularUser2);
+        verifyAllowed(readTable(FULL_TABLE_NAME, IDX1_TABLE_NAME), regularUser2);
+        verifyAllowed(readTable(FULL_TABLE_NAME, IDX2_TABLE_NAME), regularUser2);
+        verifyAllowed(readTable(FULL_TABLE_NAME, LOCAL_IDX1_TABLE_NAME), regularUser2);
+        verifyAllowed(readTableWithoutVerification(SCHEMA_NAME + "." + IDX1_TABLE_NAME), regularUser2);
+        verifyAllowed(readTable(VIEW1_TABLE_NAME), regularUser2);
+        verifyAllowed(readMultiTenantTableWithIndex(VIEW1_TABLE_NAME), regularUser2);
+
+        // Revoke READ permissions to RegularUser2 on the table
+        // Permissions should propagate automatically to relevant physical tables such as global index and view index.
+        verifyAllowed(revokePermissions(regularUser2, FULL_TABLE_NAME, false), regularUser1);
+        // READ query should fail now
+        verifyDenied(readTable(FULL_TABLE_NAME), AccessDeniedException.class, regularUser2);
+        verifyDenied(readTableWithoutVerification(SCHEMA_NAME + "." + IDX1_TABLE_NAME), AccessDeniedException.class, regularUser2);
+
+    }
+
+    /**
+     * Verifies permissions for users present inside a group
+     */
+    @Test
+    public void testGroupUserPerms() throws Exception {
+
+        startNewMiniCluster();
+
+        if(isNamespaceMapped) {
+            verifyAllowed(createSchema(SCHEMA_NAME), superUser1);
+        }
+        verifyAllowed(createTable(FULL_TABLE_NAME), superUser1);
+
+        // Grant SYSTEM table access to GROUP_SYSTEM_ACCESS and regularUser1
+        verifyAllowed(grantPermissions("RX", GROUP_SYSTEM_ACCESS, PHOENIX_SYSTEM_TABLES_IDENTIFIERS, false), superUser1);
+        grantSystemTableAccess(superUser1, regularUser1);
+
+        // Grant Permissions to Groups (Should be automatically applicable to all users inside it)
+        verifyAllowed(grantPermissions("AR", GROUP_SYSTEM_ACCESS, FULL_TABLE_NAME, false), superUser1);
+        verifyAllowed(readTable(FULL_TABLE_NAME), groupUser);
+
+        // GroupUser is an admin and can grant perms to other users
+        verifyDenied(readTable(FULL_TABLE_NAME), AccessDeniedException.class, regularUser1);
+        verifyAllowed(grantPermissions("R", regularUser1, FULL_TABLE_NAME, false), groupUser);
+        verifyAllowed(readTable(FULL_TABLE_NAME), regularUser1);
+
+        // Revoke the perms and try accessing data again
+        verifyAllowed(revokePermissions(GROUP_SYSTEM_ACCESS, FULL_TABLE_NAME, false), superUser1);
+        verifyDenied(readTable(FULL_TABLE_NAME), AccessDeniedException.class, groupUser);
+    }
+
+    /**
+     * Tests permissions for MultiTenant Tables and view index tables
+     */
+    @Test
+    public void testMultiTenantTables() throws Exception {
+
+        startNewMiniCluster();
+
+        grantSystemTableAccess(superUser1, regularUser1, regularUser2, regularUser3);
+
+        if(isNamespaceMapped) {
+            verifyAllowed(createSchema(SCHEMA_NAME), superUser1);
+            verifyAllowed(grantPermissions("C", regularUser1, SCHEMA_NAME, true), superUser1);
+        } else {
+            verifyAllowed(grantPermissions("C", regularUser1, "\"" + QueryConstants.HBASE_DEFAULT_SCHEMA_NAME + "\"", true), superUser1);
+        }
+
+        // Create MultiTenant Table (View Index Table should be automatically created)
+        // At this point, the index table doesn't contain any data
+        verifyAllowed(createMultiTenantTable(FULL_TABLE_NAME), regularUser1);
+
+        // RegularUser2 doesn't have access yet, RegularUser1 should have RWXCA on the table
+        verifyDenied(readMultiTenantTableWithoutIndex(FULL_TABLE_NAME), AccessDeniedException.class, regularUser2);
+
+        // Grant perms to base table (Should propagate to View Index as well)
+        verifyAllowed(grantPermissions("R", regularUser2, FULL_TABLE_NAME, false), regularUser1);
+        // Try reading full table
+        verifyAllowed(readMultiTenantTableWithoutIndex(FULL_TABLE_NAME), regularUser2);
+
+        // Create tenant specific views on the table using tenant specific Phoenix Connection
+        verifyAllowed(createView(VIEW1_TABLE_NAME, FULL_TABLE_NAME, "o1"), regularUser1);
+        verifyAllowed(createView(VIEW2_TABLE_NAME, FULL_TABLE_NAME, "o2"), regularUser1);
+
+        // Create indexes on those views using tenant specific Phoenix Connection
+        // It is not possible to create indexes on tenant specific views without tenant connection
+        verifyAllowed(createIndex(IDX1_TABLE_NAME, VIEW1_TABLE_NAME, "o1"), regularUser1);
+        verifyAllowed(createIndex(IDX2_TABLE_NAME, VIEW2_TABLE_NAME, "o2"), regularUser1);
+
+        // Read the tables as regularUser2, with and without the use of Index table
+        // If perms are propagated correctly, then both of them should work
+        // The test checks if the query plan uses the index table by searching for "_IDX_" string
+        // _IDX_ is the prefix used with base table name to derieve the name of view index table
+        verifyAllowed(readMultiTenantTableWithIndex(VIEW1_TABLE_NAME, "o1"), regularUser2);
+        verifyAllowed(readMultiTenantTableWithoutIndex(VIEW2_TABLE_NAME, "o2"), regularUser2);
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bd72ce3d/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemTablePermissionsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemTablePermissionsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemTablePermissionsIT.java
index 49202a4..bbe7114 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemTablePermissionsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemTablePermissionsIT.java
@@ -16,177 +16,60 @@
  */
 package org.apache.phoenix.end2end;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 
-import java.io.IOException;
 import java.security.PrivilegedExceptionAction;
-import java.sql.Connection;
-import java.sql.DriverManager;
-import java.sql.PreparedStatement;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.sql.Statement;
-import java.util.Arrays;
 import java.util.Collections;
-import java.util.HashSet;
-import java.util.Properties;
 import java.util.Set;
 
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.security.access.AccessControlClient;
 import org.apache.hadoop.hbase.security.access.Permission.Action;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.phoenix.query.QueryServices;
-import org.junit.After;
-import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
 /**
  * Test that verifies a user can read Phoenix tables with a minimal set of permissions.
+ * Uses HBase API directly to grant/revoke permissions
  */
 @Category(NeedsOwnMiniClusterTest.class)
-public class SystemTablePermissionsIT {
-    private static String SUPERUSER;
-
-    private static final Set<String> PHOENIX_SYSTEM_TABLES = new HashSet<>(Arrays.asList(
-            "SYSTEM.CATALOG", "SYSTEM.SEQUENCE", "SYSTEM.STATS", "SYSTEM.FUNCTION",
-                "SYSTEM.MUTEX"));
-    private static final Set<String> PHOENIX_NAMESPACE_MAPPED_SYSTEM_TABLES = new HashSet<>(
-            Arrays.asList("SYSTEM:CATALOG", "SYSTEM:SEQUENCE", "SYSTEM:STATS", "SYSTEM:FUNCTION",
-                "SYSTEM:MUTEX"));
+public class SystemTablePermissionsIT extends BasePermissionsIT {
 
     private static final String TABLE_NAME =
         SystemTablePermissionsIT.class.getSimpleName().toUpperCase();
-    private static final int NUM_RECORDS = 5;
-
-    private HBaseTestingUtility testUtil = null;
-    private Properties clientProperties = null;
 
-    @BeforeClass
-    public static void setup() throws Exception {
-        SUPERUSER = System.getProperty("user.name");
-    }
-
-    private static void setCommonConfigProperties(Configuration conf) {
-        conf.set("hbase.coprocessor.master.classes",
-            "org.apache.hadoop.hbase.security.access.AccessController");
-        conf.set("hbase.coprocessor.region.classes",
-            "org.apache.hadoop.hbase.security.access.AccessController");
-        conf.set("hbase.coprocessor.regionserver.classes",
-            "org.apache.hadoop.hbase.security.access.AccessController");
-        conf.set("hbase.security.exec.permission.checks", "true");
-        conf.set("hbase.security.authorization", "true");
-        conf.set("hbase.superuser", SUPERUSER);
-    }
-
-    @After
-    public void cleanup() throws Exception {
-        if (null != testUtil) {
-          testUtil.shutdownMiniCluster();
-          testUtil = null;
-        }
+    public SystemTablePermissionsIT(boolean isNamespaceMapped) throws Exception {
+        super(isNamespaceMapped);
     }
 
     @Test
-    public void testSystemTablePermissions() throws Exception {
-        testUtil = new HBaseTestingUtility();
-        clientProperties = new Properties();
-        Configuration conf = testUtil.getConfiguration();
-        setCommonConfigProperties(conf);
-        conf.set(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, "false");
-        clientProperties.setProperty(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, "false");
-        testUtil.startMiniCluster(1);
-        final UserGroupInformation superUser = UserGroupInformation.createUserForTesting(
-            SUPERUSER, new String[0]);
-        final UserGroupInformation regularUser = UserGroupInformation.createUserForTesting(
-            "user", new String[0]);
+    public void testSystemTablePermissions() throws Throwable {
 
-        superUser.doAs(new PrivilegedExceptionAction<Void>() {
-            @Override
-            public Void run() throws Exception {
-                createTable();
-                readTable();
-                return null;
-            }
-        });
+        startNewMiniCluster();
+
+        verifyAllowed(createTable(TABLE_NAME), superUser1);
+        verifyAllowed(readTable(TABLE_NAME), superUser1);
 
         Set<String> tables = getHBaseTables();
-        assertTrue("HBase tables do not include expected Phoenix tables: " + tables,
-            tables.containsAll(PHOENIX_SYSTEM_TABLES));
+        if(isNamespaceMapped) {
+            assertTrue("HBase tables do not include expected Phoenix tables: " + tables,
+                    tables.containsAll(PHOENIX_NAMESPACE_MAPPED_SYSTEM_TABLES));
+        } else {
+            assertTrue("HBase tables do not include expected Phoenix tables: " + tables,
+                    tables.containsAll(PHOENIX_SYSTEM_TABLES));
+        }
 
         // Grant permission to the system tables for the unprivileged user
-        superUser.doAs(new PrivilegedExceptionAction<Void>() {
+        superUser1.runAs(new PrivilegedExceptionAction<Void>() {
             @Override
             public Void run() throws Exception {
                 try {
-                    grantPermissions(regularUser.getShortUserName(), PHOENIX_SYSTEM_TABLES,
-                        Action.EXEC, Action.READ);
-                    grantPermissions(regularUser.getShortUserName(),
-                        Collections.singleton(TABLE_NAME), Action.READ);
-                } catch (Throwable e) {
-                    if (e instanceof Exception) {
-                        throw (Exception) e;
+                    if(isNamespaceMapped) {
+                        grantPermissions(regularUser1.getShortName(),
+                                PHOENIX_NAMESPACE_MAPPED_SYSTEM_TABLES, Action.EXEC, Action.READ);
                     } else {
-                        throw new Exception(e);
+                        grantPermissions(regularUser1.getShortName(), PHOENIX_SYSTEM_TABLES,
+                                Action.EXEC, Action.READ);
                     }
-                }
-                return null;
-            }
-        });
-
-        // Make sure that the unprivileged user can read the table
-        regularUser.doAs(new PrivilegedExceptionAction<Void>() {
-            @Override
-            public Void run() throws Exception {
-                // We expect this to not throw an error
-                readTable();
-                return null;
-            }
-        });
-    }
-
-    @Test
-    public void testNamespaceMappedSystemTables() throws Exception {
-        testUtil = new HBaseTestingUtility();
-        clientProperties = new Properties();
-        Configuration conf = testUtil.getConfiguration();
-        setCommonConfigProperties(conf);
-        testUtil.getConfiguration().set(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, "true");
-        clientProperties.setProperty(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, "true");
-        testUtil.startMiniCluster(1);
-        final UserGroupInformation superUser =
-            UserGroupInformation.createUserForTesting(SUPERUSER, new String[0]);
-        final UserGroupInformation regularUser =
-            UserGroupInformation.createUserForTesting("user", new String[0]);
-
-        superUser.doAs(new PrivilegedExceptionAction<Void>() {
-            @Override
-            public Void run() throws Exception {
-                createTable();
-                readTable();
-                return null;
-            }
-        });
-
-        Set<String> tables = getHBaseTables();
-        assertTrue("HBase tables do not include expected Phoenix tables: " + tables,
-            tables.containsAll(PHOENIX_NAMESPACE_MAPPED_SYSTEM_TABLES));
-
-        // Grant permission to the system tables for the unprivileged user
-        // An unprivileged user should only need to be able to Read and eXecute on them.
-        superUser.doAs(new PrivilegedExceptionAction<Void>() {
-            @Override
-            public Void run() throws Exception {
-                try {
-                    grantPermissions(regularUser.getShortUserName(),
-                        PHOENIX_NAMESPACE_MAPPED_SYSTEM_TABLES, Action.EXEC, Action.READ);
-                    grantPermissions(regularUser.getShortUserName(),
+                    grantPermissions(regularUser1.getShortName(),
                         Collections.singleton(TABLE_NAME), Action.READ);
                 } catch (Throwable e) {
                     if (e instanceof Exception) {
@@ -199,66 +82,7 @@ public class SystemTablePermissionsIT {
             }
         });
 
-        regularUser.doAs(new PrivilegedExceptionAction<Void>() {
-            @Override
-            public Void run() throws Exception {
-                // We expect this to not throw an error
-                readTable();
-                return null;
-            }
-        });
-    }
-
-    private String getJdbcUrl() {
-        return "jdbc:phoenix:localhost:" + testUtil.getZkCluster().getClientPort() + ":/hbase";
-    }
-
-    private void createTable() throws SQLException {
-        try (Connection conn = DriverManager.getConnection(getJdbcUrl(), clientProperties);
-            Statement stmt = conn.createStatement();) {
-            assertFalse(stmt.execute("DROP TABLE IF EXISTS " + TABLE_NAME));
-            assertFalse(stmt.execute("CREATE TABLE " + TABLE_NAME
-                + "(pk INTEGER not null primary key, data VARCHAR)"));
-            try (PreparedStatement pstmt = conn.prepareStatement("UPSERT INTO "
-                + TABLE_NAME + " values(?, ?)")) {
-                for (int i = 0; i < NUM_RECORDS; i++) {
-                    pstmt.setInt(1, i);
-                    pstmt.setString(2, Integer.toString(i));
-                    assertEquals(1, pstmt.executeUpdate());
-                }
-            }
-            conn.commit();
-        }
-    }
-
-    private void readTable() throws SQLException {
-        try (Connection conn = DriverManager.getConnection(getJdbcUrl(), clientProperties);
-            Statement stmt = conn.createStatement()) {
-            ResultSet rs = stmt.executeQuery("SELECT pk, data FROM " + TABLE_NAME);
-            assertNotNull(rs);
-            int i = 0;
-            while (rs.next()) {
-                assertEquals(i, rs.getInt(1));
-                assertEquals(Integer.toString(i), rs.getString(2));
-                i++;
-            }
-            assertEquals(NUM_RECORDS, i);
-        }
-    }
-
-    private void grantPermissions(String toUser, Set<String> tablesToGrant, Action... actions)
-            throws Throwable {
-          for (String table : tablesToGrant) {
-              AccessControlClient.grant(testUtil.getConnection(), TableName.valueOf(table), toUser,
-                  null, null, actions);
-          }
-    }
-
-    private Set<String> getHBaseTables() throws IOException {
-        Set<String> tables = new HashSet<>();
-        for (TableName tn : testUtil.getHBaseAdmin().listTableNames()) {
-            tables.add(tn.getNameAsString());
-        }
-        return tables;
+        // Make sure that the unprivileged user can now read the table
+        verifyAllowed(readTable(TABLE_NAME), regularUser1);
     }
 }


[07/16] phoenix git commit: PHOENIX-4386 Calculate the estimatedSize of MutationState using Map> mutations (addendum)

Posted by ja...@apache.org.
PHOENIX-4386 Calculate the estimatedSize of MutationState using Map<TableRef, Map<ImmutableBytesPtr,RowMutationState>> mutations (addendum)


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

Branch: refs/heads/4.x-HBase-1.2
Commit: eb0f744dc1f453e559cc8f9722122f6ed8daf739
Parents: a6db039
Author: Thomas D'Silva <td...@apache.org>
Authored: Tue Nov 21 03:13:53 2017 +0000
Committer: James Taylor <jt...@salesforce.com>
Committed: Sat Dec 16 16:42:54 2017 -0800

----------------------------------------------------------------------
 .../apache/phoenix/execute/PartialCommitIT.java |   5 +-
 .../apache/phoenix/compile/DeleteCompiler.java  |  11 +-
 .../apache/phoenix/compile/UpsertCompiler.java  |   7 +-
 .../apache/phoenix/execute/MutationState.java   | 127 ++++++++++++-------
 .../java/org/apache/phoenix/util/IndexUtil.java |   4 +-
 .../org/apache/phoenix/util/KeyValueUtil.java   |   5 +-
 6 files changed, 98 insertions(+), 61 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/eb0f744d/phoenix-core/src/it/java/org/apache/phoenix/execute/PartialCommitIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/execute/PartialCommitIT.java b/phoenix-core/src/it/java/org/apache/phoenix/execute/PartialCommitIT.java
index 10fd7f8..e5b57e3 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/execute/PartialCommitIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/execute/PartialCommitIT.java
@@ -33,7 +33,6 @@ import java.sql.SQLException;
 import java.sql.Statement;
 import java.util.Arrays;
 import java.util.Collection;
-import java.util.Collections;
 import java.util.Comparator;
 import java.util.List;
 import java.util.Map;
@@ -52,8 +51,8 @@ import org.apache.hadoop.hbase.coprocessor.SimpleRegionObserver;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.end2end.BaseOwnClusterIT;
+import org.apache.phoenix.execute.MutationState.MultiRowMutationState;
 import org.apache.phoenix.hbase.index.Indexer;
-import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.monitoring.GlobalMetric;
 import org.apache.phoenix.monitoring.MetricType;
@@ -285,7 +284,7 @@ public class PartialCommitIT extends BaseOwnClusterIT {
     private PhoenixConnection getConnectionWithTableOrderPreservingMutationState() throws SQLException {
         Connection con = driver.connect(url, new Properties());
         PhoenixConnection phxCon = new PhoenixConnection(con.unwrap(PhoenixConnection.class));
-        final Map<TableRef,Map<ImmutableBytesPtr,MutationState.RowMutationState>> mutations = Maps.newTreeMap(new TableRefComparator());
+        final Map<TableRef, MultiRowMutationState> mutations = Maps.newTreeMap(new TableRefComparator());
         // passing a null mutation state forces the connection.newMutationState() to be used to create the MutationState
         return new PhoenixConnection(phxCon, null) {
             @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/eb0f744d/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
----------------------------------------------------------------------
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 f9ca300..a06e2ca 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
@@ -26,7 +26,6 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
-import java.util.Map;
 import java.util.Set;
 
 import org.apache.hadoop.hbase.Cell;
@@ -43,6 +42,7 @@ import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.exception.SQLExceptionInfo;
 import org.apache.phoenix.execute.AggregatePlan;
 import org.apache.phoenix.execute.MutationState;
+import org.apache.phoenix.execute.MutationState.MultiRowMutationState;
 import org.apache.phoenix.execute.MutationState.RowMutationState;
 import org.apache.phoenix.filter.SkipScanFilter;
 import org.apache.phoenix.hbase.index.ValueGetter;
@@ -91,7 +91,6 @@ import org.apache.phoenix.util.ScanUtil;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
 import com.sun.istack.NotNull;
 
 public class DeleteCompiler {
@@ -121,14 +120,14 @@ public class DeleteCompiler {
         final int maxSize = services.getProps().getInt(QueryServices.MAX_MUTATION_SIZE_ATTRIB,QueryServicesOptions.DEFAULT_MAX_MUTATION_SIZE);
         final int maxSizeBytes = services.getProps().getInt(QueryServices.MAX_MUTATION_SIZE_BYTES_ATTRIB,QueryServicesOptions.DEFAULT_MAX_MUTATION_SIZE_BYTES);
         final int batchSize = Math.min(connection.getMutateBatchSize(), maxSize);
-        Map<ImmutableBytesPtr,RowMutationState> mutations = Maps.newHashMapWithExpectedSize(batchSize);
-        List<Map<ImmutableBytesPtr,RowMutationState>> indexMutations = null;
+        MultiRowMutationState mutations = new MultiRowMutationState(batchSize);
+        List<MultiRowMutationState> indexMutations = null;
         // If indexTableRef is set, we're deleting the rows from both the index table and
         // the data table through a single query to save executing an additional one.
         if (!otherTableRefs.isEmpty()) {
             indexMutations = Lists.newArrayListWithExpectedSize(otherTableRefs.size());
             for (int i = 0; i < otherTableRefs.size(); i++) {
-                indexMutations.add(Maps.<ImmutableBytesPtr,RowMutationState>newHashMapWithExpectedSize(batchSize));
+                indexMutations.add(new MultiRowMutationState(batchSize));
             }
         }
         List<PColumn> pkColumns = table.getPKColumns();
@@ -644,7 +643,7 @@ public class DeleteCompiler {
             // keys for our ranges
             ScanRanges ranges = context.getScanRanges();
             Iterator<KeyRange> iterator = ranges.getPointLookupKeyIterator();
-            Map<ImmutableBytesPtr,RowMutationState> mutation = Maps.newHashMapWithExpectedSize(ranges.getPointLookupCount());
+            MultiRowMutationState mutation = new MultiRowMutationState(ranges.getPointLookupCount());
             while (iterator.hasNext()) {
                 mutation.put(new ImmutableBytesPtr(iterator.next().getLowerRange()),
                         new RowMutationState(PRow.DELETE_MARKER, 0,

http://git-wip-us.apache.org/repos/asf/phoenix/blob/eb0f744d/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
index a51fd4c..a81a427 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
@@ -47,6 +47,7 @@ import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.exception.SQLExceptionInfo;
 import org.apache.phoenix.execute.AggregatePlan;
 import org.apache.phoenix.execute.MutationState;
+import org.apache.phoenix.execute.MutationState.MultiRowMutationState;
 import org.apache.phoenix.execute.MutationState.RowMutationState;
 import org.apache.phoenix.execute.MutationState.RowTimestampColInfo;
 import org.apache.phoenix.expression.Determinism;
@@ -116,7 +117,7 @@ import com.google.common.collect.Sets;
 public class UpsertCompiler {
 
     private static void setValues(byte[][] values, int[] pkSlotIndex, int[] columnIndexes,
-            PTable table, Map<ImmutableBytesPtr, RowMutationState> mutation,
+            PTable table, MultiRowMutationState mutation,
             PhoenixStatement statement, boolean useServerTimestamp, IndexMaintainer maintainer,
             byte[][] viewConstants, byte[] onDupKeyBytes, int numSplColumns) throws SQLException {
         long columnValueSize = 0;
@@ -197,7 +198,7 @@ public class UpsertCompiler {
             }
         }
         int rowCount = 0;
-        Map<ImmutableBytesPtr, RowMutationState> mutation = Maps.newHashMapWithExpectedSize(batchSize);
+        MultiRowMutationState mutation = new MultiRowMutationState(batchSize);
         PTable table = tableRef.getTable();
         IndexMaintainer indexMaintainer = null;
         byte[][] viewConstants = null;
@@ -1177,7 +1178,7 @@ public class UpsertCompiler {
                     throw new IllegalStateException();
                 }
             }
-            Map<ImmutableBytesPtr, RowMutationState> mutation = Maps.newHashMapWithExpectedSize(1);
+            MultiRowMutationState mutation = new MultiRowMutationState(1);
             IndexMaintainer indexMaintainer = null;
             byte[][] viewConstants = null;
             if (table.getIndexType() == IndexType.LOCAL) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/eb0f744d/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
index 15e905a..993438e 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
@@ -123,7 +123,7 @@ public class MutationState implements SQLCloseable {
     private final long batchSize;
     private final long batchSizeBytes;
     private long batchCount = 0L;
-    private final Map<TableRef, Map<ImmutableBytesPtr,RowMutationState>> mutations;
+    private final Map<TableRef, MultiRowMutationState> mutations;
     private final Set<String> uncommittedPhysicalNames = Sets.newHashSetWithExpectedSize(10);
 
     private long sizeOffset;
@@ -131,7 +131,7 @@ public class MutationState implements SQLCloseable {
     private long estimatedSize = 0;
     private int[] uncommittedStatementIndexes = EMPTY_STATEMENT_INDEX_ARRAY;
     private boolean isExternalTxContext = false;
-    private Map<TableRef, Map<ImmutableBytesPtr,RowMutationState>> txMutations = Collections.emptyMap();
+    private Map<TableRef, MultiRowMutationState> txMutations = Collections.emptyMap();
 
     final PhoenixTransactionContext phoenixTransactionContext;
 
@@ -159,12 +159,12 @@ public class MutationState implements SQLCloseable {
     }
 
     private MutationState(long maxSize, long maxSizeBytes, PhoenixConnection connection, boolean subTask, PhoenixTransactionContext txContext, long sizeOffset) {
-        this(maxSize, maxSizeBytes, connection, Maps.<TableRef, Map<ImmutableBytesPtr,RowMutationState>>newHashMapWithExpectedSize(5), subTask, txContext);
+        this(maxSize, maxSizeBytes, connection, Maps.<TableRef, MultiRowMutationState>newHashMapWithExpectedSize(5), subTask, txContext);
         this.sizeOffset = sizeOffset;
     }
 
     MutationState(long maxSize, long maxSizeBytes, PhoenixConnection connection,
-            Map<TableRef, Map<ImmutableBytesPtr, RowMutationState>> mutations,
+            Map<TableRef, MultiRowMutationState> mutations,
             boolean subTask, PhoenixTransactionContext txContext) {
         this.maxSize = maxSize;
         this.maxSizeBytes = maxSizeBytes;
@@ -189,7 +189,7 @@ public class MutationState implements SQLCloseable {
         }
     }
 
-    public MutationState(TableRef table, Map<ImmutableBytesPtr,RowMutationState> mutations, long sizeOffset, long maxSize, long maxSizeBytes, PhoenixConnection connection) throws SQLException {
+    public MutationState(TableRef table, MultiRowMutationState mutations, long sizeOffset, long maxSize, long maxSizeBytes, PhoenixConnection connection)  throws SQLException {
         this(maxSize, maxSizeBytes, connection, false, null, sizeOffset);
         if (!mutations.isEmpty()) {
             this.mutations.put(table, mutations);
@@ -350,7 +350,7 @@ public class MutationState implements SQLCloseable {
     }
 
     public static MutationState emptyMutationState(long maxSize, long maxSizeBytes, PhoenixConnection connection) {
-        MutationState state = new MutationState(maxSize, maxSizeBytes, connection, Collections.<TableRef, Map<ImmutableBytesPtr,RowMutationState>>emptyMap(), false, null);
+        MutationState state = new MutationState(maxSize, maxSizeBytes, connection, Collections.<TableRef, MultiRowMutationState>emptyMap(), false, null);
         state.sizeOffset = 0;
         return state;
     }
@@ -372,12 +372,12 @@ public class MutationState implements SQLCloseable {
         return sizeOffset + numRows;
     }
     
-    private void joinMutationState(TableRef tableRef, Map<ImmutableBytesPtr,RowMutationState> srcRows,
-            Map<TableRef, Map<ImmutableBytesPtr, RowMutationState>> dstMutations) {
+    private void joinMutationState(TableRef tableRef, MultiRowMutationState srcRows,
+            Map<TableRef, MultiRowMutationState> dstMutations) {
         PTable table = tableRef.getTable();
         boolean isIndex = table.getType() == PTableType.INDEX;
         boolean incrementRowCount = dstMutations == this.mutations;
-        Map<ImmutableBytesPtr,RowMutationState> existingRows = dstMutations.put(tableRef, srcRows);
+        MultiRowMutationState existingRows = dstMutations.put(tableRef, srcRows);
         if (existingRows != null) { // Rows for that table already exist
             // Loop through new rows and replace existing with new
             for (Map.Entry<ImmutableBytesPtr,RowMutationState> rowEntry : srcRows.entrySet()) {
@@ -389,8 +389,12 @@ public class MutationState implements SQLCloseable {
                         Map<PColumn,byte[]> newRow = rowEntry.getValue().getColumnValues();
                         // if new row is PRow.DELETE_MARKER, it means delete, and we don't need to merge it with existing row. 
                         if (newRow != PRow.DELETE_MARKER) {
+                            // decrement estimated size by the size of the old row
+                            estimatedSize-=existingRowMutationState.calculateEstimatedSize();
                             // Merge existing column values with new column values
                             existingRowMutationState.join(rowEntry.getValue());
+                            // increment estimated size by the size of the new row
+                            estimatedSize+=existingRowMutationState.calculateEstimatedSize();
                             // Now that the existing row has been merged with the new row, replace it back
                             // again (since it was merged with the new one above).
                             existingRows.put(rowEntry.getKey(), existingRowMutationState);
@@ -399,6 +403,8 @@ public class MutationState implements SQLCloseable {
                 } else {
                     if (incrementRowCount && !isIndex) { // Don't count index rows in row count
                         numRows++;
+                        // increment estimated size by the size of the new row
+                        estimatedSize += rowEntry.getValue().calculateEstimatedSize();
                     }
                 }
             }
@@ -406,22 +412,25 @@ public class MutationState implements SQLCloseable {
             dstMutations.put(tableRef, existingRows);
         } else {
             // Size new map at batch size as that's what it'll likely grow to.
-            Map<ImmutableBytesPtr,RowMutationState> newRows = Maps.newHashMapWithExpectedSize(connection.getMutateBatchSize());
+            MultiRowMutationState newRows = new MultiRowMutationState(connection.getMutateBatchSize());
             newRows.putAll(srcRows);
             dstMutations.put(tableRef, newRows);
             if (incrementRowCount && !isIndex) {
                 numRows += srcRows.size();
+                // if we added all the rows from newMutationState we can just increment the
+                // estimatedSize by newMutationState.estimatedSize
+                estimatedSize +=  srcRows.estimatedSize;
             }
         }
     }
     
-    private void joinMutationState(Map<TableRef, Map<ImmutableBytesPtr, RowMutationState>> srcMutations, 
-            Map<TableRef, Map<ImmutableBytesPtr, RowMutationState>> dstMutations) {
+    private void joinMutationState(Map<TableRef, MultiRowMutationState> srcMutations, 
+            Map<TableRef, MultiRowMutationState> dstMutations) {
         // Merge newMutation with this one, keeping state from newMutation for any overlaps
-        for (Map.Entry<TableRef, Map<ImmutableBytesPtr,RowMutationState>> entry : srcMutations.entrySet()) {
+        for (Map.Entry<TableRef, MultiRowMutationState> entry : srcMutations.entrySet()) {
             // Replace existing entries for the table with new entries
             TableRef tableRef = entry.getKey();
-            Map<ImmutableBytesPtr,RowMutationState> srcRows = entry.getValue();
+            MultiRowMutationState srcRows = entry.getValue();
             joinMutationState(tableRef, srcRows, dstMutations);
         }
     }
@@ -439,19 +448,7 @@ public class MutationState implements SQLCloseable {
         phoenixTransactionContext.join(newMutationState.getPhoenixTransactionContext());
 
         this.sizeOffset += newMutationState.sizeOffset;
-        int oldNumRows = this.numRows;
         joinMutationState(newMutationState.mutations, this.mutations);
-        if (newMutationState.numRows>0) {
-            // if we added all the rows from newMutationState we can just increment the
-            // estimatedSize by newMutationState.estimatedSize
-            if (newMutationState.numRows == this.numRows-oldNumRows) {
-                this.estimatedSize +=  newMutationState.estimatedSize;
-            }
-            // we merged the two mutation states so we need to recalculate the size
-            else {
-                this.estimatedSize = KeyValueUtil.getEstimatedRowMutationSize(this.mutations);
-            }
-        }
         if (!newMutationState.txMutations.isEmpty()) {
             if (txMutations.isEmpty()) {
                 txMutations = Maps.newHashMapWithExpectedSize(mutations.size());
@@ -489,7 +486,7 @@ public class MutationState implements SQLCloseable {
         return ptr;
     }
     
-    private Iterator<Pair<PName,List<Mutation>>> addRowMutations(final TableRef tableRef, final Map<ImmutableBytesPtr, RowMutationState> values,
+    private Iterator<Pair<PName,List<Mutation>>> addRowMutations(final TableRef tableRef, final MultiRowMutationState values,
             final long mutationTimestamp, final long serverTimestamp, boolean includeAllIndexes, final boolean sendAll) { 
         final PTable table = tableRef.getTable();
         final Iterator<PTable> indexes = // Only maintain tables with immutable rows through this client-side mechanism
@@ -524,10 +521,10 @@ public class MutationState implements SQLCloseable {
                     // we may also have to include delete mutations for immutable tables if we are not processing all the tables in the mutations map
                     if (!sendAll) {
                         TableRef key = new TableRef(index);
-                        Map<ImmutableBytesPtr, RowMutationState> rowToColumnMap = mutations.remove(key);
-                        if (rowToColumnMap!=null) {
+                        MultiRowMutationState multiRowMutationState = mutations.remove(key);
+                        if (multiRowMutationState!=null) {
                             final List<Mutation> deleteMutations = Lists.newArrayList();
-                            generateMutations(tableRef, mutationTimestamp, serverTimestamp, rowToColumnMap, deleteMutations, null);
+                            generateMutations(tableRef, mutationTimestamp, serverTimestamp, multiRowMutationState, deleteMutations, null);
                             indexMutations.addAll(deleteMutations);
                         }
                     }
@@ -546,14 +543,14 @@ public class MutationState implements SQLCloseable {
     }
 
     private void generateMutations(final TableRef tableRef, final long mutationTimestamp,
-            final long serverTimestamp, final Map<ImmutableBytesPtr, RowMutationState> values,
+            final long serverTimestamp, final MultiRowMutationState values,
             final List<Mutation> mutationList, final List<Mutation> mutationsPertainingToIndex) {
         final PTable table = tableRef.getTable();
         boolean tableWithRowTimestampCol = table.getRowTimestampColPos() != -1;
         Iterator<Map.Entry<ImmutableBytesPtr, RowMutationState>> iterator =
                 values.entrySet().iterator();
         long timestampToUse = mutationTimestamp;
-        Map<ImmutableBytesPtr, RowMutationState> modifiedValues = Maps.newHashMap();
+        MultiRowMutationState modifiedValues = new MultiRowMutationState(16);
         while (iterator.hasNext()) {
             Map.Entry<ImmutableBytesPtr, RowMutationState> rowEntry = iterator.next();
             byte[] onDupKeyBytes = rowEntry.getValue().getOnDupKeyBytes();
@@ -628,7 +625,7 @@ public class MutationState implements SQLCloseable {
     }
     
     public Iterator<Pair<byte[],List<Mutation>>> toMutations(final boolean includeMutableIndexes, final Long tableTimestamp) {
-        final Iterator<Map.Entry<TableRef, Map<ImmutableBytesPtr,RowMutationState>>> iterator = this.mutations.entrySet().iterator();
+        final Iterator<Map.Entry<TableRef, MultiRowMutationState>> iterator = this.mutations.entrySet().iterator();
         if (!iterator.hasNext()) {
             return Collections.emptyIterator();
         }
@@ -636,7 +633,7 @@ public class MutationState implements SQLCloseable {
         final long serverTimestamp = getTableTimestamp(tableTimestamp, scn);
         final long mutationTimestamp = getMutationTimestamp(scn);
         return new Iterator<Pair<byte[],List<Mutation>>>() {
-            private Map.Entry<TableRef, Map<ImmutableBytesPtr,RowMutationState>> current = iterator.next();
+            private Map.Entry<TableRef, MultiRowMutationState> current = iterator.next();
             private Iterator<Pair<byte[],List<Mutation>>> innerIterator = init();
                     
             private Iterator<Pair<byte[],List<Mutation>>> init() {
@@ -700,14 +697,14 @@ public class MutationState implements SQLCloseable {
     private long[] validateAll() throws SQLException {
         int i = 0;
         long[] timeStamps = new long[this.mutations.size()];
-        for (Map.Entry<TableRef, Map<ImmutableBytesPtr,RowMutationState>> entry : mutations.entrySet()) {
+        for (Map.Entry<TableRef, MultiRowMutationState> entry : mutations.entrySet()) {
             TableRef tableRef = entry.getKey();
             timeStamps[i++] = validateAndGetServerTimestamp(tableRef, entry.getValue());
         }
         return timeStamps;
     }
     
-    private long validateAndGetServerTimestamp(TableRef tableRef, Map<ImmutableBytesPtr, RowMutationState> rowKeyToColumnMap) throws SQLException {
+    private long validateAndGetServerTimestamp(TableRef tableRef, MultiRowMutationState rowKeyToColumnMap) throws SQLException {
         Long scn = connection.getSCN();
         MetaDataClient client = new MetaDataClient(connection);
         long serverTimeStamp = tableRef.getTimeStamp();
@@ -919,7 +916,7 @@ public class MutationState implements SQLCloseable {
             sendAll = true;
         }
 
-        Map<ImmutableBytesPtr, RowMutationState> valuesMap;
+        MultiRowMutationState multiRowMutationState;
         Map<TableInfo,List<Mutation>> physicalTableMutationMap = Maps.newLinkedHashMap(); 
         // add tracing for this operation
         try (TraceScope trace = Tracing.startNewSpan(connection, "Committing mutations to tables")) {
@@ -928,16 +925,16 @@ public class MutationState implements SQLCloseable {
             while (tableRefIterator.hasNext()) {
                 // at this point we are going through mutations for each table
                 final TableRef tableRef = tableRefIterator.next();
-                valuesMap = mutations.get(tableRef);
-                if (valuesMap == null || valuesMap.isEmpty()) {
+                multiRowMutationState = mutations.get(tableRef);
+                if (multiRowMutationState == null || multiRowMutationState.isEmpty()) {
                     continue;
                 }
                 // Validate as we go if transactional since we can undo if a problem occurs (which is unlikely)
-                long serverTimestamp = serverTimeStamps == null ? validateAndGetServerTimestamp(tableRef, valuesMap) : serverTimeStamps[i++];
+                long serverTimestamp = serverTimeStamps == null ? validateAndGetServerTimestamp(tableRef, multiRowMutationState) : serverTimeStamps[i++];
                 Long scn = connection.getSCN();
                 long mutationTimestamp = scn == null ? HConstants.LATEST_TIMESTAMP : scn;
                 final PTable table = tableRef.getTable();
-                Iterator<Pair<PName,List<Mutation>>> mutationsIterator = addRowMutations(tableRef, valuesMap, mutationTimestamp, serverTimestamp, false, sendAll);
+                Iterator<Pair<PName,List<Mutation>>> mutationsIterator = addRowMutations(tableRef, multiRowMutationState, mutationTimestamp, serverTimestamp, false, sendAll);
                 // build map from physical table to mutation list
                 boolean isDataTable = true;
                 while (mutationsIterator.hasNext()) {
@@ -955,7 +952,7 @@ public class MutationState implements SQLCloseable {
                 // involved in the transaction since none of them would have been
                 // committed in the event of a failure.
                 if (table.isTransactional()) {
-                    addUncommittedStatementIndexes(valuesMap.values());
+                    addUncommittedStatementIndexes(multiRowMutationState.values());
                     if (txMutations.isEmpty()) {
                         txMutations = Maps.newHashMapWithExpectedSize(mutations.size());
                     }
@@ -964,7 +961,7 @@ public class MutationState implements SQLCloseable {
                     // in the event that we need to replay the commit.
                     // Copy TableRef so we have the original PTable and know when the
                     // indexes have changed.
-                    joinMutationState(new TableRef(tableRef), valuesMap, txMutations);
+                    joinMutationState(new TableRef(tableRef), multiRowMutationState, txMutations);
                 }
             }
             long serverTimestamp = HConstants.LATEST_TIMESTAMP;
@@ -1188,7 +1185,7 @@ public class MutationState implements SQLCloseable {
     }
     
     private int[] getUncommittedStatementIndexes() {
-        for (Map<ImmutableBytesPtr, RowMutationState> rowMutationMap : mutations.values()) {
+        for (MultiRowMutationState rowMutationMap : mutations.values()) {
             addUncommittedStatementIndexes(rowMutationMap.values());
         }
         return uncommittedStatementIndexes;
@@ -1221,7 +1218,7 @@ public class MutationState implements SQLCloseable {
     }
 
     public void commit() throws SQLException {
-        Map<TableRef, Map<ImmutableBytesPtr,RowMutationState>> txMutations = Collections.emptyMap();
+        Map<TableRef, MultiRowMutationState> txMutations = Collections.emptyMap();
         int retryCount = 0;
         do {
             boolean sendSuccessful=false;
@@ -1431,6 +1428,46 @@ public class MutationState implements SQLCloseable {
         }
     }
     
+    public static class MultiRowMutationState {
+        private Map<ImmutableBytesPtr,RowMutationState> rowKeyToRowMutationState;
+        private long estimatedSize;
+        
+        public MultiRowMutationState(int size) {
+            this.rowKeyToRowMutationState = Maps.newHashMapWithExpectedSize(size);
+            this.estimatedSize = 0;
+        }
+        
+        public RowMutationState put(ImmutableBytesPtr ptr, RowMutationState rowMutationState) { 
+            estimatedSize += rowMutationState.calculateEstimatedSize();
+            return rowKeyToRowMutationState.put(ptr, rowMutationState);
+        }
+        
+        public void putAll(MultiRowMutationState other) {
+            estimatedSize += other.estimatedSize;
+            rowKeyToRowMutationState.putAll(other.rowKeyToRowMutationState);
+        }
+        
+        public boolean isEmpty() {
+            return rowKeyToRowMutationState.isEmpty();
+        }
+        
+        public int size() {
+            return rowKeyToRowMutationState.size();
+        }
+        
+        public Set<Entry<ImmutableBytesPtr, RowMutationState>> entrySet() {
+            return rowKeyToRowMutationState.entrySet();
+        }
+        
+        public void clear(){
+            rowKeyToRowMutationState.clear();
+        }
+        
+        public Collection<RowMutationState> values() {
+            return rowKeyToRowMutationState.values();
+        }
+    }
+    
     public static class RowMutationState {
         @Nonnull private Map<PColumn,byte[]> columnValues;
         private int[] statementIndexes;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/eb0f744d/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
index b23ea1b..74f91b4 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
@@ -73,7 +73,7 @@ import org.apache.phoenix.coprocessor.generated.MetaDataProtos.MetaDataService;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.UpdateIndexStateRequest;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.exception.SQLExceptionInfo;
-import org.apache.phoenix.execute.MutationState.RowMutationState;
+import org.apache.phoenix.execute.MutationState.MultiRowMutationState;
 import org.apache.phoenix.execute.TupleProjector;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.expression.KeyValueColumnExpression;
@@ -296,7 +296,7 @@ public class IndexUtil {
     }
     
     public static List<Mutation> generateIndexData(final PTable table, PTable index,
-            final Map<ImmutableBytesPtr, RowMutationState> valuesMap, List<Mutation> dataMutations, final KeyValueBuilder kvBuilder, PhoenixConnection connection)
+            final MultiRowMutationState multiRowMutationState, List<Mutation> dataMutations, final KeyValueBuilder kvBuilder, PhoenixConnection connection)
             throws SQLException {
         try {
         	final ImmutableBytesPtr ptr = new ImmutableBytesPtr();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/eb0f744d/phoenix-core/src/main/java/org/apache/phoenix/util/KeyValueUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/KeyValueUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/KeyValueUtil.java
index 318c9d6..df6a349 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/KeyValueUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/KeyValueUtil.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValue.Type;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.phoenix.execute.MutationState.MultiRowMutationState;
 import org.apache.phoenix.execute.MutationState.RowMutationState;
 import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.hbase.index.util.KeyValueBuilder;
@@ -188,10 +189,10 @@ public class KeyValueUtil {
      * @return estimated row size
      */
     public static long
-            getEstimatedRowMutationSize(Map<TableRef, Map<ImmutableBytesPtr, RowMutationState>> tableMutationMap) {
+            getEstimatedRowMutationSize(Map<TableRef, MultiRowMutationState> tableMutationMap) {
         long size = 0;
         // iterate over table
-        for (Entry<TableRef, Map<ImmutableBytesPtr, RowMutationState>> tableEntry : tableMutationMap.entrySet()) {
+        for (Entry<TableRef, MultiRowMutationState> tableEntry : tableMutationMap.entrySet()) {
             // iterate over rows
             for (Entry<ImmutableBytesPtr, RowMutationState> rowEntry : tableEntry.getValue().entrySet()) {
                 size += calculateRowMutationSize(rowEntry);


[14/16] phoenix git commit: PHOENIX-3050 Handle DESC columns in child/parent join optimization

Posted by ja...@apache.org.
PHOENIX-3050 Handle DESC columns in child/parent join optimization


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

Branch: refs/heads/4.x-HBase-1.2
Commit: 4968e88e6d5f4d650ee05303dbce8baa5c911087
Parents: 7e61234
Author: maryannxue <ma...@gmail.com>
Authored: Mon Nov 6 02:37:55 2017 +0000
Committer: James Taylor <jt...@salesforce.com>
Committed: Sat Dec 16 16:42:54 2017 -0800

----------------------------------------------------------------------
 .../phoenix/end2end/join/HashJoinMoreIT.java     |  5 +++++
 .../org/apache/phoenix/compile/JoinCompiler.java | 19 +++++++++++++------
 .../apache/phoenix/compile/QueryCompiler.java    |  6 +++---
 .../apache/phoenix/compile/WhereOptimizer.java   |  5 -----
 4 files changed, 21 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/4968e88e/phoenix-core/src/it/java/org/apache/phoenix/end2end/join/HashJoinMoreIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/join/HashJoinMoreIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/join/HashJoinMoreIT.java
index 37ffd02..f09f1d3 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/join/HashJoinMoreIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/join/HashJoinMoreIT.java
@@ -895,6 +895,11 @@ public class HashJoinMoreIT extends ParallelStatsDisabledIT {
                     + "FROM ( SELECT ACCOUNT_ID, BUCKET_ID, OBJECT_ID, MAX(OBJECT_VERSION) AS MAXVER "
                     + "       FROM test2961 GROUP BY ACCOUNT_ID, BUCKET_ID, OBJECT_ID) AS X "
                     + "       INNER JOIN test2961 AS OBJ ON X.ACCOUNT_ID = OBJ.ACCOUNT_ID AND X.BUCKET_ID = OBJ.BUCKET_ID AND X.OBJECT_ID = OBJ.OBJECT_ID AND  X.MAXVER = OBJ.OBJECT_VERSION";
+            rs = conn.createStatement().executeQuery("explain " + q);
+            String plan = QueryUtil.getExplainPlan(rs);
+            String dynamicFilter = "DYNAMIC SERVER FILTER BY (OBJ.ACCOUNT_ID, OBJ.BUCKET_ID, OBJ.OBJECT_ID, OBJ.OBJECT_VERSION) IN ((X.ACCOUNT_ID, X.BUCKET_ID, X.OBJECT_ID, X.MAXVER))";
+            assertTrue("Expected '" + dynamicFilter + "' to be used for the query, but got:\n" + plan,
+                    plan.contains(dynamicFilter));
             rs = conn.createStatement().executeQuery(q);
             assertTrue(rs.next());
             assertEquals("2222", rs.getString(4));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4968e88e/phoenix-core/src/main/java/org/apache/phoenix/compile/JoinCompiler.java
----------------------------------------------------------------------
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 887e2d2..439a79b 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
@@ -110,6 +110,12 @@ import com.google.common.collect.Sets;
 
 public class JoinCompiler {
 
+    public enum Strategy {
+        HASH_BUILD_LEFT,
+        HASH_BUILD_RIGHT,
+        SORT_MERGE,
+    }
+
     public enum ColumnRefType {
         JOINLOCAL,
         GENERAL,
@@ -489,7 +495,7 @@ public class JoinCompiler {
             return dependencies;
         }
 
-        public Pair<List<Expression>, List<Expression>> compileJoinConditions(StatementContext lhsCtx, StatementContext rhsCtx, boolean sortExpressions) throws SQLException {
+        public Pair<List<Expression>, List<Expression>> compileJoinConditions(StatementContext lhsCtx, StatementContext rhsCtx, Strategy strategy) throws SQLException {
             if (onConditions.isEmpty()) {
                 return new Pair<List<Expression>, List<Expression>>(
                         Collections.<Expression> singletonList(LiteralExpression.newConstant(1)),
@@ -505,15 +511,16 @@ public class JoinCompiler {
                 rhsCompiler.reset();
                 Expression right = condition.getRHS().accept(rhsCompiler);
                 PDataType toType = getCommonType(left.getDataType(), right.getDataType());
-                if (left.getDataType() != toType || left.getSortOrder() == SortOrder.DESC) {
-                    left = CoerceExpression.create(left, toType, SortOrder.ASC, left.getMaxLength());
+                SortOrder toSortOrder = strategy == Strategy.SORT_MERGE ? SortOrder.ASC : (strategy == Strategy.HASH_BUILD_LEFT ? right.getSortOrder() : left.getSortOrder());
+                if (left.getDataType() != toType || left.getSortOrder() != toSortOrder) {
+                    left = CoerceExpression.create(left, toType, toSortOrder, left.getMaxLength());
                 }
-                if (right.getDataType() != toType || right.getSortOrder() == SortOrder.DESC) {
-                    right = CoerceExpression.create(right, toType, SortOrder.ASC, right.getMaxLength());
+                if (right.getDataType() != toType || right.getSortOrder() != toSortOrder) {
+                    right = CoerceExpression.create(right, toType, toSortOrder, right.getMaxLength());
                 }
                 compiled.add(new Pair<Expression, Expression>(left, right));
             }
-            if (sortExpressions) {
+            if (strategy != Strategy.SORT_MERGE) {
                 Collections.sort(compiled, new Comparator<Pair<Expression, Expression>>() {
                     @Override
                     public int compare(Pair<Expression, Expression> o1, Pair<Expression, Expression> o2) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4968e88e/phoenix-core/src/main/java/org/apache/phoenix/compile/QueryCompiler.java
----------------------------------------------------------------------
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 af2254b..287f9e0 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
@@ -307,7 +307,7 @@ public class QueryCompiler {
                 JoinSpec joinSpec = joinSpecs.get(i);
                 context.setResolver(FromCompiler.getResolverForProjectedTable(projectedTable, context.getConnection(), query.getUdfParseNodes()));
                 joinIds[i] = new ImmutableBytesPtr(emptyByteArray); // place-holder
-                Pair<List<Expression>, List<Expression>> joinConditions = joinSpec.compileJoinConditions(context, subContexts[i], true);
+                Pair<List<Expression>, List<Expression>> joinConditions = joinSpec.compileJoinConditions(context, subContexts[i], JoinCompiler.Strategy.HASH_BUILD_RIGHT);
                 joinExpressions[i] = joinConditions.getFirst();
                 List<Expression> hashExpressions = joinConditions.getSecond();
                 Pair<Expression, Expression> keyRangeExpressions = new Pair<Expression, Expression>(null, null);
@@ -369,7 +369,7 @@ public class QueryCompiler {
             context.setCurrentTable(rhsTableRef);
             context.setResolver(FromCompiler.getResolverForProjectedTable(rhsProjTable, context.getConnection(), rhs.getUdfParseNodes()));
             ImmutableBytesPtr[] joinIds = new ImmutableBytesPtr[] {new ImmutableBytesPtr(emptyByteArray)};
-            Pair<List<Expression>, List<Expression>> joinConditions = lastJoinSpec.compileJoinConditions(lhsCtx, context, true);
+            Pair<List<Expression>, List<Expression>> joinConditions = lastJoinSpec.compileJoinConditions(lhsCtx, context, JoinCompiler.Strategy.HASH_BUILD_LEFT);
             List<Expression> joinExpressions = joinConditions.getSecond();
             List<Expression> hashExpressions = joinConditions.getFirst();
             boolean needsMerge = lhsJoin.hasPostReference();
@@ -422,7 +422,7 @@ public class QueryCompiler {
         QueryPlan rhsPlan = compileJoinQuery(rhsCtx, binds, rhsJoin, true, true, rhsOrderBy);
         PTable rhsProjTable = rhsCtx.getResolver().getTables().get(0).getTable();
         
-        Pair<List<Expression>, List<Expression>> joinConditions = lastJoinSpec.compileJoinConditions(type == JoinType.Right ? rhsCtx : lhsCtx, type == JoinType.Right ? lhsCtx : rhsCtx, false);
+        Pair<List<Expression>, List<Expression>> joinConditions = lastJoinSpec.compileJoinConditions(type == JoinType.Right ? rhsCtx : lhsCtx, type == JoinType.Right ? lhsCtx : rhsCtx, JoinCompiler.Strategy.SORT_MERGE);
         List<Expression> lhsKeyExpressions = type == JoinType.Right ? joinConditions.getSecond() : joinConditions.getFirst();
         List<Expression> rhsKeyExpressions = type == JoinType.Right ? joinConditions.getFirst() : joinConditions.getSecond();
         

http://git-wip-us.apache.org/repos/asf/phoenix/blob/4968e88e/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereOptimizer.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereOptimizer.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereOptimizer.java
index 5e7b996..7bf8259 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
@@ -326,11 +326,6 @@ public class WhereOptimizer {
         PTable table = context.getCurrentTable().getTable();
         for (int i = 0; i < expressions.size(); i++) {
             Expression expression = expressions.get(i);
-            // TODO this is a temporary fix for PHOENIX-3029.
-            if (expression instanceof CoerceExpression
-                    && expression.getSortOrder() != expression.getChildren().get(0).getSortOrder()) {
-                continue;
-            }
             KeyExpressionVisitor visitor = new KeyExpressionVisitor(context, table);
             KeyExpressionVisitor.KeySlots keySlots = expression.accept(visitor);
             int minPkPos = Integer.MAX_VALUE; 


[09/16] phoenix git commit: PHOENIX-4198 Remove the need for users to have access to the Phoenix SYSTEM tables to create tables

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/493a54f9/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/PhoenixAccessController.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..8437b37
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/PhoenixAccessController.java
@@ -0,0 +1,628 @@
+/*
+ * 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.coprocessor;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.AuthUtil;
+import org.apache.hadoop.hbase.CoprocessorEnvironment;
+import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.coprocessor.BaseMasterAndRegionObserver;
+import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
+import org.apache.hadoop.hbase.ipc.RpcServer;
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos;
+import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.AccessControlService;
+import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost;
+import org.apache.hadoop.hbase.security.AccessDeniedException;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.security.UserProvider;
+import org.apache.hadoop.hbase.security.access.AccessControlClient;
+import org.apache.hadoop.hbase.security.access.AuthResult;
+import org.apache.hadoop.hbase.security.access.Permission;
+import org.apache.hadoop.hbase.security.access.Permission.Action;
+import org.apache.hadoop.hbase.security.access.UserPermission;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.coprocessor.PhoenixMetaDataCoprocessorHost.PhoenixMetaDataControllerEnvironment;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.query.QueryServicesOptions;
+import org.apache.phoenix.schema.PIndexState;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTableType;
+import org.apache.phoenix.util.MetaDataUtil;
+
+import com.google.common.collect.Lists;
+import com.google.protobuf.RpcCallback;
+
+public class PhoenixAccessController extends BaseMetaDataEndpointObserver {
+
+    private PhoenixMetaDataControllerEnvironment env;
+    private ArrayList<BaseMasterAndRegionObserver> accessControllers;
+    private boolean accessCheckEnabled;
+    private UserProvider userProvider;
+    private boolean isAutomaticGrantEnabled;
+    private boolean isStrictMode;
+    public static final Log LOG = LogFactory.getLog(PhoenixAccessController.class);
+    private static final Log AUDITLOG =
+            LogFactory.getLog("SecurityLogger."+PhoenixAccessController.class.getName());
+
+    private List<BaseMasterAndRegionObserver> getAccessControllers() throws IOException {
+        if (accessControllers == null) {
+            synchronized (this) {
+                if (accessControllers == null) {
+                    accessControllers = new ArrayList<BaseMasterAndRegionObserver>();
+                    RegionCoprocessorHost cpHost = this.env.getCoprocessorHost();
+                    List<BaseMasterAndRegionObserver> coprocessors = cpHost
+                            .findCoprocessors(BaseMasterAndRegionObserver.class);
+                    for (BaseMasterAndRegionObserver cp : coprocessors) {
+                        if (cp instanceof AccessControlService.Interface) {
+                            accessControllers.add(cp);
+                        }
+                    }
+                }
+            }
+        }
+        return accessControllers;
+    }
+
+    @Override
+    public void preGetTable(ObserverContext<PhoenixMetaDataControllerEnvironment> ctx, String tenantId,
+            String tableName, TableName physicalTableName) throws IOException {
+        for (BaseMasterAndRegionObserver observer : getAccessControllers()) {
+            observer.preGetTableDescriptors(new ObserverContext<MasterCoprocessorEnvironment>(),
+                    Lists.newArrayList(physicalTableName), Collections.<HTableDescriptor> emptyList());
+        }
+    }
+
+    @Override
+    public void start(CoprocessorEnvironment env) throws IOException {
+        Configuration conf = env.getConfiguration();
+        this.accessCheckEnabled = conf.getBoolean(QueryServices.PHOENIX_ACLS_ENABLED,
+                QueryServicesOptions.DEFAULT_PHOENIX_ACLS_ENABLED);
+        this.isAutomaticGrantEnabled=conf.getBoolean(QueryServices.PHOENIX_AUTOMATIC_GRANT_ENABLED,
+                QueryServicesOptions.DEFAULT_PHOENIX_AUTOMATIC_GRANT_ENABLED);
+        if (!this.accessCheckEnabled) {
+            LOG.warn("PhoenixAccessController has been loaded with authorization checks disabled.");
+        }
+        if (env instanceof PhoenixMetaDataControllerEnvironment) {
+            this.env = (PhoenixMetaDataControllerEnvironment)env;
+        } else {
+            throw new IllegalArgumentException(
+                    "Not a valid environment, should be loaded by PhoenixMetaDataControllerEnvironment");
+        }
+        // set the user-provider.
+        this.userProvider = UserProvider.instantiate(env.getConfiguration());
+        this.isStrictMode = conf.getBoolean(QueryServices.PHOENIX_SECURITY_PERMISSION_STRICT_MODE_ENABLED,
+                QueryServicesOptions.DEFAULT_PHOENIX_SECURITY_PERMISSION_STRICT_MODE_ENABLED);
+        // init superusers and add the server principal (if using security)
+        // or process owner as default super user.
+        Superusers.initialize(env.getConfiguration());
+    }
+
+    @Override
+    public void stop(CoprocessorEnvironment env) throws IOException {}
+
+    @Override
+    public void preCreateTable(ObserverContext<PhoenixMetaDataControllerEnvironment> ctx, String tenantId,
+            String tableName, TableName physicalTableName, TableName parentPhysicalTableName, PTableType tableType,
+            Set<byte[]> familySet, Set<TableName> indexes) throws IOException {
+        if (!accessCheckEnabled) { return; }
+        
+        if (tableType != PTableType.VIEW) {
+            final HTableDescriptor htd = new HTableDescriptor(physicalTableName);
+            for (byte[] familyName : familySet) {
+                htd.addFamily(new HColumnDescriptor(familyName));
+            }
+            for (BaseMasterAndRegionObserver observer : getAccessControllers()) {
+                observer.preCreateTable(new ObserverContext<MasterCoprocessorEnvironment>(), htd, null);
+            }
+        }
+
+        // Index and view require read access on parent physical table.
+        Set<TableName> physicalTablesChecked = new HashSet<TableName>();
+        if (tableType == PTableType.VIEW || tableType == PTableType.INDEX) {
+            physicalTablesChecked.add(parentPhysicalTableName);
+            requireAccess("Create" + tableType, parentPhysicalTableName, Action.READ, Action.EXEC);
+        }
+
+        if (tableType == PTableType.VIEW) {
+            
+            Action[] requiredActions = { Action.READ, Action.EXEC };
+            for (TableName index : indexes) {
+                if (!physicalTablesChecked.add(index)) {
+                    // skip check for local index as we have already check the ACLs above
+                    // And for same physical table multiple times like view index table
+                    continue;
+                }
+
+                User user = getActiveUser();
+                List<UserPermission> permissionForUser = getPermissionForUser(
+                        getUserPermissions(index.getNameAsString()), Bytes.toBytes(user.getShortName()));
+                Set<Action> requireAccess = new HashSet<>();
+                Set<Action> accessExists = new HashSet<>();
+                if (permissionForUser != null) {
+                    for (UserPermission userPermission : permissionForUser) {
+                        for (Action action : Arrays.asList(requiredActions)) {
+                            if (!userPermission.implies(action)) {
+                                requireAccess.add(action);
+                            }
+                        }
+                    }
+                    if (!requireAccess.isEmpty()) {
+                        for (UserPermission userPermission : permissionForUser) {
+                            accessExists.addAll(Arrays.asList(userPermission.getActions()));
+                        }
+
+                    }
+                } else {
+                    requireAccess.addAll(Arrays.asList(requiredActions));
+                }
+                if (!requireAccess.isEmpty()) {
+                    byte[] indexPhysicalTable = index.getName();
+                    handleRequireAccessOnDependentTable("Create" + tableType, user.getName(),
+                            TableName.valueOf(indexPhysicalTable), tableName, requireAccess, accessExists);
+                }
+            }
+
+        }
+
+        if (tableType == PTableType.INDEX) {
+            // All the users who have READ access on data table should have access to Index table as well.
+            // WRITE is needed for the index updates done by the user who has WRITE access on data table.
+            // CREATE is needed during the drop of the table.
+            // We are doing this because existing user while querying data table should not see access denied for the
+            // new indexes.
+            // TODO: confirm whether granting permission from coprocessor is a security leak.(currently it is done if
+            // automatic grant is enabled explicitly by user in configuration
+            // skip check for local index
+            if (physicalTableName != null && !parentPhysicalTableName.equals(physicalTableName)
+                    && !MetaDataUtil.isViewIndex(physicalTableName.getNameAsString())) {
+                authorizeOrGrantAccessToUsers("Create" + tableType, parentPhysicalTableName,
+                        Arrays.asList(Action.READ, Action.WRITE, Action.CREATE, Action.EXEC, Action.ADMIN),
+                        physicalTableName);
+            }
+        }
+    }
+
+    
+    public void handleRequireAccessOnDependentTable(String request, String userName, TableName dependentTable,
+            String requestTable, Set<Action> requireAccess, Set<Action> accessExists) throws IOException {
+
+        if (!isStrictMode) {
+            AUDITLOG.warn("Strict mode is not enabled, so " + request + " is allowed but User:" + userName
+                    + " will not have following access " + requireAccess + " to the existing dependent physical table "
+                    + dependentTable);
+            return;
+        }
+        if (isAutomaticGrantEnabled) {
+            Set<Action> unionSet = new HashSet<Action>();
+            unionSet.addAll(requireAccess);
+            unionSet.addAll(accessExists);
+            AUDITLOG.info(request + ": Automatically granting access to index table during creation of view:"
+                    + requestTable + authString(userName, dependentTable, requireAccess));
+            grantPermissions(userName, dependentTable.getName(), unionSet.toArray(new Action[0]));
+        } else {
+            throw new AccessDeniedException(
+                    "Insufficient permissions for users of dependent table" + authString(userName, dependentTable, requireAccess));
+        }
+    }
+    
+    private void grantPermissions(final String toUser, final byte[] table, final Action... actions) throws IOException {
+        User.runAsLoginUser(new PrivilegedExceptionAction<Void>() {
+            @Override
+            public Void run() throws Exception {
+                try (Connection conn = ConnectionFactory.createConnection(env.getConfiguration())) {
+                    AccessControlClient.grant(conn, TableName.valueOf(table), toUser , null, null,
+                            actions);
+                } catch (Throwable e) {
+                    new DoNotRetryIOException(e);
+                }
+                return null;
+            }
+        });
+    }
+
+    private void authorizeOrGrantAccessToUsers(final String request, final TableName fromTable,
+            final List<Action> requiredActionsOnTable, final TableName toTable)
+            throws IOException {
+        User.runAsLoginUser(new PrivilegedExceptionAction<Void>() {
+            @Override
+            public Void run() throws IOException {
+                try (Connection conn = ConnectionFactory.createConnection(env.getConfiguration())) {
+                    List<UserPermission> userPermissions = getUserPermissions(fromTable.getNameAsString());
+                    List<UserPermission> permissionsOnTheTable = getUserPermissions(toTable.getNameAsString());
+                    if (userPermissions != null) {
+                        for (UserPermission userPermission : userPermissions) {
+                            Set<Action> requireAccess = new HashSet<Action>();
+                            Set<Action> accessExists = new HashSet<Action>();
+                            List<UserPermission> permsToTable = getPermissionForUser(permissionsOnTheTable,
+                                    userPermission.getUser());
+                            for (Action action : requiredActionsOnTable) {
+                                boolean haveAccess=false;
+                                if (userPermission.implies(action)) {
+                                    if (permsToTable == null) {
+                                        requireAccess.add(action);
+                                    } else {
+                                        for (UserPermission permToTable : permsToTable) {
+                                            if (permToTable.implies(action)) {
+                                                haveAccess=true;
+                                            }
+                                        }
+                                        if (!haveAccess) {
+                                            requireAccess.add(action);
+                                        }
+                                    }
+                                }
+                            }
+                            if (permsToTable != null) {
+                                // Append access to already existing access for the user
+                                for (UserPermission permToTable : permsToTable) {
+                                    accessExists.addAll(Arrays.asList(permToTable.getActions()));
+                                }
+                            }
+                            if (!requireAccess.isEmpty()) {
+                                if(AuthUtil.isGroupPrincipal(Bytes.toString(userPermission.getUser()))){
+                                    AUDITLOG.warn("Users of GROUP:" + Bytes.toString(userPermission.getUser())
+                                            + " will not have following access " + requireAccess
+                                            + " to the newly created index " + toTable
+                                            + ", Automatic grant is not yet allowed on Groups");
+                                    continue;
+                                }
+                                handleRequireAccessOnDependentTable(request, Bytes.toString(userPermission.getUser()),
+                                        toTable, toTable.getNameAsString(), requireAccess, accessExists);
+                            }
+                        }
+                    }
+                }
+                return null;
+            }
+        });
+    }
+
+    private List<UserPermission> getPermissionForUser(List<UserPermission> perms, byte[] user) {
+        if (perms != null) {
+            // get list of permissions for the user as multiple implementation of AccessControl coprocessors can give
+            // permissions for same users
+            List<UserPermission> permissions = new ArrayList<>();
+            for (UserPermission p : perms) {
+                if (Bytes.equals(p.getUser(),user)){
+                     permissions.add(p);
+                }
+            }
+            if (!permissions.isEmpty()){
+               return permissions;
+            }
+        }
+        return null;
+    }
+
+    @Override
+    public void preDropTable(ObserverContext<PhoenixMetaDataControllerEnvironment> ctx, String tenantId,
+            String tableName, TableName physicalTableName, TableName parentPhysicalTableName, PTableType tableType,
+            List<PTable> indexes) throws IOException {
+        if (!accessCheckEnabled) { return; }
+
+        for (BaseMasterAndRegionObserver observer : getAccessControllers()) {
+            if (tableType != PTableType.VIEW) {
+                observer.preDeleteTable(new ObserverContext<MasterCoprocessorEnvironment>(), physicalTableName);
+            }
+            if (indexes != null) {
+                for (PTable index : indexes) {
+                    observer.preDeleteTable(new ObserverContext<MasterCoprocessorEnvironment>(),
+                            TableName.valueOf(index.getPhysicalName().getBytes()));
+                }
+            }
+        }
+        //checking similar permission checked during the create of the view.
+        if (tableType == PTableType.VIEW || tableType == PTableType.INDEX) {
+            requireAccess("Drop "+tableType, parentPhysicalTableName, Action.READ, Action.EXEC);
+        }
+    }
+
+    @Override
+    public void preAlterTable(ObserverContext<PhoenixMetaDataControllerEnvironment> ctx, String tenantId,
+            String tableName, TableName physicalTableName, TableName parentPhysicalTableName, PTableType tableType) throws IOException {
+        if (!accessCheckEnabled) { return; }
+        for (BaseMasterAndRegionObserver observer : getAccessControllers()) {
+            if (tableType != PTableType.VIEW) {
+            observer.preModifyTable(new ObserverContext<MasterCoprocessorEnvironment>(), physicalTableName,
+                    new HTableDescriptor(physicalTableName));
+            }
+        }
+        if (tableType == PTableType.VIEW) {
+            requireAccess("Alter "+tableType, parentPhysicalTableName, Action.READ, Action.EXEC);
+        }
+    }
+
+    @Override
+    public void preGetSchema(ObserverContext<PhoenixMetaDataControllerEnvironment> ctx, String schemaName)
+            throws IOException {
+        if (!accessCheckEnabled) { return; }
+        for (BaseMasterAndRegionObserver observer : getAccessControllers()) {
+            observer.preListNamespaceDescriptors(new ObserverContext<MasterCoprocessorEnvironment>(),
+                    Arrays.asList(NamespaceDescriptor.create(schemaName).build()));
+        }
+    }
+
+    @Override
+    public void preCreateSchema(ObserverContext<PhoenixMetaDataControllerEnvironment> ctx, String schemaName)
+            throws IOException {
+        if (!accessCheckEnabled) { return; }
+        for (BaseMasterAndRegionObserver observer : getAccessControllers()) {
+            observer.preCreateNamespace(new ObserverContext<MasterCoprocessorEnvironment>(),
+                    NamespaceDescriptor.create(schemaName).build());
+        }
+    }
+
+    @Override
+    public void preDropSchema(ObserverContext<PhoenixMetaDataControllerEnvironment> ctx, String schemaName)
+            throws IOException {
+        if (!accessCheckEnabled) { return; }
+        for (BaseMasterAndRegionObserver observer : getAccessControllers()) {
+            observer.preDeleteNamespace(new ObserverContext<MasterCoprocessorEnvironment>(), schemaName);
+        }
+    }
+
+    @Override
+    public void preIndexUpdate(ObserverContext<PhoenixMetaDataControllerEnvironment> ctx, String tenantId,
+            String indexName, TableName physicalTableName, TableName parentPhysicalTableName, PIndexState newState)
+            throws IOException {
+        if (!accessCheckEnabled) { return; }
+        for (BaseMasterAndRegionObserver observer : getAccessControllers()) {
+            observer.preModifyTable(new ObserverContext<MasterCoprocessorEnvironment>(), physicalTableName,
+                    new HTableDescriptor(physicalTableName));
+        }
+        // Check for read access in case of rebuild
+        if (newState == PIndexState.BUILDING) {
+            requireAccess("Rebuild:", parentPhysicalTableName, Action.READ, Action.EXEC);
+        }
+    }
+
+    private List<UserPermission> getUserPermissions(final String tableName) throws IOException {
+        return User.runAsLoginUser(new PrivilegedExceptionAction<List<UserPermission>>() {
+            @Override
+            public List<UserPermission> run() throws Exception {
+                final List<UserPermission> userPermissions = new ArrayList<UserPermission>();
+                try (Connection connection = ConnectionFactory.createConnection(env.getConfiguration())) {
+                    for (BaseMasterAndRegionObserver service : accessControllers) {
+                        if (service.getClass().getName().equals(org.apache.hadoop.hbase.security.access.AccessController.class.getName())) {
+                            userPermissions.addAll(AccessControlClient.getUserPermissions(connection, tableName));
+                        } else {
+                            AccessControlProtos.GetUserPermissionsRequest.Builder builder = AccessControlProtos.GetUserPermissionsRequest
+                                    .newBuilder();
+                            builder.setTableName(ProtobufUtil.toProtoTableName(TableName.valueOf(tableName)));
+                            builder.setType(AccessControlProtos.Permission.Type.Table);
+                            AccessControlProtos.GetUserPermissionsRequest request = builder.build();
+
+                            PayloadCarryingRpcController controller = ((ClusterConnection)connection)
+                                    .getRpcControllerFactory().newController();
+                            ((AccessControlService.Interface)service).getUserPermissions(controller, request,
+                                    new RpcCallback<AccessControlProtos.GetUserPermissionsResponse>() {
+                                        @Override
+                                        public void run(AccessControlProtos.GetUserPermissionsResponse message) {
+                                            if (message != null) {
+                                                for (AccessControlProtos.UserPermission perm : message
+                                                        .getUserPermissionList()) {
+                                                    userPermissions.add(ProtobufUtil.toUserPermission(perm));
+                                                }
+                                            }
+                                        }
+                                    });
+                        }
+                    }
+                } catch (Throwable e) {
+                    if (e instanceof Exception) {
+                        throw (Exception) e;
+                    } else if (e instanceof Error) {
+                        throw (Error) e;
+                    }
+                    throw new Exception(e);
+                }
+                return userPermissions;
+            }
+        });
+    }
+    
+    /**
+     * Authorizes that the current user has all the given permissions for the
+     * given table
+     * @param tableName Table requested
+     * @throws IOException if obtaining the current user fails
+     * @throws AccessDeniedException if user has no authorization
+     */
+    private void requireAccess(String request, TableName tableName, Action... permissions) throws IOException {
+        User user = getActiveUser();
+        AuthResult result = null;
+        List<Action> requiredAccess = new ArrayList<Action>();
+        for (Action permission : permissions) {
+            if (hasAccess(getUserPermissions(tableName.getNameAsString()), tableName, permission, user)) {
+                result = AuthResult.allow(request, "Table permission granted", user, permission, tableName, null, null);
+            } else {
+                result = AuthResult.deny(request, "Insufficient permissions", user, permission, tableName, null, null);
+                requiredAccess.add(permission);
+            }
+            logResult(result);
+        }
+        if (!requiredAccess.isEmpty()) {
+            result = AuthResult.deny(request, "Insufficient permissions", user, requiredAccess.get(0), tableName, null,
+                    null);
+        }
+        if (!result.isAllowed()) { throw new AccessDeniedException("Insufficient permissions "
+                + authString(user.getName(), tableName, new HashSet<Permission.Action>(Arrays.asList(permissions)))); }
+    }
+
+    /**
+     * Checks if the user has access to the table for the specified action.
+     *
+     * @param perms All table permissions
+     * @param table tablename
+     * @param action action for access is required
+     * @return true if the user has access to the table for specified action, false otherwise
+     */
+    private boolean hasAccess(List<UserPermission> perms, TableName table, Permission.Action action, User user) {
+        if (Superusers.isSuperUser(user)){
+            return true;
+        }
+        if (perms != null) {
+            List<UserPermission> permissionsForUser = getPermissionForUser(perms, user.getShortName().getBytes());
+            if (permissionsForUser != null) {
+                for (UserPermission permissionForUser : permissionsForUser) {
+                    if (permissionForUser.implies(action)) { return true; }
+                }
+            }
+            String[] groupNames = user.getGroupNames();
+            if (groupNames != null) {
+              for (String group : groupNames) {
+                List<UserPermission> groupPerms = getPermissionForUser(perms,(AuthUtil.toGroupEntry(group)).getBytes());
+                if (groupPerms != null) for (UserPermission permissionForUser : groupPerms) {
+                    if (permissionForUser.implies(action)) { return true; }
+                }
+              }
+            }
+        } else if (LOG.isDebugEnabled()) {
+            LOG.debug("No permissions found for table=" + table);
+        }
+        return false;
+    }
+
+    private User getActiveUser() throws IOException {
+        User user = RpcServer.getRequestUser();
+        if (user == null) {
+            // for non-rpc handling, fallback to system user
+            user = userProvider.getCurrent();
+        }
+        return user;
+    }
+
+    private void logResult(AuthResult result) {
+        if (AUDITLOG.isTraceEnabled()) {
+            InetAddress remoteAddr = RpcServer.getRemoteAddress();
+            AUDITLOG.trace("Access " + (result.isAllowed() ? "allowed" : "denied") + " for user "
+                    + (result.getUser() != null ? result.getUser().getShortName() : "UNKNOWN") + "; reason: "
+                    + result.getReason() + "; remote address: " + (remoteAddr != null ? remoteAddr : "") + "; request: "
+                    + result.getRequest() + "; context: " + result.toContextString());
+        }
+    }
+
+    private static final class Superusers {
+        private static final Log LOG = LogFactory.getLog(Superusers.class);
+
+        /** Configuration key for superusers */
+        public static final String SUPERUSER_CONF_KEY = org.apache.hadoop.hbase.security.Superusers.SUPERUSER_CONF_KEY; // Not getting a name
+
+        private static List<String> superUsers;
+        private static List<String> superGroups;
+        private static User systemUser;
+
+        private Superusers(){}
+
+        /**
+         * Should be called only once to pre-load list of super users and super
+         * groups from Configuration. This operation is idempotent.
+         * @param conf configuration to load users from
+         * @throws IOException if unable to initialize lists of superusers or super groups
+         * @throws IllegalStateException if current user is null
+         */
+        public static void initialize(Configuration conf) throws IOException {
+            superUsers = new ArrayList<>();
+            superGroups = new ArrayList<>();
+            systemUser = User.getCurrent();
+
+            if (systemUser == null) {
+                throw new IllegalStateException("Unable to obtain the current user, "
+                    + "authorization checks for internal operations will not work correctly!");
+            }
+
+            if (LOG.isTraceEnabled()) {
+                LOG.trace("Current user name is " + systemUser.getShortName());
+            }
+            String currentUser = systemUser.getShortName();
+            String[] superUserList = conf.getStrings(SUPERUSER_CONF_KEY, new String[0]);
+            for (String name : superUserList) {
+                if (AuthUtil.isGroupPrincipal(name)) {
+                    superGroups.add(AuthUtil.getGroupName(name));
+                } else {
+                    superUsers.add(name);
+                }
+            }
+            superUsers.add(currentUser);
+        }
+
+        /**
+         * @return true if current user is a super user (whether as user running process,
+         * declared as individual superuser or member of supergroup), false otherwise.
+         * @param user to check
+         * @throws IllegalStateException if lists of superusers/super groups
+         *   haven't been initialized properly
+         */
+        public static boolean isSuperUser(User user) {
+            if (superUsers == null) {
+                throw new IllegalStateException("Super users/super groups lists"
+                    + " haven't been initialized properly.");
+            }
+            if (superUsers.contains(user.getShortName())) {
+                return true;
+            }
+
+            for (String group : user.getGroupNames()) {
+                if (superGroups.contains(group)) {
+                    return true;
+                }
+            }
+            return false;
+        }
+
+        public static List<String> getSuperUsers() {
+            return superUsers;
+        }
+
+        public static User getSystemUser() {
+            return systemUser;
+        }
+    }
+    
+    public String authString(String user, TableName table, Set<Action> actions) {
+        StringBuilder sb = new StringBuilder();
+        sb.append(" (user=").append(user != null ? user : "UNKNOWN").append(", ");
+        sb.append("scope=").append(table == null ? "GLOBAL" : table.getNameWithNamespaceInclAsString()).append(", ");
+        sb.append(actions.size() > 1 ? "actions=" : "action=").append(actions != null ? actions.toString() : "")
+                .append(")");
+        return sb.toString();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/493a54f9/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/PhoenixMetaDataCoprocessorHost.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/PhoenixMetaDataCoprocessorHost.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/PhoenixMetaDataCoprocessorHost.java
new file mode 100644
index 0000000..15b0020
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/PhoenixMetaDataCoprocessorHost.java
@@ -0,0 +1,236 @@
+/*
+ * 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.coprocessor;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentMap;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Coprocessor;
+import org.apache.hadoop.hbase.CoprocessorEnvironment;
+import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.regionserver.Region;
+import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost;
+import org.apache.hadoop.hbase.regionserver.RegionServerServices;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.query.QueryServicesOptions;
+import org.apache.phoenix.schema.PIndexState;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTableType;
+
+public class PhoenixMetaDataCoprocessorHost
+        extends CoprocessorHost<PhoenixMetaDataCoprocessorHost.PhoenixMetaDataControllerEnvironment> {
+    private RegionCoprocessorEnvironment env;
+    public static final String PHOENIX_META_DATA_COPROCESSOR_CONF_KEY =
+            "hbase.coprocessor.phoenix.classes";
+    public static final String DEFAULT_PHOENIX_META_DATA_COPROCESSOR_CONF_KEY="org.apache.phoenix.coprocessor.PhoenixAccessController";
+
+    public PhoenixMetaDataCoprocessorHost(RegionCoprocessorEnvironment env) {
+        super(null);
+        this.env = env;
+        this.conf = env.getConfiguration();
+        boolean accessCheckEnabled = this.conf.getBoolean(QueryServices.PHOENIX_ACLS_ENABLED,
+                QueryServicesOptions.DEFAULT_PHOENIX_ACLS_ENABLED);
+        if (this.conf.get(PHOENIX_META_DATA_COPROCESSOR_CONF_KEY) == null && accessCheckEnabled) {
+            this.conf.set(PHOENIX_META_DATA_COPROCESSOR_CONF_KEY, DEFAULT_PHOENIX_META_DATA_COPROCESSOR_CONF_KEY);
+        }
+        loadSystemCoprocessors(conf, PHOENIX_META_DATA_COPROCESSOR_CONF_KEY);
+    }
+
+    private static abstract class CoprocessorOperation<T extends CoprocessorEnvironment> extends ObserverContext<T> {
+        abstract void call(MetaDataEndpointObserver oserver, ObserverContext<T> ctx) throws IOException;
+
+        public void postEnvCall(T env) {}
+    }
+
+    private boolean execOperation(
+            final CoprocessorOperation<PhoenixMetaDataCoprocessorHost.PhoenixMetaDataControllerEnvironment> ctx)
+            throws IOException {
+        if (ctx == null) return false;
+        boolean bypass = false;
+        for (PhoenixMetaDataControllerEnvironment env : coprocessors) {
+            if (env.getInstance() instanceof MetaDataEndpointObserver) {
+                ctx.prepare(env);
+                Thread currentThread = Thread.currentThread();
+                ClassLoader cl = currentThread.getContextClassLoader();
+                try {
+                    currentThread.setContextClassLoader(env.getClassLoader());
+                    ctx.call((MetaDataEndpointObserver)env.getInstance(), ctx);
+                } catch (Throwable e) {
+                    handleCoprocessorThrowable(env, e);
+                } finally {
+                    currentThread.setContextClassLoader(cl);
+                }
+                bypass |= ctx.shouldBypass();
+                if (ctx.shouldComplete()) {
+                    break;
+                }
+            }
+            ctx.postEnvCall(env);
+        }
+        return bypass;
+    }
+    
+    @Override
+    protected void handleCoprocessorThrowable(final CoprocessorEnvironment env, final Throwable e) throws IOException {
+        if (e instanceof IOException) {
+            if (e.getCause() instanceof DoNotRetryIOException) { throw (IOException)e.getCause(); }
+        }
+        super.handleCoprocessorThrowable(env, e);
+    }
+
+    /**
+     * Encapsulation of the environment of each coprocessor
+     */
+    static class PhoenixMetaDataControllerEnvironment extends CoprocessorHost.Environment
+            implements RegionCoprocessorEnvironment {
+
+        private RegionCoprocessorEnvironment env;
+
+        public PhoenixMetaDataControllerEnvironment(RegionCoprocessorEnvironment env, Coprocessor instance,
+                int priority, int sequence, Configuration conf) {
+            super(instance, priority, sequence, conf);
+            this.env = env;
+        }
+
+        @Override
+        public RegionServerServices getRegionServerServices() {
+            return env.getRegionServerServices();
+        }
+
+        public RegionCoprocessorHost getCoprocessorHost() {
+            return env.getRegion().getCoprocessorHost();
+        }
+
+        @Override
+        public Region getRegion() {
+            return env.getRegion();
+        }
+
+        @Override
+        public HRegionInfo getRegionInfo() {
+            return env.getRegionInfo();
+        }
+
+        @Override
+        public ConcurrentMap<String, Object> getSharedData() {
+            return env.getSharedData();
+        }
+    }
+
+    @Override
+    public PhoenixMetaDataControllerEnvironment createEnvironment(Class<?> implClass, Coprocessor instance,
+            int priority, int sequence, Configuration conf) {
+        return new PhoenixMetaDataControllerEnvironment(env, instance, priority, sequence, conf);
+    }
+
+    public void preGetTable(final String tenantId, final String tableName, final TableName physicalTableName)
+            throws IOException {
+        execOperation(new CoprocessorOperation<PhoenixMetaDataControllerEnvironment>() {
+            @Override
+            public void call(MetaDataEndpointObserver observer,
+                    ObserverContext<PhoenixMetaDataControllerEnvironment> ctx) throws IOException {
+                observer.preGetTable(ctx, tenantId, tableName, physicalTableName);
+            }
+        });
+    }
+
+    public void preCreateTable(final String tenantId, final String tableName, final TableName physicalTableName,
+            final TableName parentPhysicalTableName, final PTableType tableType, final Set<byte[]> familySet, final Set<TableName> indexes)
+            throws IOException {
+        execOperation(new CoprocessorOperation<PhoenixMetaDataControllerEnvironment>() {
+            @Override
+            public void call(MetaDataEndpointObserver observer,
+                    ObserverContext<PhoenixMetaDataControllerEnvironment> ctx) throws IOException {
+                observer.preCreateTable(ctx, tenantId, tableName, physicalTableName, parentPhysicalTableName, tableType,
+                        familySet, indexes);
+            }
+        });
+    }
+
+    public void preDropTable(final String tenantId, final String tableName, final TableName physicalTableName,
+            final TableName parentPhysicalTableName, final PTableType tableType, final List<PTable> indexes) throws IOException {
+        execOperation(new CoprocessorOperation<PhoenixMetaDataControllerEnvironment>() {
+            @Override
+            public void call(MetaDataEndpointObserver observer,
+                    ObserverContext<PhoenixMetaDataControllerEnvironment> ctx) throws IOException {
+                observer.preDropTable(ctx, tenantId, tableName, physicalTableName, parentPhysicalTableName, tableType, indexes);
+            }
+        });
+    }
+
+    public void preAlterTable(final String tenantId, final String tableName, final TableName physicalTableName,
+            final TableName parentPhysicalTableName, final PTableType type) throws IOException {
+        execOperation(new CoprocessorOperation<PhoenixMetaDataControllerEnvironment>() {
+            @Override
+            public void call(MetaDataEndpointObserver observer,
+                    ObserverContext<PhoenixMetaDataControllerEnvironment> ctx) throws IOException {
+                observer.preAlterTable(ctx, tenantId, tableName, physicalTableName, parentPhysicalTableName, type);
+            }
+        });
+    }
+
+    public void preGetSchema(final String schemaName) throws IOException {
+        execOperation(new CoprocessorOperation<PhoenixMetaDataControllerEnvironment>() {
+            @Override
+            public void call(MetaDataEndpointObserver observer,
+                    ObserverContext<PhoenixMetaDataControllerEnvironment> ctx) throws IOException {
+                observer.preGetSchema(ctx, schemaName);
+            }
+        });
+    }
+
+    public void preCreateSchema(final String schemaName) throws IOException {
+
+        execOperation(new CoprocessorOperation<PhoenixMetaDataControllerEnvironment>() {
+            @Override
+            public void call(MetaDataEndpointObserver observer,
+                    ObserverContext<PhoenixMetaDataControllerEnvironment> ctx) throws IOException {
+                observer.preCreateSchema(ctx, schemaName);
+            }
+        });
+    }
+
+    public void preDropSchema(final String schemaName) throws IOException {
+        execOperation(new CoprocessorOperation<PhoenixMetaDataControllerEnvironment>() {
+            @Override
+            public void call(MetaDataEndpointObserver observer,
+                    ObserverContext<PhoenixMetaDataControllerEnvironment> ctx) throws IOException {
+                observer.preDropSchema(ctx, schemaName);
+            }
+        });
+    }
+
+    public void preIndexUpdate(final String tenantId, final String indexName, final TableName physicalTableName,
+            final TableName parentPhysicalTableName, final PIndexState newState) throws IOException {
+        execOperation(new CoprocessorOperation<PhoenixMetaDataControllerEnvironment>() {
+            @Override
+            public void call(MetaDataEndpointObserver observer,
+                    ObserverContext<PhoenixMetaDataControllerEnvironment> ctx) throws IOException {
+                observer.preIndexUpdate(ctx, tenantId, indexName, physicalTableName, parentPhysicalTableName, newState);
+            }
+        });
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/493a54f9/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexFailurePolicy.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexFailurePolicy.java b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexFailurePolicy.java
index 0fc138f..ba6371b 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
@@ -18,6 +18,7 @@
 package org.apache.phoenix.index;
 
 import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
 import java.sql.SQLException;
 import java.util.Collection;
 import java.util.Collections;
@@ -39,6 +40,7 @@ import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.coprocessor.MetaDataProtocol.MetaDataMutationResult;
 import org.apache.phoenix.coprocessor.MetaDataProtocol.MutationCode;
@@ -161,12 +163,12 @@ public class PhoenixIndexFailurePolicy extends DelegateIndexFailurePolicy {
     }
 
     private long handleFailureWithExceptions(Multimap<HTableInterfaceReference, Mutation> attempted,
-            Exception cause) throws Throwable {
+            final Exception cause) throws Throwable {
         Set<HTableInterfaceReference> refs = attempted.asMap().keySet();
-        Map<String, Long> indexTableNames = new HashMap<String, Long>(refs.size());
+        final Map<String, Long> indexTableNames = new HashMap<String, Long>(refs.size());
         // start by looking at all the tables to which we attempted to write
         long timestamp = 0;
-        boolean leaveIndexActive = blockDataTableWritesOnFailure || !disableIndexOnFailure;
+        final boolean leaveIndexActive = blockDataTableWritesOnFailure || !disableIndexOnFailure;
         // if using TrackingParallelWriter, we know which indexes failed and only disable those
         Set<HTableInterfaceReference> failedTables = cause instanceof MultiIndexWriteFailureException 
                 ? new HashSet<HTableInterfaceReference>(((MultiIndexWriteFailureException)cause).getFailedTables())
@@ -210,55 +212,66 @@ public class PhoenixIndexFailurePolicy extends DelegateIndexFailurePolicy {
             return timestamp;
         }
 
-        PIndexState newState = disableIndexOnFailure ? PIndexState.DISABLE : PIndexState.PENDING_ACTIVE;
+        final PIndexState newState = disableIndexOnFailure ? PIndexState.DISABLE : PIndexState.PENDING_ACTIVE;
+        final long fTimestamp=timestamp;
         // for all the index tables that we've found, try to disable them and if that fails, try to
-        for (Map.Entry<String, Long> tableTimeElement :indexTableNames.entrySet()){
-            String indexTableName = tableTimeElement.getKey();
-            long minTimeStamp = tableTimeElement.getValue();
-            // We need a way of differentiating the block writes to data table case from
-            // the leave index active case. In either case, we need to know the time stamp
-            // at which writes started failing so we can rebuild from that point. If we
-            // keep the index active *and* have a positive INDEX_DISABLE_TIMESTAMP_BYTES,
-            // then writes to the data table will be blocked (this is client side logic
-            // and we can't change this in a minor release). So we use the sign of the
-            // time stamp to differentiate.
-            if (!disableIndexOnFailure && !blockDataTableWritesOnFailure) {
-                minTimeStamp *= -1;
-            }
-            // Disable the index by using the updateIndexState method of MetaDataProtocol end point coprocessor.
-            try (HTableInterface systemTable = env.getTable(SchemaUtil
-                    .getPhysicalTableName(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES, env.getConfiguration()))) {
-                MetaDataMutationResult result = IndexUtil.updateIndexState(indexTableName, minTimeStamp,
-                        systemTable, newState);
-                if (result.getMutationCode() == MutationCode.TABLE_NOT_FOUND) {
-                    LOG.info("Index " + indexTableName + " has been dropped. Ignore uncommitted mutations");
-                    continue;
-                }
-                if (result.getMutationCode() != MutationCode.TABLE_ALREADY_EXISTS) {
-                    if (leaveIndexActive) {
-                        LOG.warn("Attempt to update INDEX_DISABLE_TIMESTAMP " + " failed with code = "
-                                + result.getMutationCode());
-                        // If we're not disabling the index, then we don't want to throw as throwing
-                        // will lead to the RS being shutdown.
-                        if (blockDataTableWritesOnFailure) {
-                            throw new DoNotRetryIOException("Attempt to update INDEX_DISABLE_TIMESTAMP failed.");
+        return User.runAsLoginUser(new PrivilegedExceptionAction<Long>() {
+            @Override
+            public Long run() throws Exception {
+                for (Map.Entry<String, Long> tableTimeElement : indexTableNames.entrySet()) {
+                    String indexTableName = tableTimeElement.getKey();
+                    long minTimeStamp = tableTimeElement.getValue();
+                    // We need a way of differentiating the block writes to data table case from
+                    // the leave index active case. In either case, we need to know the time stamp
+                    // at which writes started failing so we can rebuild from that point. If we
+                    // keep the index active *and* have a positive INDEX_DISABLE_TIMESTAMP_BYTES,
+                    // then writes to the data table will be blocked (this is client side logic
+                    // and we can't change this in a minor release). So we use the sign of the
+                    // time stamp to differentiate.
+                    if (!disableIndexOnFailure && !blockDataTableWritesOnFailure) {
+                        minTimeStamp *= -1;
+                    }
+                    // Disable the index by using the updateIndexState method of MetaDataProtocol end point coprocessor.
+                    try (HTableInterface systemTable = env.getTable(SchemaUtil.getPhysicalTableName(
+                            PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES, env.getConfiguration()))) {
+                        MetaDataMutationResult result = IndexUtil.updateIndexState(indexTableName, minTimeStamp,
+                                systemTable, newState);
+                        if (result.getMutationCode() == MutationCode.TABLE_NOT_FOUND) {
+                            LOG.info("Index " + indexTableName + " has been dropped. Ignore uncommitted mutations");
+                            continue;
+                        }
+                        if (result.getMutationCode() != MutationCode.TABLE_ALREADY_EXISTS) {
+                            if (leaveIndexActive) {
+                                LOG.warn("Attempt to update INDEX_DISABLE_TIMESTAMP " + " failed with code = "
+                                        + result.getMutationCode());
+                                // If we're not disabling the index, then we don't want to throw as throwing
+                                // will lead to the RS being shutdown.
+                                if (blockDataTableWritesOnFailure) { throw new DoNotRetryIOException(
+                                        "Attempt to update INDEX_DISABLE_TIMESTAMP failed."); }
+                            } else {
+                                LOG.warn("Attempt to disable index " + indexTableName + " failed with code = "
+                                        + result.getMutationCode() + ". Will use default failure policy instead.");
+                                throw new DoNotRetryIOException("Attempt to disable " + indexTableName + " failed.");
+                            }
+                        }
+                        if (leaveIndexActive)
+                            LOG.info("Successfully update INDEX_DISABLE_TIMESTAMP for " + indexTableName
+                                    + " due to an exception while writing updates.", cause);
+                        else
+                            LOG.info("Successfully disabled index " + indexTableName
+                                    + " due to an exception while writing updates.", cause);
+                    } catch (Throwable t) {
+                        if (t instanceof Exception) {
+                            throw (Exception)t;
+                        } else {
+                            throw new Exception(t);
                         }
-                    } else {
-                        LOG.warn("Attempt to disable index " + indexTableName + " failed with code = "
-                                + result.getMutationCode() + ". Will use default failure policy instead.");
-                        throw new DoNotRetryIOException("Attempt to disable " + indexTableName + " failed.");
-                    } 
+                    }
                 }
-                if (leaveIndexActive)
-                    LOG.info("Successfully update INDEX_DISABLE_TIMESTAMP for " + indexTableName + " due to an exception while writing updates.",
-                            cause);
-                else
-                    LOG.info("Successfully disabled index " + indexTableName + " due to an exception while writing updates.",
-                            cause);
+                // Return the cell time stamp (note they should all be the same)
+                return fTimestamp;
             }
-        }
-        // Return the cell time stamp (note they should all be the same)
-        return timestamp;
+        });
     }
 
     private Collection<? extends String> getLocalIndexNames(HTableInterfaceReference ref,

http://git-wip-us.apache.org/repos/asf/phoenix/blob/493a54f9/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
index 874ac9c..7a255a1 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
@@ -867,7 +867,8 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     }
                 }
             }
-            if (SchemaUtil.isStatsTable(tableName) && !descriptor.hasCoprocessor(MultiRowMutationEndpoint.class.getName())) {
+            if ((SchemaUtil.isStatsTable(tableName) || SchemaUtil.isMetaTable(tableName))
+                    && !descriptor.hasCoprocessor(MultiRowMutationEndpoint.class.getName())) {
                 descriptor.addCoprocessor(MultiRowMutationEndpoint.class.getName(),
                         null, priority, null);
             }
@@ -2529,7 +2530,8 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             final TableName mutexTableName = SchemaUtil.getPhysicalTableName(
                 PhoenixDatabaseMetaData.SYSTEM_MUTEX_NAME, props);
             List<TableName> systemTables = getSystemTableNames(admin);
-            if (systemTables.contains(mutexTableName)) {
+            if (systemTables.contains(mutexTableName) || admin.tableExists( TableName.valueOf(
+                PhoenixDatabaseMetaData.SYSTEM_SCHEMA_NAME,PhoenixDatabaseMetaData.SYSTEM_MUTEX_TABLE_NAME))) {
                 logger.debug("System mutex table already appears to exist, not creating it");
                 return;
             }
@@ -2546,8 +2548,15 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 put.add(PhoenixDatabaseMetaData.SYSTEM_MUTEX_FAMILY_NAME_BYTES, UPGRADE_MUTEX, UPGRADE_MUTEX_UNLOCKED);
                 sysMutexTable.put(put);
             }
-        } catch (TableExistsException e) {
+        } catch (TableExistsException | AccessDeniedException e) {
             // Ignore
+        }catch(PhoenixIOException e){
+            if(e.getCause()!=null && e.getCause() instanceof AccessDeniedException)
+            {
+                //Ignore
+            }else{
+                throw e;
+            }
         }
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/493a54f9/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
index a4a4124..b9ed734 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
@@ -260,6 +260,10 @@ public interface QueryServices extends SQLCloseable {
     
     //currently BASE64 and ASCII is supported
     public static final String UPLOAD_BINARY_DATA_TYPE_ENCODING = "phoenix.upload.binaryDataType.encoding";
+    // Toggle for server-written updates to SYSTEM.CATALOG
+    public static final String PHOENIX_ACLS_ENABLED = "phoenix.acls.enabled";
+    public static final String PHOENIX_AUTOMATIC_GRANT_ENABLED = "phoenix.security.automatic.grant.enabled";
+    public static final String PHOENIX_SECURITY_PERMISSION_STRICT_MODE_ENABLED = "phoenix.security.strict.mode.enabled";
 
     public static final String INDEX_ASYNC_BUILD_ENABLED = "phoenix.index.async.build.enabled";
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/493a54f9/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
index af6a054..a586c28 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
@@ -58,11 +58,14 @@ import static org.apache.phoenix.query.QueryServices.MAX_TENANT_MEMORY_PERC_ATTR
 import static org.apache.phoenix.query.QueryServices.MIN_STATS_UPDATE_FREQ_MS_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.MUTATE_BATCH_SIZE_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.NUM_RETRIES_FOR_SCHEMA_UPDATE_CHECK;
+import static org.apache.phoenix.query.QueryServices.PHOENIX_ACLS_ENABLED;
+import static org.apache.phoenix.query.QueryServices.PHOENIX_AUTOMATIC_GRANT_ENABLED;
 import static org.apache.phoenix.query.QueryServices.PHOENIX_QUERY_SERVER_CLUSTER_BASE_PATH;
 import static org.apache.phoenix.query.QueryServices.PHOENIX_QUERY_SERVER_LOADBALANCER_ENABLED;
 import static org.apache.phoenix.query.QueryServices.PHOENIX_QUERY_SERVER_SERVICE_NAME;
 import static org.apache.phoenix.query.QueryServices.PHOENIX_QUERY_SERVER_ZK_ACL_PASSWORD;
 import static org.apache.phoenix.query.QueryServices.PHOENIX_QUERY_SERVER_ZK_ACL_USERNAME;
+import static org.apache.phoenix.query.QueryServices.PHOENIX_SECURITY_PERMISSION_STRICT_MODE_ENABLED;
 import static org.apache.phoenix.query.QueryServices.QUEUE_SIZE_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.REGIONSERVER_INFO_PORT_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.RENEW_LEASE_ENABLED;
@@ -316,6 +319,11 @@ public class QueryServicesOptions {
     public static final int DEFAULT_CLIENT_CONNECTION_MAX_ALLOWED_CONNECTIONS = 0;
     public static final boolean DEFAULT_STATS_COLLECTION_ENABLED = true;
     public static final boolean DEFAULT_USE_STATS_FOR_PARALLELIZATION = true;
+    
+    //Security defaults
+    public static final boolean DEFAULT_PHOENIX_ACLS_ENABLED = false;
+    public static final boolean DEFAULT_PHOENIX_AUTOMATIC_GRANT_ENABLED = false;
+    public static final boolean DEFAULT_PHOENIX_SECURITY_PERMISSION_STRICT_MODE_ENABLED = true;
 
     //default update cache frequency
     public static final int DEFAULT_UPDATE_CACHE_FREQUENCY = 0;
@@ -413,7 +421,11 @@ public class QueryServicesOptions {
             .setIfUnset(TRACING_BATCH_SIZE, DEFAULT_TRACING_BATCH_SIZE)
             .setIfUnset(TRACING_THREAD_POOL_SIZE, DEFAULT_TRACING_THREAD_POOL_SIZE)
             .setIfUnset(STATS_COLLECTION_ENABLED, DEFAULT_STATS_COLLECTION_ENABLED)
-            .setIfUnset(USE_STATS_FOR_PARALLELIZATION, DEFAULT_USE_STATS_FOR_PARALLELIZATION);
+            .setIfUnset(USE_STATS_FOR_PARALLELIZATION, DEFAULT_USE_STATS_FOR_PARALLELIZATION)
+            .setIfUnset(UPLOAD_BINARY_DATA_TYPE_ENCODING, DEFAULT_UPLOAD_BINARY_DATA_TYPE_ENCODING)
+            .setIfUnset(PHOENIX_ACLS_ENABLED,  DEFAULT_PHOENIX_ACLS_ENABLED)
+            .setIfUnset(PHOENIX_AUTOMATIC_GRANT_ENABLED,  DEFAULT_PHOENIX_AUTOMATIC_GRANT_ENABLED)
+            .setIfUnset(PHOENIX_SECURITY_PERMISSION_STRICT_MODE_ENABLED,  DEFAULT_PHOENIX_SECURITY_PERMISSION_STRICT_MODE_ENABLED);
         // HBase sets this to 1, so we reset it to something more appropriate.
         // Hopefully HBase will change this, because we can't know if a user set
         // it to 1, so we'll change it.

http://git-wip-us.apache.org/repos/asf/phoenix/blob/493a54f9/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsWriter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsWriter.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsWriter.java
index 3ae3183..8956862 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsWriter.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsWriter.java
@@ -25,6 +25,7 @@ import java.io.DataInput;
 import java.io.DataInputStream;
 import java.io.EOFException;
 import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
 import java.sql.Date;
 import java.util.ArrayList;
 import java.util.List;
@@ -46,6 +47,7 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.Mut
 import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MultiRowMutationService;
 import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MutateRowsRequest;
 import org.apache.hadoop.hbase.regionserver.Region;
+import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.coprocessor.MetaDataProtocol;
 import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
@@ -208,23 +210,31 @@ public class StatisticsWriter implements Closeable {
         }
     }
 
-    public void commitStats(List<Mutation> mutations, StatisticsCollector statsCollector) throws IOException {
-        commitLastStatsUpdatedTime(statsCollector);
-        if (mutations.size() > 0) {
-            byte[] row = mutations.get(0).getRow();
-            MutateRowsRequest.Builder mrmBuilder = MutateRowsRequest.newBuilder();
-            for (Mutation m : mutations) {
-                mrmBuilder.addMutationRequest(ProtobufUtil.toMutation(getMutationType(m), m));
-            }
-            MutateRowsRequest mrm = mrmBuilder.build();
-            CoprocessorRpcChannel channel = statsWriterTable.coprocessorService(row);
-            MultiRowMutationService.BlockingInterface service = MultiRowMutationService.newBlockingStub(channel);
-            try {
-                service.mutateRows(null, mrm);
-            } catch (ServiceException ex) {
-                ProtobufUtil.toIOException(ex);
+    public void commitStats(final List<Mutation> mutations, final StatisticsCollector statsCollector)
+            throws IOException {
+        User.runAsLoginUser(new PrivilegedExceptionAction<Void>() {
+            @Override
+            public Void run() throws Exception {
+                commitLastStatsUpdatedTime(statsCollector);
+                if (mutations.size() > 0) {
+                    byte[] row = mutations.get(0).getRow();
+                    MutateRowsRequest.Builder mrmBuilder = MutateRowsRequest.newBuilder();
+                    for (Mutation m : mutations) {
+                        mrmBuilder.addMutationRequest(ProtobufUtil.toMutation(getMutationType(m), m));
+                    }
+                    MutateRowsRequest mrm = mrmBuilder.build();
+                    CoprocessorRpcChannel channel = statsWriterTable.coprocessorService(row);
+                    MultiRowMutationService.BlockingInterface service = MultiRowMutationService
+                            .newBlockingStub(channel);
+                    try {
+                        service.mutateRows(null, mrm);
+                    } catch (ServiceException ex) {
+                        ProtobufUtil.toIOException(ex);
+                    }
+                }
+                return null;
             }
-        }
+        });
     }
 
     private Put getLastStatsUpdatedTimePut(long timeStamp) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/493a54f9/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java
----------------------------------------------------------------------
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 502ef37..2a0c8f0 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
@@ -59,6 +59,7 @@ import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PName;
 import org.apache.phoenix.schema.PNameFactory;
 import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTable.IndexType;
 import org.apache.phoenix.schema.PTable.LinkType;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.SequenceKey;
@@ -226,6 +227,16 @@ public class MetaDataUtil {
         }
         return null;
     }
+
+    public static boolean isNameSpaceMapped(List<Mutation> tableMetaData, KeyValueBuilder builder,
+            ImmutableBytesWritable value) {
+        if (getMutationValue(getPutOnlyTableHeaderRow(tableMetaData),
+            PhoenixDatabaseMetaData.IS_NAMESPACE_MAPPED_BYTES, builder, value)) {
+            return (boolean)PBoolean.INSTANCE.toObject(ByteUtil.copyKeyBytesIfNecessary(value));
+        }
+        return false;
+    }
+
     
     public static long getParentSequenceNumber(List<Mutation> tableMetaData) {
         return getSequenceNumber(getParentTableHeaderRow(tableMetaData));
@@ -670,4 +681,11 @@ public class MetaDataUtil {
         byte[] physicalTableName = Bytes.toBytes(SchemaUtil.getTableNameFromFullName(view.getPhysicalName().getString()));
         return SchemaUtil.getTableKey(ByteUtil.EMPTY_BYTE_ARRAY, physicalTableSchemaName, physicalTableName);
     }
+    
+    public static IndexType getIndexType(List<Mutation> tableMetaData, KeyValueBuilder builder,
+            ImmutableBytesWritable value) {
+        if (getMutationValue(getPutOnlyTableHeaderRow(tableMetaData), PhoenixDatabaseMetaData.INDEX_TYPE_BYTES, builder,
+                value)) { return IndexType.fromSerializedValue(value.get()[value.getOffset()]); }
+        return null;
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/493a54f9/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
index 51f6ff9..47b4b43 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
@@ -20,9 +20,11 @@ package org.apache.phoenix.util;
 import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Preconditions.checkNotNull;
 import static com.google.common.base.Strings.isNullOrEmpty;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IS_NAMESPACE_MAPPED_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_FUNCTION_NAME_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_STATS_NAME_BYTES;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES;
 
 import java.sql.Connection;
 import java.sql.DatabaseMetaData;
@@ -42,8 +44,10 @@ import java.util.TreeSet;
 import javax.annotation.Nullable;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -76,6 +80,7 @@ import org.apache.phoenix.schema.SaltingUtil;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.TableProperty;
 import org.apache.phoenix.schema.ValueSchema.Field;
+import org.apache.phoenix.schema.types.PBoolean;
 import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.types.PVarbinary;
 import org.apache.phoenix.schema.types.PVarchar;
@@ -1130,4 +1135,11 @@ public class SchemaUtil {
         }
         return false;
     }
+
+    public static boolean isNamespaceMapped(Result currentResult) {
+        Cell isNamespaceMappedCell = currentResult.getColumnLatestCell(TABLE_FAMILY_BYTES, IS_NAMESPACE_MAPPED_BYTES);
+        return isNamespaceMappedCell!=null && (boolean) PBoolean.INSTANCE.toObject(isNamespaceMappedCell.getValue());
+    }
+    
+
 }


[05/16] phoenix git commit: PHOENIX-4361: Remove redundant argument in separateAndValidateProperties in CQSI

Posted by ja...@apache.org.
PHOENIX-4361: Remove redundant argument in separateAndValidateProperties in CQSI

Signed-off-by: aertoria <ca...@gmail.com>


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

Branch: refs/heads/4.x-HBase-1.2
Commit: bab1e46eba7ccd6b1f21dcd62a0dfc40a90e97d1
Parents: a23439c
Author: Chinmay Kulkarni <ch...@gmail.com>
Authored: Thu Nov 16 02:31:20 2017 +0000
Committer: James Taylor <jt...@salesforce.com>
Committed: Sat Dec 16 16:42:54 2017 -0800

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


http://git-wip-us.apache.org/repos/asf/phoenix/blob/bab1e46e/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
index 532b586..874ac9c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
@@ -1721,7 +1721,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         Set<HTableDescriptor> tableDescriptors = Collections.emptySet();
         Set<HTableDescriptor> origTableDescriptors = Collections.emptySet();
         boolean nonTxToTx = false;
-        Pair<HTableDescriptor,HTableDescriptor> tableDescriptorPair = separateAndValidateProperties(table, stmtProperties, colFamiliesForPColumnsToBeAdded, families, tableProps);
+        Pair<HTableDescriptor,HTableDescriptor> tableDescriptorPair = separateAndValidateProperties(table, stmtProperties, colFamiliesForPColumnsToBeAdded, tableProps);
         HTableDescriptor tableDescriptor = tableDescriptorPair.getSecond();
         HTableDescriptor origTableDescriptor = tableDescriptorPair.getFirst();
         if (tableDescriptor != null) {
@@ -1939,7 +1939,8 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         this.addCoprocessors(tableDescriptor.getName(), tableDescriptor, tableType, tableProps);
     }
 
-    private Pair<HTableDescriptor,HTableDescriptor> separateAndValidateProperties(PTable table, Map<String, List<Pair<String, Object>>> properties, Set<String> colFamiliesForPColumnsToBeAdded, List<Pair<byte[], Map<String, Object>>> families, Map<String, Object> tableProps) throws SQLException {
+    private Pair<HTableDescriptor,HTableDescriptor> separateAndValidateProperties(PTable table, Map<String, List<Pair<String, Object>>> properties,
+      Set<String> colFamiliesForPColumnsToBeAdded, Map<String, Object> tableProps) throws SQLException {
         Map<String, Map<String, Object>> stmtFamiliesPropsMap = new HashMap<>(properties.size());
         Map<String,Object> commonFamilyProps = new HashMap<>();
         boolean addingColumns = colFamiliesForPColumnsToBeAdded != null && !colFamiliesForPColumnsToBeAdded.isEmpty();


[03/16] phoenix git commit: PHOENIX-4288 Indexes not used when ordering by primary key

Posted by ja...@apache.org.
PHOENIX-4288 Indexes not used when ordering by primary key


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

Branch: refs/heads/4.x-HBase-1.2
Commit: 597764c06e2ee5254856a9be58021f0363a9f765
Parents: bd72ce3
Author: maryannxue <ma...@gmail.com>
Authored: Sun Nov 5 02:37:55 2017 +0000
Committer: James Taylor <jt...@salesforce.com>
Committed: Sat Dec 16 16:42:54 2017 -0800

----------------------------------------------------------------------
 .../phoenix/end2end/CostBasedDecisionIT.java    | 466 +++++++++++++++++++
 .../apache/phoenix/end2end/MutationStateIT.java |  17 +
 .../apache/phoenix/end2end/SystemCatalogIT.java |   1 -
 .../phoenix/compile/ListJarsQueryPlan.java      |   6 +
 .../org/apache/phoenix/compile/QueryPlan.java   |   5 +-
 .../apache/phoenix/compile/TraceQueryPlan.java  |   6 +
 .../apache/phoenix/execute/AggregatePlan.java   |  30 +-
 .../apache/phoenix/execute/BaseQueryPlan.java   |  21 +-
 .../phoenix/execute/ClientAggregatePlan.java    |  28 ++
 .../apache/phoenix/execute/ClientScanPlan.java  |  25 +
 .../apache/phoenix/execute/CorrelatePlan.java   |  25 +
 .../phoenix/execute/DelegateQueryPlan.java      |   6 +
 .../apache/phoenix/execute/HashJoinPlan.java    |  29 ++
 .../execute/LiteralResultIterationPlan.java     |   6 +
 .../org/apache/phoenix/execute/ScanPlan.java    |  25 +
 .../phoenix/execute/SortMergeJoinPlan.java      |  18 +
 .../org/apache/phoenix/execute/UnionPlan.java   |  10 +
 .../apache/phoenix/jdbc/PhoenixStatement.java   |   6 +
 .../java/org/apache/phoenix/optimize/Cost.java  | 123 +++++
 .../apache/phoenix/optimize/QueryOptimizer.java |  30 +-
 .../org/apache/phoenix/query/QueryServices.java |   3 +
 .../phoenix/query/QueryServicesOptions.java     |   4 +
 .../java/org/apache/phoenix/util/CostUtil.java  |  90 ++++
 .../query/ParallelIteratorsSplitTest.java       |   6 +
 24 files changed, 971 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/597764c0/phoenix-core/src/it/java/org/apache/phoenix/end2end/CostBasedDecisionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CostBasedDecisionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CostBasedDecisionIT.java
new file mode 100644
index 0000000..a3584ce
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CostBasedDecisionIT.java
@@ -0,0 +1,466 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.end2end;
+
+import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.junit.Assert.assertTrue;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.util.Map;
+import java.util.Properties;
+
+import org.apache.phoenix.query.BaseTest;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.apache.phoenix.util.QueryUtil;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import com.google.common.collect.Maps;
+
+public class CostBasedDecisionIT extends BaseUniqueNamesOwnClusterIT {
+
+    @BeforeClass
+    public static void doSetup() throws Exception {
+        Map<String, String> props = Maps.newHashMapWithExpectedSize(1);
+        props.put(QueryServices.STATS_GUIDEPOST_WIDTH_BYTES_ATTRIB, Long.toString(20));
+        props.put(QueryServices.STATS_UPDATE_FREQ_MS_ATTRIB, Long.toString(5));
+        props.put(QueryServices.USE_STATS_FOR_PARALLELIZATION, Boolean.toString(true));
+        props.put(QueryServices.COST_BASED_OPTIMIZER_ENABLED, Boolean.toString(true));
+        setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
+    }
+
+    @Test
+    public void testCostOverridesStaticPlanOrdering1() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        conn.setAutoCommit(true);
+        try {
+            String tableName = BaseTest.generateUniqueName();
+            conn.createStatement().execute("CREATE TABLE " + tableName + " (\n" +
+                    "rowkey VARCHAR PRIMARY KEY,\n" +
+                    "c1 VARCHAR,\n" +
+                    "c2 VARCHAR)");
+            conn.createStatement().execute("CREATE LOCAL INDEX " + tableName + "_idx ON " + tableName + " (c1)");
+
+            String query = "SELECT rowkey, c1, c2 FROM " + tableName + " where c1 LIKE 'X0%' ORDER BY rowkey";
+            // Use the data table plan that opts out order-by when stats are not available.
+            ResultSet rs = conn.createStatement().executeQuery("explain " + query);
+            String plan = QueryUtil.getExplainPlan(rs);
+            assertTrue("Expected 'FULL SCAN' in the plan:\n" + plan + ".",
+                    plan.contains("FULL SCAN"));
+
+            PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " + tableName + " (rowkey, c1, c2) VALUES (?, ?, ?)");
+            for (int i = 0; i < 10000; i++) {
+                int c1 = i % 16;
+                stmt.setString(1, "k" + i);
+                stmt.setString(2, "X" + Integer.toHexString(c1) + c1);
+                stmt.setString(3, "c");
+                stmt.execute();
+            }
+
+            conn.createStatement().execute("UPDATE STATISTICS " + tableName);
+
+            // Use the index table plan that has a lower cost when stats become available.
+            rs = conn.createStatement().executeQuery("explain " + query);
+            plan = QueryUtil.getExplainPlan(rs);
+            assertTrue("Expected 'RANGE SCAN' in the plan:\n" + plan + ".",
+                    plan.contains("RANGE SCAN"));
+        } finally {
+            conn.close();
+        }
+    }
+
+    @Test
+    public void testCostOverridesStaticPlanOrdering2() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        conn.setAutoCommit(true);
+        try {
+            String tableName = BaseTest.generateUniqueName();
+            conn.createStatement().execute("CREATE TABLE " + tableName + " (\n" +
+                    "rowkey VARCHAR PRIMARY KEY,\n" +
+                    "c1 VARCHAR,\n" +
+                    "c2 VARCHAR)");
+            conn.createStatement().execute("CREATE LOCAL INDEX " + tableName + "_idx ON " + tableName + " (c1)");
+
+            String query = "SELECT rowkey, max(c1), max(c2) FROM " + tableName + " where c1 LIKE 'X%' GROUP BY rowkey";
+            // Use the index table plan that opts out order-by when stats are not available.
+            ResultSet rs = conn.createStatement().executeQuery("explain " + query);
+            String plan = QueryUtil.getExplainPlan(rs);
+            assertTrue("Expected 'RANGE SCAN' in the plan:\n" + plan + ".",
+                    plan.contains("RANGE SCAN"));
+
+            PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " + tableName + " (rowkey, c1, c2) VALUES (?, ?, ?)");
+            for (int i = 0; i < 10000; i++) {
+                int c1 = i % 16;
+                stmt.setString(1, "k" + i);
+                stmt.setString(2, "X" + Integer.toHexString(c1) + c1);
+                stmt.setString(3, "c");
+                stmt.execute();
+            }
+
+            conn.createStatement().execute("UPDATE STATISTICS " + tableName);
+
+            // Given that the range on C1 is meaningless and group-by becomes
+            // order-preserving if using the data table, the data table plan should
+            // come out as the best plan based on the costs.
+            rs = conn.createStatement().executeQuery("explain " + query);
+            plan = QueryUtil.getExplainPlan(rs);
+            assertTrue("Expected 'FULL SCAN' in the plan:\n" + plan + ".",
+                    plan.contains("FULL SCAN"));
+        } finally {
+            conn.close();
+        }
+    }
+
+    @Test
+    public void testCostOverridesStaticPlanOrdering3() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        conn.setAutoCommit(true);
+        try {
+            String tableName = BaseTest.generateUniqueName();
+            conn.createStatement().execute("CREATE TABLE " + tableName + " (\n" +
+                    "rowkey VARCHAR PRIMARY KEY,\n" +
+                    "c1 INTEGER,\n" +
+                    "c2 INTEGER,\n" +
+                    "c3 INTEGER)");
+            conn.createStatement().execute("CREATE LOCAL INDEX " + tableName + "_idx1 ON " + tableName + " (c1) INCLUDE (c2, c3)");
+            conn.createStatement().execute("CREATE LOCAL INDEX " + tableName + "_idx2 ON " + tableName + " (c2, c3) INCLUDE (c1)");
+
+            String query = "SELECT * FROM " + tableName + " where c1 BETWEEN 10 AND 20 AND c2 < 9000 AND C3 < 5000";
+            // Use the idx2 plan with a wider PK slot span when stats are not available.
+            ResultSet rs = conn.createStatement().executeQuery("explain " + query);
+            String plan = QueryUtil.getExplainPlan(rs);
+            String indexPlan =
+                    "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + tableName + " [2,*] - [2,9,000]\n" +
+                    "    SERVER FILTER BY ((\"C1\" >= 10 AND \"C1\" <= 20) AND TO_INTEGER(\"C3\") < 5000)\n" +
+                    "CLIENT MERGE SORT";
+            assertTrue("Expected '" + indexPlan + "' in the plan:\n" + plan + ".",
+                    plan.contains(indexPlan));
+
+            PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " + tableName + " (rowkey, c1, c2, c3) VALUES (?, ?, ?, ?)");
+            for (int i = 0; i < 10000; i++) {
+                stmt.setString(1, "k" + i);
+                stmt.setInt(2, i);
+                stmt.setInt(3, i);
+                stmt.setInt(4, i);
+                stmt.execute();
+            }
+
+            conn.createStatement().execute("UPDATE STATISTICS " + tableName);
+
+            // Use the idx2 plan that scans less data when stats become available.
+            rs = conn.createStatement().executeQuery("explain " + query);
+            plan = QueryUtil.getExplainPlan(rs);
+            String dataPlan =
+                    "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + tableName + " [1,10] - [1,20]\n" +
+                    "    SERVER FILTER BY (\"C2\" < 9000 AND \"C3\" < 5000)\n" +
+                    "CLIENT MERGE SORT";
+            assertTrue("Expected '" + dataPlan + "' in the plan:\n" + plan + ".",
+                    plan.contains(dataPlan));
+        } finally {
+            conn.close();
+        }
+    }
+
+    @Test
+    public void testCostOverridesStaticPlanOrderingInUpsertQuery() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        conn.setAutoCommit(true);
+        try {
+            String tableName = BaseTest.generateUniqueName();
+            conn.createStatement().execute("CREATE TABLE " + tableName + " (\n" +
+                    "rowkey VARCHAR PRIMARY KEY,\n" +
+                    "c1 INTEGER,\n" +
+                    "c2 INTEGER,\n" +
+                    "c3 INTEGER)");
+            conn.createStatement().execute("CREATE LOCAL INDEX " + tableName + "_idx1 ON " + tableName + " (c1) INCLUDE (c2, c3)");
+            conn.createStatement().execute("CREATE LOCAL INDEX " + tableName + "_idx2 ON " + tableName + " (c2, c3) INCLUDE (c1)");
+
+            String query = "UPSERT INTO " + tableName + " SELECT * FROM " + tableName + " where c1 BETWEEN 10 AND 20 AND c2 < 9000 AND C3 < 5000";
+            // Use the idx2 plan with a wider PK slot span when stats are not available.
+            ResultSet rs = conn.createStatement().executeQuery("explain " + query);
+            String plan = QueryUtil.getExplainPlan(rs);
+            String indexPlan =
+                    "UPSERT SELECT\n" +
+                    "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + tableName + " [2,*] - [2,9,000]\n" +
+                            "    SERVER FILTER BY ((\"C1\" >= 10 AND \"C1\" <= 20) AND TO_INTEGER(\"C3\") < 5000)\n" +
+                            "CLIENT MERGE SORT";
+            assertTrue("Expected '" + indexPlan + "' in the plan:\n" + plan + ".",
+                    plan.contains(indexPlan));
+
+            PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " + tableName + " (rowkey, c1, c2, c3) VALUES (?, ?, ?, ?)");
+            for (int i = 0; i < 10000; i++) {
+                stmt.setString(1, "k" + i);
+                stmt.setInt(2, i);
+                stmt.setInt(3, i);
+                stmt.setInt(4, i);
+                stmt.execute();
+            }
+
+            conn.createStatement().execute("UPDATE STATISTICS " + tableName);
+
+            // Use the idx2 plan that scans less data when stats become available.
+            rs = conn.createStatement().executeQuery("explain " + query);
+            plan = QueryUtil.getExplainPlan(rs);
+            String dataPlan =
+                    "UPSERT SELECT\n" +
+                    "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + tableName + " [1,10] - [1,20]\n" +
+                            "    SERVER FILTER BY (\"C2\" < 9000 AND \"C3\" < 5000)\n" +
+                            "CLIENT MERGE SORT";
+            assertTrue("Expected '" + dataPlan + "' in the plan:\n" + plan + ".",
+                    plan.contains(dataPlan));
+        } finally {
+            conn.close();
+        }
+    }
+
+    @Test
+    public void testCostOverridesStaticPlanOrderingInDeleteQuery() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        conn.setAutoCommit(true);
+        try {
+            String tableName = BaseTest.generateUniqueName();
+            conn.createStatement().execute("CREATE TABLE " + tableName + " (\n" +
+                    "rowkey VARCHAR PRIMARY KEY,\n" +
+                    "c1 INTEGER,\n" +
+                    "c2 INTEGER,\n" +
+                    "c3 INTEGER)");
+            conn.createStatement().execute("CREATE LOCAL INDEX " + tableName + "_idx1 ON " + tableName + " (c1) INCLUDE (c2, c3)");
+            conn.createStatement().execute("CREATE LOCAL INDEX " + tableName + "_idx2 ON " + tableName + " (c2, c3) INCLUDE (c1)");
+
+            String query = "DELETE FROM " + tableName + " where c1 BETWEEN 10 AND 20 AND c2 < 9000 AND C3 < 5000";
+            // Use the idx2 plan with a wider PK slot span when stats are not available.
+            ResultSet rs = conn.createStatement().executeQuery("explain " + query);
+            String plan = QueryUtil.getExplainPlan(rs);
+            String indexPlan =
+                    "DELETE ROWS\n" +
+                    "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + tableName + " [2,*] - [2,9,000]\n" +
+                            "    SERVER FILTER BY ((\"C1\" >= 10 AND \"C1\" <= 20) AND TO_INTEGER(\"C3\") < 5000)\n" +
+                            "CLIENT MERGE SORT";
+            assertTrue("Expected '" + indexPlan + "' in the plan:\n" + plan + ".",
+                    plan.contains(indexPlan));
+
+            PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " + tableName + " (rowkey, c1, c2, c3) VALUES (?, ?, ?, ?)");
+            for (int i = 0; i < 10000; i++) {
+                stmt.setString(1, "k" + i);
+                stmt.setInt(2, i);
+                stmt.setInt(3, i);
+                stmt.setInt(4, i);
+                stmt.execute();
+            }
+
+            conn.createStatement().execute("UPDATE STATISTICS " + tableName);
+
+            // Use the idx2 plan that scans less data when stats become available.
+            rs = conn.createStatement().executeQuery("explain " + query);
+            plan = QueryUtil.getExplainPlan(rs);
+            String dataPlan =
+                    "DELETE ROWS\n" +
+                    "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + tableName + " [1,10] - [1,20]\n" +
+                            "    SERVER FILTER BY (\"C2\" < 9000 AND \"C3\" < 5000)\n" +
+                            "CLIENT MERGE SORT";
+            assertTrue("Expected '" + dataPlan + "' in the plan:\n" + plan + ".",
+                    plan.contains(dataPlan));
+        } finally {
+            conn.close();
+        }
+    }
+
+    @Test
+    public void testCostOverridesStaticPlanOrderingInUnionQuery() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        conn.setAutoCommit(true);
+        try {
+            String tableName = BaseTest.generateUniqueName();
+            conn.createStatement().execute("CREATE TABLE " + tableName + " (\n" +
+                    "rowkey VARCHAR PRIMARY KEY,\n" +
+                    "c1 VARCHAR,\n" +
+                    "c2 VARCHAR)");
+            conn.createStatement().execute("CREATE LOCAL INDEX " + tableName + "_idx ON " + tableName + " (c1)");
+
+            String query = "SELECT c1, max(rowkey), max(c2) FROM " + tableName + " where rowkey LIKE 'k%' GROUP BY c1 "
+                    + "UNION ALL SELECT rowkey, max(c1), max(c2) FROM " + tableName + " where c1 LIKE 'X%' GROUP BY rowkey";
+            // Use the default plan when stats are not available.
+            ResultSet rs = conn.createStatement().executeQuery("explain " + query);
+            String plan = QueryUtil.getExplainPlan(rs);
+            String defaultPlan =
+                    "UNION ALL OVER 2 QUERIES\n" +
+                    "    CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + tableName + " ['k'] - ['l']\n" +
+                    "        SERVER AGGREGATE INTO DISTINCT ROWS BY [C1]\n" +
+                    "    CLIENT MERGE SORT\n" +
+                    "    CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + tableName + " [1,'X'] - [1,'Y']\n" +
+                    "        SERVER FILTER BY FIRST KEY ONLY\n" +
+                    "        SERVER AGGREGATE INTO DISTINCT ROWS BY [\"ROWKEY\"]\n" +
+                    "    CLIENT MERGE SORT";
+            assertTrue("Expected '" + defaultPlan + "' in the plan:\n" + plan + ".",
+                    plan.contains(defaultPlan));
+
+            PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " + tableName + " (rowkey, c1, c2) VALUES (?, ?, ?)");
+            for (int i = 0; i < 10000; i++) {
+                int c1 = i % 16;
+                stmt.setString(1, "k" + i);
+                stmt.setString(2, "X" + Integer.toHexString(c1) + c1);
+                stmt.setString(3, "c");
+                stmt.execute();
+            }
+
+            conn.createStatement().execute("UPDATE STATISTICS " + tableName);
+
+            // Use the optimal plan based on cost when stats become available.
+            rs = conn.createStatement().executeQuery("explain " + query);
+            plan = QueryUtil.getExplainPlan(rs);
+            String optimizedPlan =
+                    "UNION ALL OVER 2 QUERIES\n" +
+                    "    CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + tableName + " [1]\n" +
+                    "        SERVER FILTER BY FIRST KEY ONLY AND \"ROWKEY\" LIKE 'k%'\n" +
+                    "        SERVER AGGREGATE INTO ORDERED DISTINCT ROWS BY [\"C1\"]\n" +
+                    "    CLIENT MERGE SORT\n" +
+                    "    CLIENT PARALLEL 1-WAY FULL SCAN OVER " + tableName + "\n" +
+                    "        SERVER FILTER BY C1 LIKE 'X%'\n" +
+                    "        SERVER AGGREGATE INTO ORDERED DISTINCT ROWS BY [ROWKEY]";
+            assertTrue("Expected '" + optimizedPlan + "' in the plan:\n" + plan + ".",
+                    plan.contains(optimizedPlan));
+        } finally {
+            conn.close();
+        }
+    }
+
+    @Test
+    public void testCostOverridesStaticPlanOrderingInJoinQuery() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        conn.setAutoCommit(true);
+        try {
+            String tableName = BaseTest.generateUniqueName();
+            conn.createStatement().execute("CREATE TABLE " + tableName + " (\n" +
+                    "rowkey VARCHAR PRIMARY KEY,\n" +
+                    "c1 VARCHAR,\n" +
+                    "c2 VARCHAR)");
+            conn.createStatement().execute("CREATE LOCAL INDEX " + tableName + "_idx ON " + tableName + " (c1)");
+
+            String query = "SELECT t1.rowkey, t1.c1, t1.c2, mc1, mc2 FROM " + tableName + " t1 "
+                    + "JOIN (SELECT rowkey, max(c1) mc1, max(c2) mc2 FROM " + tableName + " where c1 LIKE 'X%' GROUP BY rowkey) t2 "
+                    + "ON t1.rowkey = t2.rowkey WHERE t1.c1 LIKE 'X0%' ORDER BY t1.rowkey";
+            // Use the default plan when stats are not available.
+            ResultSet rs = conn.createStatement().executeQuery("explain " + query);
+            String plan = QueryUtil.getExplainPlan(rs);
+            String defaultPlan =
+                    "CLIENT PARALLEL 1-WAY FULL SCAN OVER " + tableName + "\n" +
+                    "    SERVER FILTER BY C1 LIKE 'X0%'\n" +
+                    "    PARALLEL INNER-JOIN TABLE 0\n" +
+                    "        CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + tableName + " [1,'X'] - [1,'Y']\n" +
+                    "            SERVER FILTER BY FIRST KEY ONLY\n" +
+                    "            SERVER AGGREGATE INTO DISTINCT ROWS BY [\"ROWKEY\"]\n" +
+                    "        CLIENT MERGE SORT\n" +
+                    "    DYNAMIC SERVER FILTER BY T1.ROWKEY IN (T2.ROWKEY)";
+            assertTrue("Expected '" + defaultPlan + "' in the plan:\n" + plan + ".",
+                    plan.contains(defaultPlan));
+
+            PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " + tableName + " (rowkey, c1, c2) VALUES (?, ?, ?)");
+            for (int i = 0; i < 10000; i++) {
+                int c1 = i % 16;
+                stmt.setString(1, "k" + i);
+                stmt.setString(2, "X" + Integer.toHexString(c1) + c1);
+                stmt.setString(3, "c");
+                stmt.execute();
+            }
+
+            conn.createStatement().execute("UPDATE STATISTICS " + tableName);
+
+            // Use the optimal plan based on cost when stats become available.
+            rs = conn.createStatement().executeQuery("explain " + query);
+            plan = QueryUtil.getExplainPlan(rs);
+            String optimizedPlan =
+                    "CLIENT PARALLEL 626-WAY RANGE SCAN OVER " + tableName + " [1,'X0'] - [1,'X1']\n" +
+                    "    SERVER FILTER BY FIRST KEY ONLY\n" +
+                    "    SERVER SORTED BY [\"T1.:ROWKEY\"]\n" +
+                    "CLIENT MERGE SORT\n" +
+                    "    PARALLEL INNER-JOIN TABLE 0\n" +
+                    "        CLIENT PARALLEL 1-WAY FULL SCAN OVER " + tableName + "\n" +
+                    "            SERVER FILTER BY C1 LIKE 'X%'\n" +
+                    "            SERVER AGGREGATE INTO ORDERED DISTINCT ROWS BY [ROWKEY]\n" +
+                    "    DYNAMIC SERVER FILTER BY \"T1.:ROWKEY\" IN (T2.ROWKEY)";
+            assertTrue("Expected '" + optimizedPlan + "' in the plan:\n" + plan + ".",
+                    plan.contains(optimizedPlan));
+        } finally {
+            conn.close();
+        }
+    }
+
+    @Test
+    public void testHintOverridesCost() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        conn.setAutoCommit(true);
+        try {
+            String tableName = BaseTest.generateUniqueName();
+            conn.createStatement().execute("CREATE TABLE " + tableName + " (\n" +
+                    "rowkey INTEGER PRIMARY KEY,\n" +
+                    "c1 VARCHAR,\n" +
+                    "c2 VARCHAR)");
+            conn.createStatement().execute("CREATE LOCAL INDEX " + tableName + "_idx ON " + tableName + " (c1)");
+
+            String query = "SELECT rowkey, c1, c2 FROM " + tableName + " where rowkey between 1 and 10 ORDER BY c1";
+            String hintedQuery = query.replaceFirst("SELECT",
+                    "SELECT  /*+ INDEX(" + tableName + " " + tableName + "_idx) */");
+            String dataPlan = "SERVER SORTED BY [C1]";
+            String indexPlan = "SERVER FILTER BY FIRST KEY ONLY AND (\"ROWKEY\" >= 1 AND \"ROWKEY\" <= 10)";
+
+            // Use the index table plan that opts out order-by when stats are not available.
+            ResultSet rs = conn.createStatement().executeQuery("explain " + query);
+            String plan = QueryUtil.getExplainPlan(rs);
+            assertTrue("Expected '" + indexPlan + "' in the plan:\n" + plan + ".",
+                    plan.contains(indexPlan));
+
+            PreparedStatement stmt = conn.prepareStatement("UPSERT INTO " + tableName + " (rowkey, c1, c2) VALUES (?, ?, ?)");
+            for (int i = 0; i < 10000; i++) {
+                int c1 = i % 16;
+                stmt.setInt(1, i);
+                stmt.setString(2, "X" + Integer.toHexString(c1) + c1);
+                stmt.setString(3, "c");
+                stmt.execute();
+            }
+
+            conn.createStatement().execute("UPDATE STATISTICS " + tableName);
+
+            // Use the data table plan that has a lower cost when stats are available.
+            rs = conn.createStatement().executeQuery("explain " + query);
+            plan = QueryUtil.getExplainPlan(rs);
+            assertTrue("Expected '" + dataPlan + "' in the plan:\n" + plan + ".",
+                    plan.contains(dataPlan));
+
+            // Use the index table plan as has been hinted.
+            rs = conn.createStatement().executeQuery("explain " + hintedQuery);
+            plan = QueryUtil.getExplainPlan(rs);
+            assertTrue("Expected '" + indexPlan + "' in the plan:\n" + plan + ".",
+                    plan.contains(indexPlan));
+        } finally {
+            conn.close();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/597764c0/phoenix-core/src/it/java/org/apache/phoenix/end2end/MutationStateIT.java
----------------------------------------------------------------------
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 2d5f360..36782c1 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
@@ -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 static org.junit.Assert.assertEquals;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/597764c0/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogIT.java
index 15af2af..7b6a543 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogIT.java
@@ -15,7 +15,6 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.phoenix.end2end;
 
 import static org.junit.Assert.assertEquals;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/597764c0/phoenix-core/src/main/java/org/apache/phoenix/compile/ListJarsQueryPlan.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/ListJarsQueryPlan.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/ListJarsQueryPlan.java
index 839e7c9..0688b94 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/ListJarsQueryPlan.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/ListJarsQueryPlan.java
@@ -49,6 +49,7 @@ import org.apache.phoenix.iterate.ResultIterator;
 import org.apache.phoenix.jdbc.PhoenixParameterMetaData;
 import org.apache.phoenix.jdbc.PhoenixStatement;
 import org.apache.phoenix.jdbc.PhoenixStatement.Operation;
+import org.apache.phoenix.optimize.Cost;
 import org.apache.phoenix.parse.FilterableStatement;
 import org.apache.phoenix.parse.LiteralParseNode;
 import org.apache.phoenix.parse.ParseNodeFactory;
@@ -186,6 +187,11 @@ public class ListJarsQueryPlan implements QueryPlan {
     }
 
     @Override
+    public Cost getCost() {
+        return Cost.ZERO;
+    }
+
+    @Override
     public TableRef getTableRef() {
         return null;
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/597764c0/phoenix-core/src/main/java/org/apache/phoenix/compile/QueryPlan.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/QueryPlan.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/QueryPlan.java
index f7cdcbf..ca88984 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/QueryPlan.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/QueryPlan.java
@@ -26,6 +26,7 @@ import org.apache.phoenix.compile.GroupByCompiler.GroupBy;
 import org.apache.phoenix.compile.OrderByCompiler.OrderBy;
 import org.apache.phoenix.iterate.ParallelScanGrouper;
 import org.apache.phoenix.iterate.ResultIterator;
+import org.apache.phoenix.optimize.Cost;
 import org.apache.phoenix.parse.FilterableStatement;
 import org.apache.phoenix.query.KeyRange;
 import org.apache.phoenix.schema.TableRef;
@@ -52,7 +53,9 @@ public interface QueryPlan extends StatementPlan {
     public ResultIterator iterator(ParallelScanGrouper scanGrouper, Scan scan) throws SQLException;
 
     public long getEstimatedSize();
-    
+
+    public Cost getCost();
+
     // TODO: change once joins are supported
     TableRef getTableRef();
     /**

http://git-wip-us.apache.org/repos/asf/phoenix/blob/597764c0/phoenix-core/src/main/java/org/apache/phoenix/compile/TraceQueryPlan.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/TraceQueryPlan.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/TraceQueryPlan.java
index 62e6991..2714858 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/TraceQueryPlan.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/TraceQueryPlan.java
@@ -46,6 +46,7 @@ import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixStatement;
 import org.apache.phoenix.jdbc.PhoenixStatement.Operation;
 import org.apache.phoenix.metrics.MetricInfo;
+import org.apache.phoenix.optimize.Cost;
 import org.apache.phoenix.parse.FilterableStatement;
 import org.apache.phoenix.parse.LiteralParseNode;
 import org.apache.phoenix.parse.ParseNodeFactory;
@@ -194,6 +195,11 @@ public class TraceQueryPlan implements QueryPlan {
     }
 
     @Override
+    public Cost getCost() {
+        return Cost.ZERO;
+    }
+
+    @Override
     public Set<TableRef> getSourceRefs() {
         return Collections.emptySet();
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/597764c0/phoenix-core/src/main/java/org/apache/phoenix/execute/AggregatePlan.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/AggregatePlan.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/AggregatePlan.java
index 4c29abe..369769e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/AggregatePlan.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/AggregatePlan.java
@@ -59,6 +59,7 @@ import org.apache.phoenix.iterate.SequenceResultIterator;
 import org.apache.phoenix.iterate.SerialIterators;
 import org.apache.phoenix.iterate.SpoolingResultIterator;
 import org.apache.phoenix.iterate.UngroupedAggregatingResultIterator;
+import org.apache.phoenix.optimize.Cost;
 import org.apache.phoenix.parse.FilterableStatement;
 import org.apache.phoenix.parse.HintNode;
 import org.apache.phoenix.query.KeyRange;
@@ -67,6 +68,7 @@ import org.apache.phoenix.query.QueryServicesOptions;
 import org.apache.phoenix.schema.PTable.IndexType;
 import org.apache.phoenix.schema.TableRef;
 import org.apache.phoenix.schema.types.PInteger;
+import org.apache.phoenix.util.CostUtil;
 import org.apache.phoenix.util.ScanUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -112,7 +114,33 @@ public class AggregatePlan extends BaseQueryPlan {
     public Expression getHaving() {
         return having;
     }
-    
+
+    @Override
+    public Cost getCost() {
+        Long byteCount = null;
+        try {
+            byteCount = getEstimatedBytesToScan();
+        } catch (SQLException e) {
+            // ignored.
+        }
+
+        if (byteCount == null) {
+            return Cost.UNKNOWN;
+        }
+
+        int parallelLevel = CostUtil.estimateParallelLevel(
+                true, context.getConnection().getQueryServices());
+        Cost cost = CostUtil.estimateAggregateCost(byteCount,
+                groupBy, aggregators.getEstimatedByteSize(), parallelLevel);
+        if (!orderBy.getOrderByExpressions().isEmpty()) {
+            double outputBytes = CostUtil.estimateAggregateOutputBytes(
+                    byteCount, groupBy, aggregators.getEstimatedByteSize());
+            Cost orderByCost = CostUtil.estimateOrderByCost(outputBytes, parallelLevel);
+            cost = cost.plus(orderByCost);
+        }
+        return cost;
+    }
+
     @Override
     public List<KeyRange> getSplits() {
         if (splits == null)

http://git-wip-us.apache.org/repos/asf/phoenix/blob/597764c0/phoenix-core/src/main/java/org/apache/phoenix/execute/BaseQueryPlan.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/BaseQueryPlan.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/BaseQueryPlan.java
index c1ddd44..31f67b7 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/BaseQueryPlan.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/BaseQueryPlan.java
@@ -63,6 +63,8 @@ import org.apache.phoenix.parse.HintNode.Hint;
 import org.apache.phoenix.parse.ParseNodeFactory;
 import org.apache.phoenix.parse.TableName;
 import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.query.QueryServicesOptions;
 import org.apache.phoenix.schema.KeyValueSchema;
 import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PName;
@@ -500,13 +502,24 @@ public abstract class BaseQueryPlan implements QueryPlan {
         if (context.getScanRanges() == ScanRanges.NOTHING) {
             return new ExplainPlan(Collections.singletonList("DEGENERATE SCAN OVER " + getTableRef().getTable().getName().getString()));
         }
-        
+
+        // If cost-based optimizer is enabled, we need to initialize a dummy iterator to
+        // get the stats for computing costs.
+        boolean costBased =
+                context.getConnection().getQueryServices().getConfiguration().getBoolean(
+                        QueryServices.COST_BASED_OPTIMIZER_ENABLED, QueryServicesOptions.DEFAULT_COST_BASED_OPTIMIZER_ENABLED);
+        if (costBased) {
+            ResultIterator iterator = iterator();
+            iterator.close();
+        }
         // Optimize here when getting explain plan, as queries don't get optimized until after compilation
         QueryPlan plan = context.getConnection().getQueryServices().getOptimizer().optimize(context.getStatement(), this);
         ExplainPlan exp = plan instanceof BaseQueryPlan ? new ExplainPlan(getPlanSteps(plan.iterator())) : plan.getExplainPlan();
-        this.estimatedRows = plan.getEstimatedRowsToScan();
-        this.estimatedSize = plan.getEstimatedBytesToScan();
-        this.estimateInfoTimestamp = plan.getEstimateInfoTimestamp();
+        if (!costBased) { // do not override estimates if they are used for cost calculation.
+            this.estimatedRows = plan.getEstimatedRowsToScan();
+            this.estimatedSize = plan.getEstimatedBytesToScan();
+            this.estimateInfoTimestamp = plan.getEstimateInfoTimestamp();
+        }
         return exp;
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/597764c0/phoenix-core/src/main/java/org/apache/phoenix/execute/ClientAggregatePlan.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/ClientAggregatePlan.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/ClientAggregatePlan.java
index 8ef1f8d..a15ab35 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/ClientAggregatePlan.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/ClientAggregatePlan.java
@@ -56,12 +56,14 @@ import org.apache.phoenix.iterate.PeekingResultIterator;
 import org.apache.phoenix.iterate.ResultIterator;
 import org.apache.phoenix.iterate.SequenceResultIterator;
 import org.apache.phoenix.iterate.UngroupedAggregatingResultIterator;
+import org.apache.phoenix.optimize.Cost;
 import org.apache.phoenix.parse.FilterableStatement;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
 import org.apache.phoenix.schema.TableRef;
 import org.apache.phoenix.schema.tuple.MultiKeyValueTuple;
 import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.CostUtil;
 import org.apache.phoenix.util.TupleUtil;
 
 import com.google.common.collect.Lists;
@@ -87,6 +89,32 @@ public class ClientAggregatePlan extends ClientProcessingPlan {
     }
 
     @Override
+    public Cost getCost() {
+        Long byteCount = null;
+        try {
+            byteCount = getEstimatedBytesToScan();
+        } catch (SQLException e) {
+            // ignored.
+        }
+
+        if (byteCount == null) {
+            return Cost.UNKNOWN;
+        }
+
+        int parallelLevel = CostUtil.estimateParallelLevel(
+                false, context.getConnection().getQueryServices());
+        Cost cost = CostUtil.estimateAggregateCost(byteCount,
+                groupBy, clientAggregators.getEstimatedByteSize(), parallelLevel);
+        if (!orderBy.getOrderByExpressions().isEmpty()) {
+            double outputBytes = CostUtil.estimateAggregateOutputBytes(
+                    byteCount, groupBy, clientAggregators.getEstimatedByteSize());
+            Cost orderByCost = CostUtil.estimateOrderByCost(outputBytes, parallelLevel);
+            cost = cost.plus(orderByCost);
+        }
+        return super.getCost().plus(cost);
+    }
+
+    @Override
     public ResultIterator iterator(ParallelScanGrouper scanGrouper, Scan scan) throws SQLException {
         ResultIterator iterator = delegate.iterator(scanGrouper, scan);
         if (where != null) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/597764c0/phoenix-core/src/main/java/org/apache/phoenix/execute/ClientScanPlan.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/ClientScanPlan.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/ClientScanPlan.java
index 6bbc545..5799990 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/ClientScanPlan.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/ClientScanPlan.java
@@ -34,10 +34,12 @@ import org.apache.phoenix.iterate.OrderedResultIterator;
 import org.apache.phoenix.iterate.ParallelScanGrouper;
 import org.apache.phoenix.iterate.ResultIterator;
 import org.apache.phoenix.iterate.SequenceResultIterator;
+import org.apache.phoenix.optimize.Cost;
 import org.apache.phoenix.parse.FilterableStatement;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
 import org.apache.phoenix.schema.TableRef;
+import org.apache.phoenix.util.CostUtil;
 
 import com.google.common.collect.Lists;
 
@@ -50,6 +52,29 @@ public class ClientScanPlan extends ClientProcessingPlan {
     }
 
     @Override
+    public Cost getCost() {
+        Long byteCount = null;
+        try {
+            byteCount = getEstimatedBytesToScan();
+        } catch (SQLException e) {
+            // ignored.
+        }
+
+        if (byteCount == null) {
+            return Cost.UNKNOWN;
+        }
+
+        Cost cost = new Cost(0, 0, byteCount);
+        int parallelLevel = CostUtil.estimateParallelLevel(
+                false, context.getConnection().getQueryServices());
+        if (!orderBy.getOrderByExpressions().isEmpty()) {
+            Cost orderByCost = CostUtil.estimateOrderByCost(byteCount, parallelLevel);
+            cost = cost.plus(orderByCost);
+        }
+        return super.getCost().plus(cost);
+    }
+
+    @Override
     public ResultIterator iterator(ParallelScanGrouper scanGrouper, Scan scan) throws SQLException {
         ResultIterator iterator = delegate.iterator(scanGrouper, scan);
         if (where != null) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/597764c0/phoenix-core/src/main/java/org/apache/phoenix/execute/CorrelatePlan.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/CorrelatePlan.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/CorrelatePlan.java
index ee81c36..270ad3d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/CorrelatePlan.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/CorrelatePlan.java
@@ -30,6 +30,7 @@ import org.apache.phoenix.exception.SQLExceptionInfo;
 import org.apache.phoenix.execute.TupleProjector.ProjectedValueTuple;
 import org.apache.phoenix.iterate.ParallelScanGrouper;
 import org.apache.phoenix.iterate.ResultIterator;
+import org.apache.phoenix.optimize.Cost;
 import org.apache.phoenix.parse.JoinTableNode.JoinType;
 import org.apache.phoenix.schema.KeyValueSchema;
 import org.apache.phoenix.schema.PColumn;
@@ -200,4 +201,28 @@ public class CorrelatePlan extends DelegateQueryPlan {
         return null;
     }
 
+    @Override
+    public Cost getCost() {
+        Long lhsByteCount = null;
+        try {
+            lhsByteCount = delegate.getEstimatedBytesToScan();
+        } catch (SQLException e) {
+            // ignored.
+        }
+        Long rhsRowCount = null;
+        try {
+            rhsRowCount = rhs.getEstimatedRowsToScan();
+        } catch (SQLException e) {
+            // ignored.
+        }
+
+        if (lhsByteCount == null || rhsRowCount == null) {
+            return Cost.UNKNOWN;
+        }
+
+        Cost cost = new Cost(0, 0, lhsByteCount * rhsRowCount);
+        Cost lhsCost = delegate.getCost();
+        return cost.plus(lhsCost).plus(rhs.getCost());
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/597764c0/phoenix-core/src/main/java/org/apache/phoenix/execute/DelegateQueryPlan.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/DelegateQueryPlan.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/DelegateQueryPlan.java
index 3c62c5b..3da06db 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/DelegateQueryPlan.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/DelegateQueryPlan.java
@@ -32,6 +32,7 @@ import org.apache.phoenix.compile.QueryPlan;
 import org.apache.phoenix.compile.RowProjector;
 import org.apache.phoenix.compile.StatementContext;
 import org.apache.phoenix.jdbc.PhoenixStatement.Operation;
+import org.apache.phoenix.optimize.Cost;
 import org.apache.phoenix.parse.FilterableStatement;
 import org.apache.phoenix.query.KeyRange;
 import org.apache.phoenix.schema.TableRef;
@@ -59,6 +60,11 @@ public abstract class DelegateQueryPlan implements QueryPlan {
     }
 
     @Override
+    public Cost getCost() {
+        return delegate.getCost();
+    }
+
+    @Override
     public TableRef getTableRef() {
         return delegate.getTableRef();
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/597764c0/phoenix-core/src/main/java/org/apache/phoenix/execute/HashJoinPlan.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/HashJoinPlan.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/HashJoinPlan.java
index 2b90dcb..2d2ff4e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/HashJoinPlan.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/HashJoinPlan.java
@@ -62,6 +62,7 @@ import org.apache.phoenix.job.JobManager.JobCallable;
 import org.apache.phoenix.join.HashCacheClient;
 import org.apache.phoenix.join.HashJoinInfo;
 import org.apache.phoenix.monitoring.TaskExecutionMetricsHolder;
+import org.apache.phoenix.optimize.Cost;
 import org.apache.phoenix.parse.FilterableStatement;
 import org.apache.phoenix.parse.ParseNode;
 import org.apache.phoenix.parse.SQLParser;
@@ -290,6 +291,34 @@ public class HashJoinPlan extends DelegateQueryPlan {
         return statement;
     }
 
+    @Override
+    public Cost getCost() {
+        Long byteCount = null;
+        try {
+            byteCount = getEstimatedBytesToScan();
+        } catch (SQLException e) {
+            // ignored.
+        }
+
+        if (byteCount == null) {
+            return Cost.UNKNOWN;
+        }
+
+        Cost cost = new Cost(0, 0, byteCount);
+        Cost lhsCost = delegate.getCost();
+        if (keyRangeExpressions != null) {
+            // The selectivity of the dynamic rowkey filter.
+            // TODO replace the constant with an estimate value.
+            double selectivity = 0.01;
+            lhsCost = lhsCost.multiplyBy(selectivity);
+        }
+        Cost rhsCost = Cost.ZERO;
+        for (SubPlan subPlan : subPlans) {
+            rhsCost = rhsCost.plus(subPlan.getInnerPlan().getCost());
+        }
+        return cost.plus(lhsCost).plus(rhsCost);
+    }
+
     protected interface SubPlan {
         public ServerCache execute(HashJoinPlan parent) throws SQLException;
         public void postProcess(ServerCache result, HashJoinPlan parent) throws SQLException;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/597764c0/phoenix-core/src/main/java/org/apache/phoenix/execute/LiteralResultIterationPlan.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/LiteralResultIterationPlan.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/LiteralResultIterationPlan.java
index 86f59c5..1d1332d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/LiteralResultIterationPlan.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/LiteralResultIterationPlan.java
@@ -35,6 +35,7 @@ import org.apache.phoenix.iterate.ParallelIteratorFactory;
 import org.apache.phoenix.iterate.ParallelScanGrouper;
 import org.apache.phoenix.iterate.ResultIterator;
 import org.apache.phoenix.iterate.SequenceResultIterator;
+import org.apache.phoenix.optimize.Cost;
 import org.apache.phoenix.parse.FilterableStatement;
 import org.apache.phoenix.query.KeyRange;
 import org.apache.phoenix.schema.TableRef;
@@ -60,6 +61,11 @@ public class LiteralResultIterationPlan extends BaseQueryPlan {
     }
 
     @Override
+    public Cost getCost() {
+        return Cost.ZERO;
+    }
+
+    @Override
     public List<KeyRange> getSplits() {
         return Collections.emptyList();
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/597764c0/phoenix-core/src/main/java/org/apache/phoenix/execute/ScanPlan.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/ScanPlan.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/ScanPlan.java
index 1e1cb0d..31d7097 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/ScanPlan.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/ScanPlan.java
@@ -53,6 +53,7 @@ import org.apache.phoenix.iterate.RoundRobinResultIterator;
 import org.apache.phoenix.iterate.SequenceResultIterator;
 import org.apache.phoenix.iterate.SerialIterators;
 import org.apache.phoenix.iterate.SpoolingResultIterator;
+import org.apache.phoenix.optimize.Cost;
 import org.apache.phoenix.parse.FilterableStatement;
 import org.apache.phoenix.parse.HintNode;
 import org.apache.phoenix.query.ConnectionQueryServices;
@@ -64,6 +65,7 @@ import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTable.IndexType;
 import org.apache.phoenix.schema.SaltingUtil;
 import org.apache.phoenix.schema.TableRef;
+import org.apache.phoenix.util.CostUtil;
 import org.apache.phoenix.util.EnvironmentEdgeManager;
 import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.ScanUtil;
@@ -189,6 +191,29 @@ public class ScanPlan extends BaseQueryPlan {
     }
 
     @Override
+    public Cost getCost() {
+        Long byteCount = null;
+        try {
+            byteCount = getEstimatedBytesToScan();
+        } catch (SQLException e) {
+            // ignored.
+        }
+
+        if (byteCount == null) {
+            return Cost.UNKNOWN;
+        }
+
+        Cost cost = new Cost(0, 0, byteCount);
+        int parallelLevel = CostUtil.estimateParallelLevel(
+                true, context.getConnection().getQueryServices());
+        if (!orderBy.getOrderByExpressions().isEmpty()) {
+            Cost orderByCost = CostUtil.estimateOrderByCost(byteCount, parallelLevel);
+            cost = cost.plus(orderByCost);
+        }
+        return cost;
+    }
+
+    @Override
     public List<KeyRange> getSplits() {
         if (splits == null)
             return Collections.emptyList();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/597764c0/phoenix-core/src/main/java/org/apache/phoenix/execute/SortMergeJoinPlan.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/SortMergeJoinPlan.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/SortMergeJoinPlan.java
index fab7c59..3e380da 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/SortMergeJoinPlan.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/SortMergeJoinPlan.java
@@ -54,6 +54,7 @@ import org.apache.phoenix.iterate.ParallelScanGrouper;
 import org.apache.phoenix.iterate.ResultIterator;
 import org.apache.phoenix.jdbc.PhoenixParameterMetaData;
 import org.apache.phoenix.jdbc.PhoenixStatement.Operation;
+import org.apache.phoenix.optimize.Cost;
 import org.apache.phoenix.parse.FilterableStatement;
 import org.apache.phoenix.parse.JoinTableNode.JoinType;
 import org.apache.phoenix.query.KeyRange;
@@ -192,6 +193,23 @@ public class SortMergeJoinPlan implements QueryPlan {
     }
 
     @Override
+    public Cost getCost() {
+        Long byteCount = null;
+        try {
+            byteCount = getEstimatedBytesToScan();
+        } catch (SQLException e) {
+            // ignored.
+        }
+
+        if (byteCount == null) {
+            return Cost.UNKNOWN;
+        }
+
+        Cost cost = new Cost(0, 0, byteCount);
+        return cost.plus(lhsPlan.getCost()).plus(rhsPlan.getCost());
+    }
+
+    @Override
     public StatementContext getContext() {
         return context;
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/597764c0/phoenix-core/src/main/java/org/apache/phoenix/execute/UnionPlan.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/UnionPlan.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/UnionPlan.java
index e06522f..e6bf654 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/UnionPlan.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/UnionPlan.java
@@ -43,6 +43,7 @@ import org.apache.phoenix.iterate.ParallelScanGrouper;
 import org.apache.phoenix.iterate.ResultIterator;
 import org.apache.phoenix.iterate.UnionResultIterators;
 import org.apache.phoenix.jdbc.PhoenixStatement.Operation;
+import org.apache.phoenix.optimize.Cost;
 import org.apache.phoenix.parse.FilterableStatement;
 import org.apache.phoenix.query.KeyRange;
 import org.apache.phoenix.schema.TableRef;
@@ -210,6 +211,15 @@ public class UnionPlan implements QueryPlan {
     }
 
     @Override
+    public Cost getCost() {
+        Cost cost = Cost.ZERO;
+        for (QueryPlan plan : plans) {
+            cost = cost.plus(plan.getCost());
+        }
+        return cost;
+    }
+
+    @Override
     public ParameterMetaData getParameterMetaData() {
         return paramMetaData;
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/597764c0/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
index 384c8cc..26b4415 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
@@ -91,6 +91,7 @@ import org.apache.phoenix.expression.RowKeyColumnExpression;
 import org.apache.phoenix.iterate.MaterializedResultIterator;
 import org.apache.phoenix.iterate.ParallelScanGrouper;
 import org.apache.phoenix.iterate.ResultIterator;
+import org.apache.phoenix.optimize.Cost;
 import org.apache.phoenix.parse.AddColumnStatement;
 import org.apache.phoenix.parse.AddJarsStatement;
 import org.apache.phoenix.parse.AliasedNode;
@@ -647,6 +648,11 @@ public class PhoenixStatement implements Statement, SQLCloseable {
                 }
 
                 @Override
+                public Cost getCost() {
+                    return Cost.ZERO;
+                }
+
+                @Override
                 public TableRef getTableRef() {
                     return null;
                 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/597764c0/phoenix-core/src/main/java/org/apache/phoenix/optimize/Cost.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/optimize/Cost.java b/phoenix-core/src/main/java/org/apache/phoenix/optimize/Cost.java
new file mode 100644
index 0000000..b83f354
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/optimize/Cost.java
@@ -0,0 +1,123 @@
+/*
+ * 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.optimize;
+
+import java.util.Objects;
+
+/**
+ * Optimizer cost in terms of CPU, memory, and I/O usage, the unit of which is now the
+ * number of bytes processed.
+ *
+ */
+public class Cost implements Comparable<Cost> {
+    /** The unknown cost. */
+    public static Cost UNKNOWN = new Cost(Double.NaN, Double.NaN, Double.NaN) {
+        @Override
+        public String toString() {
+            return "{unknown}";
+        }
+    };
+
+    /** The zero cost. */
+    public static Cost ZERO = new Cost(0, 0, 0) {
+        @Override
+        public String toString() {
+            return "{zero}";
+        }        
+    };
+
+    private final double cpu;
+    private final double memory;
+    private final double io;
+
+    public Cost(double cpu, double memory, double io) {
+        this.cpu = cpu;
+        this.memory = memory;
+        this.io = io;
+    }
+
+    public double getCpu() {
+        return cpu;
+    }
+
+    public double getMemory() {
+        return memory;
+    }
+
+    public double getIo() {
+        return io;
+    }
+
+    public boolean isUnknown() {
+        return this == UNKNOWN;
+    }
+
+    public Cost plus(Cost other) {
+        if (isUnknown() || other.isUnknown()) {
+            return UNKNOWN;
+        }
+
+        return new Cost(
+                this.cpu + other.cpu,
+                this.memory + other.memory,
+                this.io + other.io);
+    }
+
+    public Cost multiplyBy(double factor) {
+        if (isUnknown()) {
+            return UNKNOWN;
+        }
+
+        return new Cost(
+                this.cpu * factor,
+                this.memory * factor,
+                this.io * factor);
+    }
+
+    // TODO right now for simplicity, we choose to ignore CPU and memory costs. We may
+    // add those into account as our cost model mature.
+    @Override
+    public int compareTo(Cost other) {
+        if (isUnknown() && other.isUnknown()) {
+            return 0;
+        } else if (isUnknown() && !other.isUnknown()) {
+            return 1;
+        } else if (!isUnknown() && other.isUnknown()) {
+            return -1;
+        }
+
+        double d = this.io - other.io;
+        return d == 0 ? 0 : (d > 0 ? 1 : -1);
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        return this == obj
+                || (obj instanceof Cost && this.compareTo((Cost) obj) == 0);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(cpu, memory, io);
+    }
+
+    @Override
+    public String toString() {
+        return "{cpu: " + cpu + ", memory: " + memory + ", io: " + io + "}";
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/597764c0/phoenix-core/src/main/java/org/apache/phoenix/optimize/QueryOptimizer.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/optimize/QueryOptimizer.java b/phoenix-core/src/main/java/org/apache/phoenix/optimize/QueryOptimizer.java
index b3df50b..64dad58 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/optimize/QueryOptimizer.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/optimize/QueryOptimizer.java
@@ -67,10 +67,12 @@ public class QueryOptimizer {
 
     private final QueryServices services;
     private final boolean useIndexes;
+    private final boolean costBased;
 
     public QueryOptimizer(QueryServices services) {
         this.services = services;
         this.useIndexes = this.services.getProps().getBoolean(QueryServices.USE_INDEXES_ATTRIB, QueryServicesOptions.DEFAULT_USE_INDEXES);
+        this.costBased = this.services.getProps().getBoolean(QueryServices.COST_BASED_OPTIMIZER_ENABLED, QueryServicesOptions.DEFAULT_COST_BASED_OPTIMIZER_ENABLED);
     }
 
     public QueryPlan optimize(PhoenixStatement statement, QueryPlan dataPlan) throws SQLException {
@@ -91,7 +93,7 @@ public class QueryOptimizer {
     }
     
     public QueryPlan optimize(QueryPlan dataPlan, PhoenixStatement statement, List<? extends PDatum> targetColumns, ParallelIteratorFactory parallelIteratorFactory) throws SQLException {
-        List<QueryPlan>plans = getApplicablePlans(dataPlan, statement, targetColumns, parallelIteratorFactory, true);
+        List<QueryPlan> plans = getApplicablePlans(dataPlan, statement, targetColumns, parallelIteratorFactory, true);
         return plans.get(0);
     }
     
@@ -309,10 +311,11 @@ public class QueryOptimizer {
         }
         return null;
     }
-    
+
     /**
      * Order the plans among all the possible ones from best to worst.
-     * Since we don't keep stats yet, we use the following simple algorithm:
+     * If option COST_BASED_OPTIMIZER_ENABLED is on and stats are available, we order the plans based on
+     * their costs, otherwise we use the following simple algorithm:
      * 1) If the query is a point lookup (i.e. we have a set of exact row keys), choose that one immediately.
      * 2) If the query has an ORDER BY and a LIMIT, choose the plan that has all the ORDER BY expression
      * in the same order as the row key columns.
@@ -320,9 +323,6 @@ public class QueryOptimizer {
      *    a) the most row key columns that may be used to form the start/stop scan key (i.e. bound slots).
      *    b) the plan that preserves ordering for a group by.
      *    c) the non local index table plan
-     * TODO: We should make more of a cost based choice: The largest number of bound slots does not necessarily
-     * correspond to the least bytes scanned. We could consider the slots bound for upper and lower ranges 
-     * separately, or we could calculate the bytes scanned between the start and stop row of each table.
      * @param plans the list of candidate plans
      * @return list of plans ordered from best to worst.
      */
@@ -331,7 +331,21 @@ public class QueryOptimizer {
         if (plans.size() == 1) {
             return plans;
         }
-        
+
+        if (this.costBased) {
+            Collections.sort(plans, new Comparator<QueryPlan>() {
+                @Override
+                public int compare(QueryPlan plan1, QueryPlan plan2) {
+                    return plan1.getCost().compareTo(plan2.getCost());
+                }
+            });
+            // Return ordered list based on cost if stats are available; otherwise fall
+            // back to static ordering.
+            if (!plans.get(0).getCost().isUnknown()) {
+                return stopAtBestPlan ? plans.subList(0, 1) : plans;
+            }
+        }
+
         /**
          * If we have a plan(s) that are just point lookups (i.e. fully qualified row
          * keys), then favor those first.
@@ -428,7 +442,7 @@ public class QueryOptimizer {
             }
             
         });
-        
+
         return stopAtBestPlan ? bestCandidates.subList(0, 1) : bestCandidates;
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/597764c0/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
index 59f7385..0b80f4d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
@@ -294,6 +294,9 @@ public interface QueryServices extends SQLCloseable {
     //Update Cache Frequency default config attribute
     public static final String DEFAULT_UPDATE_CACHE_FREQUENCY_ATRRIB  = "phoenix.default.update.cache.frequency";
 
+    // Whether to enable cost-based-decision in the query optimizer
+    public static final String COST_BASED_OPTIMIZER_ENABLED = "phoenix.costbased.optimizer.enabled";
+
     /**
      * Get executor service used for parallel scans
      */

http://git-wip-us.apache.org/repos/asf/phoenix/blob/597764c0/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
index 3ceb084..4d31974 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
@@ -25,6 +25,7 @@ import static org.apache.phoenix.query.QueryServices.CALL_QUEUE_PRODUCER_ATTRIB_
 import static org.apache.phoenix.query.QueryServices.CALL_QUEUE_ROUND_ROBIN_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.COLLECT_REQUEST_LEVEL_METRICS;
 import static org.apache.phoenix.query.QueryServices.COMMIT_STATS_ASYNC;
+import static org.apache.phoenix.query.QueryServices.COST_BASED_OPTIMIZER_ENABLED;
 import static org.apache.phoenix.query.QueryServices.DATE_FORMAT_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.DATE_FORMAT_TIMEZONE_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.DELAY_FOR_SCHEMA_UPDATE_CHECK;
@@ -341,6 +342,8 @@ public class QueryServicesOptions {
     // RS -> RS calls for upsert select statements are disabled by default
     public static final boolean DEFAULT_ENABLE_SERVER_UPSERT_SELECT = false;
 
+    public static final boolean DEFAULT_COST_BASED_OPTIMIZER_ENABLED = false;
+
     private final Configuration config;
 
     private QueryServicesOptions(Configuration config) {
@@ -418,6 +421,7 @@ public class QueryServicesOptions {
             .setIfUnset(TRACING_THREAD_POOL_SIZE, DEFAULT_TRACING_THREAD_POOL_SIZE)
             .setIfUnset(STATS_COLLECTION_ENABLED, DEFAULT_STATS_COLLECTION_ENABLED)
             .setIfUnset(USE_STATS_FOR_PARALLELIZATION, DEFAULT_USE_STATS_FOR_PARALLELIZATION)
+            .setIfUnset(COST_BASED_OPTIMIZER_ENABLED, DEFAULT_COST_BASED_OPTIMIZER_ENABLED)
             .setIfUnset(UPLOAD_BINARY_DATA_TYPE_ENCODING, DEFAULT_UPLOAD_BINARY_DATA_TYPE_ENCODING)
             .setIfUnset(PHOENIX_ACLS_ENABLED,  DEFAULT_PHOENIX_ACLS_ENABLED);
         // HBase sets this to 1, so we reset it to something more appropriate.

http://git-wip-us.apache.org/repos/asf/phoenix/blob/597764c0/phoenix-core/src/main/java/org/apache/phoenix/util/CostUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/CostUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/CostUtil.java
new file mode 100644
index 0000000..1d4b8e0
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/CostUtil.java
@@ -0,0 +1,90 @@
+/*
+ * 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.util;
+
+import org.apache.phoenix.compile.GroupByCompiler.GroupBy;
+import org.apache.phoenix.optimize.Cost;
+import org.apache.phoenix.query.QueryServices;
+
+/**
+ * Utilities for computing costs.
+ *
+ * Some of the methods here should eventually be replaced by a metadata framework which
+ * estimates output metrics for each QueryPlan or operation, e.g. row count, byte count,
+ * etc.
+ */
+public class CostUtil {
+
+    // An estimate of the ratio of result data from group-by against the input data.
+    private final static double GROUPING_FACTOR = 0.1;
+
+    // Io operations conducted in intermediate evaluations like sorting or aggregation
+    // should be counted twice since they usually involve both read and write.
+    private final static double IO_COST_MULTIPLIER = 2.0;
+
+    /**
+     * Estimate the number of output bytes of an aggregate.
+     * @param byteCount the number of input bytes
+     * @param groupBy the compiled GroupBy object
+     * @param aggregatorsSize the byte size of aggregators
+     * @return the output byte count
+     */
+    public static double estimateAggregateOutputBytes(
+            double byteCount, GroupBy groupBy, int aggregatorsSize) {
+        if (groupBy.isUngroupedAggregate()) {
+            return aggregatorsSize;
+        }
+        return byteCount * GROUPING_FACTOR;
+    }
+
+    /**
+     * Estimate the cost of an aggregate.
+     * @param byteCount the number of input bytes
+     * @param groupBy the compiled GroupBy object
+     * @param aggregatorsSize the byte size of aggregators
+     * @param parallelLevel number of parallel workers or threads
+     * @return the cost
+     */
+    public static Cost estimateAggregateCost(
+            double byteCount, GroupBy groupBy, int aggregatorsSize, int parallelLevel) {
+        double outputBytes = estimateAggregateOutputBytes(byteCount, groupBy, aggregatorsSize);
+        double orderedFactor = groupBy.isOrderPreserving() ? 0.2 : 1.0;
+        return new Cost(0, 0, outputBytes * orderedFactor * IO_COST_MULTIPLIER / parallelLevel);
+    }
+
+    /**
+     * Estimate the cost of an order-by
+     * @param byteCount the number of input bytes
+     * @param parallelLevel number of parallel workers or threads
+     * @return the cost
+     */
+    public static Cost estimateOrderByCost(double byteCount, int parallelLevel) {
+        return new Cost(0, 0, byteCount * IO_COST_MULTIPLIER / parallelLevel);
+    }
+
+    /**
+     * Estimate the parallel level of an operation
+     * @param runningOnServer if the operation will be running on server side
+     * @param services the QueryServices object
+     * @return the parallel level
+     */
+    public static int estimateParallelLevel(boolean runningOnServer, QueryServices services) {
+        // TODO currently return constants for simplicity, should derive from cluster config.
+        return runningOnServer ? 10 : 1;
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/597764c0/phoenix-core/src/test/java/org/apache/phoenix/query/ParallelIteratorsSplitTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/query/ParallelIteratorsSplitTest.java b/phoenix-core/src/test/java/org/apache/phoenix/query/ParallelIteratorsSplitTest.java
index 935d8cb..0f12d9c 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/query/ParallelIteratorsSplitTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/query/ParallelIteratorsSplitTest.java
@@ -52,6 +52,7 @@ import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixParameterMetaData;
 import org.apache.phoenix.jdbc.PhoenixStatement;
 import org.apache.phoenix.jdbc.PhoenixStatement.Operation;
+import org.apache.phoenix.optimize.Cost;
 import org.apache.phoenix.parse.FilterableStatement;
 import org.apache.phoenix.parse.PFunction;
 import org.apache.phoenix.parse.PSchema;
@@ -486,6 +487,11 @@ public class ParallelIteratorsSplitTest extends BaseConnectionlessQueryTest {
             public Long getEstimateInfoTimestamp() throws SQLException {
                 return null;
             }
+
+            @Override
+            public Cost getCost() {
+                return Cost.ZERO;
+            }
             
         }, null, new SpoolingResultIterator.SpoolingResultIteratorFactory(context.getConnection().getQueryServices()), context.getScan(), false, null);
         List<KeyRange> keyRanges = parallelIterators.getSplits();


[10/16] phoenix git commit: PHOENIX-4198 Remove the need for users to have access to the Phoenix SYSTEM tables to create tables

Posted by ja...@apache.org.
PHOENIX-4198 Remove the need for users to have access to the Phoenix SYSTEM tables to create tables


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

Branch: refs/heads/4.x-HBase-1.2
Commit: 493a54f9fdca2c7dcdae8802ca1208b1e8fdfd33
Parents: d523350
Author: Ankit Singhal <an...@gmail.com>
Authored: Thu Nov 9 02:37:55 2017 +0000
Committer: James Taylor <jt...@salesforce.com>
Committed: Sat Dec 16 16:42:54 2017 -0800

----------------------------------------------------------------------
 .../phoenix/end2end/TableDDLPermissionsIT.java  | 692 +++++++++++++++++++
 .../org/apache/hadoop/hbase/ipc/RpcUtil.java    |  32 +
 .../BaseMetaDataEndpointObserver.java           | 111 +++
 .../coprocessor/MetaDataEndpointImpl.java       | 339 +++++++--
 .../coprocessor/MetaDataEndpointObserver.java   |  68 ++
 .../coprocessor/MetaDataRegionObserver.java     |  17 +-
 .../coprocessor/PhoenixAccessController.java    | 628 +++++++++++++++++
 .../PhoenixMetaDataCoprocessorHost.java         | 236 +++++++
 .../index/PhoenixIndexFailurePolicy.java        | 109 +--
 .../query/ConnectionQueryServicesImpl.java      |  15 +-
 .../org/apache/phoenix/query/QueryServices.java |   4 +
 .../phoenix/query/QueryServicesOptions.java     |  14 +-
 .../phoenix/schema/stats/StatisticsWriter.java  |  42 +-
 .../org/apache/phoenix/util/MetaDataUtil.java   |  18 +
 .../org/apache/phoenix/util/SchemaUtil.java     |  12 +
 15 files changed, 2196 insertions(+), 141 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/493a54f9/phoenix-core/src/it/java/org/apache/phoenix/end2end/TableDDLPermissionsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TableDDLPermissionsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TableDDLPermissionsIT.java
new file mode 100644
index 0000000..971383b
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TableDDLPermissionsIT.java
@@ -0,0 +1,692 @@
+/*
+ * 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.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.lang.reflect.UndeclaredThrowableException;
+import java.security.PrivilegedExceptionAction;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.AuthUtil;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.security.AccessDeniedException;
+import org.apache.hadoop.hbase.security.access.AccessControlClient;
+import org.apache.hadoop.hbase.security.access.Permission.Action;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.phoenix.exception.PhoenixIOException;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.util.MetaDataUtil;
+import org.apache.phoenix.util.SchemaUtil;
+import org.junit.After;
+import org.junit.BeforeClass;
+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;
+
+import com.google.common.collect.Maps;
+
+/**
+ * Test that verifies a user can read Phoenix tables with a minimal set of permissions.
+ */
+@Category(NeedsOwnMiniClusterTest.class)
+@RunWith(Parameterized.class)
+public class TableDDLPermissionsIT{
+    private static String SUPERUSER;
+
+    private static HBaseTestingUtility testUtil;
+
+    private static final Set<String> PHOENIX_SYSTEM_TABLES = new HashSet<>(Arrays.asList(
+            "SYSTEM.CATALOG", "SYSTEM.SEQUENCE", "SYSTEM.STATS", "SYSTEM.FUNCTION",
+                "SYSTEM.MUTEX"));
+    // PHOENIX-XXXX SYSTEM.MUTEX isn't being created in the SYSTEM namespace as it should be.
+    private static final Set<String> PHOENIX_NAMESPACE_MAPPED_SYSTEM_TABLES = new HashSet<>(
+            Arrays.asList("SYSTEM:CATALOG", "SYSTEM:SEQUENCE", "SYSTEM:STATS", "SYSTEM:FUNCTION",
+                "SYSTEM.MUTEX"));
+    private static final String GROUP_SYSTEM_ACCESS = "group_system_access";
+    final UserGroupInformation superUser = UserGroupInformation.createUserForTesting(SUPERUSER, new String[0]);
+    final UserGroupInformation superUser2 = UserGroupInformation.createUserForTesting("superuser", new String[0]);
+    final UserGroupInformation regularUser = UserGroupInformation.createUserForTesting("user",  new String[0]);
+    final UserGroupInformation groupUser = UserGroupInformation.createUserForTesting("user2", new String[] { GROUP_SYSTEM_ACCESS });
+    final UserGroupInformation unprivilegedUser = UserGroupInformation.createUserForTesting("unprivilegedUser",
+            new String[0]);
+
+
+    private static final int NUM_RECORDS = 5;
+
+    private boolean isNamespaceMapped;
+
+    public TableDDLPermissionsIT(final boolean isNamespaceMapped) throws Exception {
+        this.isNamespaceMapped = isNamespaceMapped;
+        Map<String, String> clientProps = Maps.newHashMapWithExpectedSize(1);
+        clientProps.put(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, "true");
+    }
+
+    private void startNewMiniCluster(Configuration overrideConf) throws Exception{
+        if (null != testUtil) {
+            testUtil.shutdownMiniCluster();
+            testUtil = null;
+        }
+        testUtil = new HBaseTestingUtility();
+
+        Configuration config = testUtil.getConfiguration();
+        
+        config.set("hbase.coprocessor.master.classes",
+                "org.apache.hadoop.hbase.security.access.AccessController");
+        config.set("hbase.coprocessor.region.classes",
+                "org.apache.hadoop.hbase.security.access.AccessController");
+        config.set("hbase.coprocessor.regionserver.classes",
+                "org.apache.hadoop.hbase.security.access.AccessController");
+        config.set("hbase.security.exec.permission.checks", "true");
+        config.set("hbase.security.authorization", "true");
+        config.set("hbase.superuser", SUPERUSER+","+superUser2.getShortUserName());
+        config.set("hbase.regionserver.wal.codec", "org.apache.hadoop.hbase.regionserver.wal.IndexedWALEditCodec");
+        config.set(QueryServices.PHOENIX_ACLS_ENABLED,"true");
+        config.set(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, Boolean.toString(isNamespaceMapped));
+        // Avoid multiple clusters trying to bind the master's info port (16010)
+        config.setInt(HConstants.MASTER_INFO_PORT, -1);
+        
+        if (overrideConf != null) {
+            config.addResource(overrideConf);
+        }
+        testUtil.startMiniCluster(1);
+    }
+    
+    private void grantSystemTableAccess() throws Exception{
+        try (Connection conn = getConnection()) {
+            if (isNamespaceMapped) {
+                grantPermissions(regularUser.getShortUserName(), PHOENIX_NAMESPACE_MAPPED_SYSTEM_TABLES, Action.READ,
+                        Action.EXEC);
+                grantPermissions(unprivilegedUser.getShortUserName(), PHOENIX_NAMESPACE_MAPPED_SYSTEM_TABLES,
+                        Action.READ, Action.EXEC);
+                grantPermissions(AuthUtil.toGroupEntry(GROUP_SYSTEM_ACCESS), PHOENIX_NAMESPACE_MAPPED_SYSTEM_TABLES,
+                        Action.READ, Action.EXEC);
+                // Local Index requires WRITE permission on SYSTEM.SEQUENCE TABLE.
+                grantPermissions(regularUser.getShortUserName(), Collections.singleton("SYSTEM:SEQUENCE"), Action.WRITE,
+                        Action.READ, Action.EXEC);
+                grantPermissions(unprivilegedUser.getShortUserName(), Collections.singleton("SYSTEM:SEQUENCE"), Action.WRITE,
+                        Action.READ, Action.EXEC);
+                
+            } else {
+                grantPermissions(regularUser.getShortUserName(), PHOENIX_SYSTEM_TABLES, Action.READ, Action.EXEC);
+                grantPermissions(unprivilegedUser.getShortUserName(), PHOENIX_SYSTEM_TABLES, Action.READ, Action.EXEC);
+                grantPermissions(AuthUtil.toGroupEntry(GROUP_SYSTEM_ACCESS), PHOENIX_SYSTEM_TABLES, Action.READ, Action.EXEC);
+                // Local Index requires WRITE permission on SYSTEM.SEQUENCE TABLE.
+                grantPermissions(regularUser.getShortUserName(), Collections.singleton("SYSTEM.SEQUENCE"), Action.WRITE,
+                        Action.READ, Action.EXEC);
+                grantPermissions(unprivilegedUser.getShortUserName(), Collections.singleton("SYSTEM:SEQUENCE"), Action.WRITE,
+                        Action.READ, Action.EXEC);
+            }
+        } catch (Throwable e) {
+            if (e instanceof Exception) {
+                throw (Exception)e;
+            } else {
+                throw new Exception(e);
+            }
+        }
+    }
+
+    @Parameters(name = "isNamespaceMapped={0}") // name is used by failsafe as file name in reports
+    public static Collection<Boolean> data() {
+        return Arrays.asList(true, false);
+    }
+
+    @BeforeClass
+    public static void doSetup() throws Exception {
+        SUPERUSER = System.getProperty("user.name");
+        //setUpTestDriver(new ReadOnlyProps(serverProps.entrySet().iterator()), new ReadOnlyProps(clientProps.entrySet().iterator()));
+    }
+
+    protected static String getUrl() {
+        return "jdbc:phoenix:localhost:" + testUtil.getZkCluster().getClientPort() + ":/hbase";
+    }
+
+    public Connection getConnection() throws SQLException{
+        Properties props = new Properties();
+        props.setProperty(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, Boolean.toString(isNamespaceMapped));
+        return DriverManager.getConnection(getUrl(),props);
+    }
+
+    @Test
+    public void testSchemaPermissions() throws Throwable{
+
+        if (!isNamespaceMapped) { return; }
+        try {
+            startNewMiniCluster(null);
+            grantSystemTableAccess();
+            final String schemaName = "TEST_SCHEMA_PERMISSION";
+            superUser.doAs(new PrivilegedExceptionAction<Void>() {
+                @Override
+                public Void run() throws Exception {
+                    try {
+                        AccessControlClient.grant(getUtility().getConnection(), regularUser.getShortUserName(),
+                                Action.ADMIN);
+                    } catch (Throwable e) {
+                        if (e instanceof Exception) {
+                            throw (Exception)e;
+                        } else {
+                            throw new Exception(e);
+                        }
+                    }
+                    return null;
+                }
+            });
+            verifyAllowed(createSchema(schemaName), regularUser);
+            // Unprivileged user cannot drop a schema
+            verifyDenied(dropSchema(schemaName), unprivilegedUser);
+            verifyDenied(createSchema(schemaName), unprivilegedUser);
+
+            verifyAllowed(dropSchema(schemaName), regularUser);
+        } finally {
+            revokeAll();
+        }
+    }
+
+    @Test
+    public void testAutomaticGrantDisabled() throws Throwable{
+        testIndexAndView(false);
+    }
+    
+    public void testIndexAndView(boolean isAutomaticGrant) throws Throwable {
+        Configuration conf = new Configuration();
+        conf.set(QueryServices.PHOENIX_AUTOMATIC_GRANT_ENABLED, Boolean.toString(isAutomaticGrant));
+        startNewMiniCluster(conf);
+        final String schema = "TEST_INDEX_VIEW";
+        final String tableName = "TABLE_DDL_PERMISSION_IT";
+        final String phoenixTableName = schema + "." + tableName;
+        final String indexName1 = tableName + "_IDX1";
+        final String indexName2 = tableName + "_IDX2";
+        final String lIndexName1 = tableName + "_LIDX1";
+        final String viewName1 = schema+"."+tableName + "_V1";
+        final String viewName2 = schema+"."+tableName + "_V2";
+        final String viewName3 = schema+"."+tableName + "_V3";
+        final String viewName4 = schema+"."+tableName + "_V4";
+        final String viewIndexName1 = tableName + "_VIDX1";
+        final String viewIndexName2 = tableName + "_VIDX2";
+        grantSystemTableAccess();
+        try {
+            superUser.doAs(new PrivilegedExceptionAction<Void>() {
+                @Override
+                public Void run() throws Exception {
+                    try {
+                        verifyAllowed(createSchema(schema), superUser);
+                        if (isNamespaceMapped) {
+                            grantPermissions(regularUser.getShortUserName(), schema, Action.CREATE);
+                            grantPermissions(AuthUtil.toGroupEntry(GROUP_SYSTEM_ACCESS), schema, Action.CREATE);
+
+                        } else {
+                            grantPermissions(regularUser.getShortUserName(),
+                                    NamespaceDescriptor.DEFAULT_NAMESPACE.getName(), Action.CREATE);
+                            grantPermissions(AuthUtil.toGroupEntry(GROUP_SYSTEM_ACCESS),
+                                    NamespaceDescriptor.DEFAULT_NAMESPACE.getName(), Action.CREATE);
+
+                        }
+                    } catch (Throwable e) {
+                        if (e instanceof Exception) {
+                            throw (Exception)e;
+                        } else {
+                            throw new Exception(e);
+                        }
+                    }
+                    return null;
+                }
+            });
+
+            verifyAllowed(createTable(phoenixTableName), regularUser);
+            verifyAllowed(createIndex(indexName1, phoenixTableName), regularUser);
+            verifyAllowed(createView(viewName1, phoenixTableName), regularUser);
+            verifyAllowed(createLocalIndex(lIndexName1, phoenixTableName), regularUser);
+            verifyAllowed(createIndex(viewIndexName1, viewName1), regularUser);
+            verifyAllowed(createIndex(viewIndexName2, viewName1), regularUser);
+            verifyAllowed(createView(viewName4, viewName1), regularUser);
+            verifyAllowed(readTable(phoenixTableName), regularUser);
+
+            verifyDenied(createIndex(indexName2, phoenixTableName), unprivilegedUser);
+            verifyDenied(createView(viewName2, phoenixTableName), unprivilegedUser);
+            verifyDenied(createView(viewName3, viewName1), unprivilegedUser);
+            verifyDenied(dropView(viewName1), unprivilegedUser);
+            
+            verifyDenied(dropIndex(indexName1, phoenixTableName), unprivilegedUser);
+            verifyDenied(dropTable(phoenixTableName), unprivilegedUser);
+            verifyDenied(rebuildIndex(indexName1, phoenixTableName), unprivilegedUser);
+            verifyDenied(addColumn(phoenixTableName, "val1"), unprivilegedUser);
+            verifyDenied(dropColumn(phoenixTableName, "val"), unprivilegedUser);
+            verifyDenied(addProperties(phoenixTableName, "GUIDE_POSTS_WIDTH", "100"), unprivilegedUser);
+
+            // Granting read permission to unprivileged user, now he should be able to create view but not index
+            grantPermissions(unprivilegedUser.getShortUserName(),
+                    Collections.singleton(
+                            SchemaUtil.getPhysicalHBaseTableName(schema, tableName, isNamespaceMapped).getString()),
+                    Action.READ, Action.EXEC);
+            grantPermissions(AuthUtil.toGroupEntry(GROUP_SYSTEM_ACCESS),
+                    Collections.singleton(
+                            SchemaUtil.getPhysicalHBaseTableName(schema, tableName, isNamespaceMapped).getString()),
+                    Action.READ, Action.EXEC);
+            verifyDenied(createIndex(indexName2, phoenixTableName), unprivilegedUser);
+            if (!isAutomaticGrant) {
+                // Automatic grant will read access for all indexes
+                verifyDenied(createView(viewName2, phoenixTableName), unprivilegedUser);
+
+                // Granting read permission to unprivileged user on index so that a new view can read a index as well,
+                // now
+                // he should be able to create view but not index
+                grantPermissions(unprivilegedUser.getShortUserName(),
+                        Collections.singleton(SchemaUtil
+                                .getPhysicalHBaseTableName(schema, indexName1, isNamespaceMapped).getString()),
+                        Action.READ, Action.EXEC);
+                verifyDenied(createView(viewName3, viewName1), unprivilegedUser);
+            }
+            
+            verifyAllowed(createView(viewName2, phoenixTableName), unprivilegedUser);
+            
+            if (!isAutomaticGrant) {
+                // Grant access to view index for parent view
+                grantPermissions(unprivilegedUser.getShortUserName(),
+                        Collections.singleton(Bytes.toString(MetaDataUtil.getViewIndexPhysicalName(SchemaUtil
+                                .getPhysicalHBaseTableName(schema, tableName, isNamespaceMapped).getBytes()))),
+                        Action.READ, Action.EXEC);
+            }
+            verifyAllowed(createView(viewName3, viewName1), unprivilegedUser);
+            
+            // Grant create permission in namespace
+            if (isNamespaceMapped) {
+                grantPermissions(unprivilegedUser.getShortUserName(), schema, Action.CREATE);
+            } else {
+                grantPermissions(unprivilegedUser.getShortUserName(), NamespaceDescriptor.DEFAULT_NAMESPACE.getName(),
+                        Action.CREATE);
+            }
+            if (!isAutomaticGrant) {
+                verifyDenied(createIndex(indexName2, phoenixTableName), unprivilegedUser);
+                // Give user of data table access to index table which will be created by unprivilegedUser
+                grantPermissions(regularUser.getShortUserName(),
+                        Collections.singleton(SchemaUtil
+                                .getPhysicalHBaseTableName(schema, indexName2, isNamespaceMapped).getString()),
+                        Action.WRITE);
+                verifyDenied(createIndex(indexName2, phoenixTableName), unprivilegedUser);
+                grantPermissions(regularUser.getShortUserName(),
+                        Collections.singleton(SchemaUtil
+                                .getPhysicalHBaseTableName(schema, indexName2, isNamespaceMapped).getString()),
+                        Action.WRITE, Action.READ, Action.CREATE, Action.EXEC, Action.ADMIN);
+            }
+            // we should be able to read the data from another index as well to which we have not given any access to
+            // this user
+            verifyAllowed(createIndex(indexName2, phoenixTableName), unprivilegedUser);
+            verifyAllowed(readTable(phoenixTableName, indexName1), unprivilegedUser);
+            verifyAllowed(readTable(phoenixTableName, indexName2), unprivilegedUser);
+            verifyAllowed(rebuildIndex(indexName2, phoenixTableName), unprivilegedUser);
+
+            // data table user should be able to read new index
+            verifyAllowed(rebuildIndex(indexName2, phoenixTableName), regularUser);
+            verifyAllowed(readTable(phoenixTableName, indexName2), regularUser);
+
+            verifyAllowed(readTable(phoenixTableName), regularUser);
+            verifyAllowed(rebuildIndex(indexName1, phoenixTableName), regularUser);
+            verifyAllowed(addColumn(phoenixTableName, "val1"), regularUser);
+            verifyAllowed(addProperties(phoenixTableName, "GUIDE_POSTS_WIDTH", "100"), regularUser);
+            verifyAllowed(dropView(viewName1), regularUser);
+            verifyAllowed(dropView(viewName2), regularUser);
+            verifyAllowed(dropColumn(phoenixTableName, "val1"), regularUser);
+            verifyAllowed(dropIndex(indexName2, phoenixTableName), regularUser);
+            verifyAllowed(dropIndex(indexName1, phoenixTableName), regularUser);
+            verifyAllowed(dropTable(phoenixTableName), regularUser);
+
+            // check again with super users
+            verifyAllowed(createTable(phoenixTableName), superUser2);
+            verifyAllowed(createIndex(indexName1, phoenixTableName), superUser2);
+            verifyAllowed(createView(viewName1, phoenixTableName), superUser2);
+            verifyAllowed(readTable(phoenixTableName), superUser2);
+            verifyAllowed(dropView(viewName1), superUser2);
+            verifyAllowed(dropTable(phoenixTableName), superUser2);
+
+        } finally {
+            revokeAll();
+        }
+    }
+    
+    
+    @Test
+    public void testAutomaticGrantEnabled() throws Throwable{
+        testIndexAndView(true);
+    }
+
+    private void revokeAll() throws IOException, Throwable {
+        AccessControlClient.revoke(getUtility().getConnection(), AuthUtil.toGroupEntry(GROUP_SYSTEM_ACCESS),Action.values() );
+        AccessControlClient.revoke(getUtility().getConnection(), regularUser.getShortUserName(),Action.values() );
+        AccessControlClient.revoke(getUtility().getConnection(), unprivilegedUser.getShortUserName(),Action.values() );
+        
+    }
+
+    protected void grantPermissions(String groupEntry, Action... actions) throws IOException, Throwable {
+        AccessControlClient.grant(getUtility().getConnection(), groupEntry, actions);
+    }
+
+    private AccessTestAction dropTable(final String tableName) throws SQLException {
+        return new AccessTestAction() {
+            @Override
+            public Object run() throws Exception {
+                try (Connection conn = getConnection(); Statement stmt = conn.createStatement();) {
+                    assertFalse(stmt.execute("DROP TABLE IF EXISTS " + tableName));
+                }
+                return null;
+            }
+        };
+
+    }
+
+    private AccessTestAction createTable(final String tableName) throws SQLException {
+        return new AccessTestAction() {
+            @Override
+            public Object run() throws Exception {
+        try (Connection conn = getConnection(); Statement stmt = conn.createStatement();) {
+            assertFalse(stmt.execute("CREATE TABLE " + tableName + "(pk INTEGER not null primary key, data VARCHAR,val integer)"));
+            try (PreparedStatement pstmt = conn.prepareStatement("UPSERT INTO " + tableName + " values(?, ?, ?)")) {
+                for (int i = 0; i < NUM_RECORDS; i++) {
+                    pstmt.setInt(1, i);
+                    pstmt.setString(2, Integer.toString(i));
+                    pstmt.setInt(3, i);
+                    assertEquals(1, pstmt.executeUpdate());
+                }
+            }
+            conn.commit();
+        }
+        return null;
+            }
+        };
+    }
+
+    private AccessTestAction readTable(final String tableName) throws SQLException {
+        return readTable(tableName,null);
+    }
+    private AccessTestAction readTable(final String tableName, final String indexName) throws SQLException {
+        return new AccessTestAction() {
+            @Override
+            public Object run() throws Exception {
+                try (Connection conn = getConnection(); Statement stmt = conn.createStatement()) {
+                    ResultSet rs = stmt.executeQuery("SELECT "+(indexName!=null?"/*+ INDEX("+tableName+" "+indexName+")*/":"")+" pk, data,val FROM " + tableName +" where data>='0'");
+                    assertNotNull(rs);
+                    int i = 0;
+                    while (rs.next()) {
+                        assertEquals(i, rs.getInt(1));
+                        assertEquals(Integer.toString(i), rs.getString(2));
+                        assertEquals(i, rs.getInt(3));
+                        i++;
+                    }
+                    assertEquals(NUM_RECORDS, i);
+                }
+                return null;
+                }
+            };
+    }
+
+    public static HBaseTestingUtility getUtility(){
+        return testUtil;
+    }
+
+    private void grantPermissions(String toUser, Set<String> tablesToGrant, Action... actions) throws Throwable {
+        for (String table : tablesToGrant) {
+            AccessControlClient.grant(getUtility().getConnection(), TableName.valueOf(table), toUser, null, null,
+                    actions);
+        }
+    }
+
+    private void grantPermissions(String toUser, String namespace, Action... actions) throws Throwable {
+        AccessControlClient.grant(getUtility().getConnection(), namespace, toUser, actions);
+    }
+    
+
+    private AccessTestAction dropColumn(final String tableName, final String columnName) throws SQLException {
+        return new AccessTestAction() {
+            @Override
+            public Object run() throws Exception {
+                try (Connection conn = getConnection(); Statement stmt = conn.createStatement();) {
+                    assertFalse(stmt.execute("ALTER TABLE " + tableName + " DROP COLUMN "+columnName));
+                }
+                return null;
+            }
+        };
+    }
+
+    private AccessTestAction addColumn(final String tableName, final String columnName) throws SQLException {
+        return new AccessTestAction() {
+            @Override
+            public Object run() throws Exception {
+                try (Connection conn = getConnection(); Statement stmt = conn.createStatement();) {
+                    assertFalse(stmt.execute("ALTER TABLE " + tableName + " ADD "+columnName+" varchar"));
+                }
+                return null;
+            }
+        };
+    }
+
+    private AccessTestAction addProperties(final String tableName, final String property, final String value)
+            throws SQLException {
+        return new AccessTestAction() {
+            @Override
+            public Object run() throws Exception {
+                try (Connection conn = getConnection(); Statement stmt = conn.createStatement();) {
+                    assertFalse(stmt.execute("ALTER TABLE " + tableName + " SET " + property + "=" + value));
+                }
+                return null;
+            }
+        };
+    }
+
+    private AccessTestAction dropView(final String viewName) throws SQLException {
+        return new AccessTestAction() {
+            @Override
+            public Object run() throws Exception {
+                try (Connection conn = getConnection(); Statement stmt = conn.createStatement();) {
+                    assertFalse(stmt.execute("DROP VIEW " + viewName));
+                }
+                return null;
+            }
+        };
+    }
+
+    private AccessTestAction createView(final String viewName, final String dataTable) throws SQLException {
+        return new AccessTestAction() {
+            @Override
+            public Object run() throws Exception {
+                try (Connection conn = getConnection(); Statement stmt = conn.createStatement();) {
+                    assertFalse(stmt.execute("CREATE VIEW " + viewName + " AS SELECT * FROM " + dataTable));
+                }
+                return null;
+            }
+        };
+    }
+
+    private AccessTestAction createIndex(final String indexName, final String dataTable) throws SQLException {
+        return new AccessTestAction() {
+            @Override
+            public Object run() throws Exception {
+
+                try (Connection conn = getConnection(); Statement stmt = conn.createStatement();) {
+                    assertFalse(stmt.execute("CREATE INDEX " + indexName + " on " + dataTable + "(data)"));
+                }
+                return null;
+            }
+        };
+    }
+    
+    private AccessTestAction createLocalIndex(final String indexName, final String dataTable) throws SQLException {
+        return new AccessTestAction() {
+            @Override
+            public Object run() throws Exception {
+
+                try (Connection conn = getConnection(); Statement stmt = conn.createStatement();) {
+                    assertFalse(stmt.execute("CREATE LOCAL INDEX " + indexName + " on " + dataTable + "(data)"));
+                }
+                return null;
+            }
+        };
+    }
+
+    private AccessTestAction dropIndex(final String indexName, final String dataTable) throws SQLException {
+        return new AccessTestAction() {
+            @Override
+            public Object run() throws Exception {
+                try (Connection conn = getConnection(); Statement stmt = conn.createStatement();) {
+                    assertFalse(stmt.execute("DROP INDEX " + indexName + " on " + dataTable));
+                }
+                return null;
+            }
+        };
+    }
+
+    private AccessTestAction createSchema(final String schemaName) throws SQLException {
+        return new AccessTestAction() {
+            @Override
+            public Object run() throws Exception {
+                if (isNamespaceMapped) {
+                    try (Connection conn = getConnection(); Statement stmt = conn.createStatement();) {
+                        assertFalse(stmt.execute("CREATE SCHEMA " + schemaName));
+                    }
+                }
+                return null;
+            }
+        };
+    }
+
+    private AccessTestAction dropSchema(final String schemaName) throws SQLException {
+        return new AccessTestAction() {
+            @Override
+            public Object run() throws Exception {
+                if (isNamespaceMapped) {
+                    try (Connection conn = getConnection(); Statement stmt = conn.createStatement();) {
+                        assertFalse(stmt.execute("DROP SCHEMA " + schemaName));
+                    }
+                }
+                return null;
+            }
+        };
+    }
+
+    private AccessTestAction rebuildIndex(final String indexName, final String dataTable) throws SQLException {
+        return new AccessTestAction() {
+            @Override
+            public Object run() throws Exception {
+                try (Connection conn = getConnection(); Statement stmt = conn.createStatement();) {
+                    assertFalse(stmt.execute("ALTER INDEX " + indexName + " on " + dataTable + " DISABLE"));
+                    assertFalse(stmt.execute("ALTER INDEX " + indexName + " on " + dataTable + " REBUILD"));
+                }
+                return null;
+            }
+        };
+    }
+
+    static interface AccessTestAction extends PrivilegedExceptionAction<Object> { }
+
+    @After
+    public void cleanup() throws Exception {
+        if (null != testUtil) {
+          testUtil.shutdownMiniCluster();
+          testUtil = null;
+        }
+    }
+
+    /** This fails only in case of ADE or empty list for any of the users. */
+    private void verifyAllowed(AccessTestAction action, UserGroupInformation... users) throws Exception {
+      for (UserGroupInformation user : users) {
+        verifyAllowed(user, action);
+      }
+    }
+
+    /** This passes only in case of ADE for all users. */
+    private void verifyDenied(AccessTestAction action, UserGroupInformation... users) throws Exception {
+      for (UserGroupInformation user : users) {
+        verifyDenied(user, action);
+      }
+    }
+
+    /** This fails only in case of ADE or empty list for any of the actions. */
+    private void verifyAllowed(UserGroupInformation user, AccessTestAction... actions) throws Exception {
+      for (AccessTestAction action : actions) {
+        try {
+          Object obj = user.doAs(action);
+          if (obj != null && obj instanceof List<?>) {
+            List<?> results = (List<?>) obj;
+            if (results != null && results.isEmpty()) {
+              fail("Empty non null results from action for user '" + user.getShortUserName() + "'");
+            }
+          }
+        } catch (AccessDeniedException ade) {
+          fail("Expected action to pass for user '" + user.getShortUserName() + "' but was denied");
+        }
+      }
+    }
+
+    /** This passes only in case of ADE for all actions. */
+    private void verifyDenied(UserGroupInformation user, AccessTestAction... actions) throws Exception {
+        for (AccessTestAction action : actions) {
+            try {
+                user.doAs(action);
+                fail("Expected exception was not thrown for user '" + user.getShortUserName() + "'");
+            } catch (IOException e) {
+                fail("Expected exception was not thrown for user '" + user.getShortUserName() + "'");
+            } catch (UndeclaredThrowableException ute) {
+                Throwable ex = ute.getUndeclaredThrowable();
+
+                if (ex instanceof PhoenixIOException) {
+                    if (ex.getCause() instanceof AccessDeniedException) {
+                        // expected result
+                        validateAccessDeniedException((AccessDeniedException) ex.getCause());
+                        return;
+                    }
+                }
+            }catch(RuntimeException ex){
+                // This can occur while accessing tabledescriptors from client by the unprivileged user
+                if (ex.getCause() instanceof AccessDeniedException) {
+                    // expected result
+                    validateAccessDeniedException((AccessDeniedException) ex.getCause());
+                    return;
+                }
+            }
+            fail("Expected exception was not thrown for user '" + user.getShortUserName() + "'");
+        }
+    }
+
+    private void validateAccessDeniedException(AccessDeniedException ade) {
+        String msg = ade.getMessage();
+        assertTrue("Exception contained unexpected message: '" + msg + "'",
+            !msg.contains("is not the scanner owner"));
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/493a54f9/phoenix-core/src/main/java/org/apache/hadoop/hbase/ipc/RpcUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/ipc/RpcUtil.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/ipc/RpcUtil.java
new file mode 100644
index 0000000..ac281f1
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/ipc/RpcUtil.java
@@ -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.
+ */
+package org.apache.hadoop.hbase.ipc;
+
+import org.apache.hadoop.hbase.ipc.RpcServer.Call;
+
+public class RpcUtil {
+
+    public static Call getRpcContext() {
+        return RpcServer.CurCall.get();
+    }
+    
+    public static void setRpcContext(Call c){
+        RpcServer.CurCall.set(c);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/493a54f9/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseMetaDataEndpointObserver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseMetaDataEndpointObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseMetaDataEndpointObserver.java
new file mode 100644
index 0000000..8decc8c
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseMetaDataEndpointObserver.java
@@ -0,0 +1,111 @@
+/*
+ * 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.coprocessor;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.hadoop.hbase.CoprocessorEnvironment;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.phoenix.coprocessor.PhoenixMetaDataCoprocessorHost.PhoenixMetaDataControllerEnvironment;
+import org.apache.phoenix.schema.PIndexState;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTableType;
+
+public class BaseMetaDataEndpointObserver implements MetaDataEndpointObserver{
+
+    @Override
+    public void start(CoprocessorEnvironment env) throws IOException {
+
+    }
+
+    @Override
+    public void stop(CoprocessorEnvironment env) throws IOException {
+
+    }
+
+    @Override
+    public void preGetTable(
+            org.apache.hadoop.hbase.coprocessor.ObserverContext<PhoenixMetaDataControllerEnvironment> ctx,
+            String tenantId, String tableName, TableName physicalTableName) throws IOException {
+
+    }
+
+    
+    @Override
+    public void preCreateTable(ObserverContext<PhoenixMetaDataControllerEnvironment> ctx, String tenantId,
+            String tableName, TableName physicalTableName, TableName parentPhysicalTableName, PTableType tableType,
+            Set<byte[]> familySet, Set<TableName> indexes) throws IOException {
+
+    }
+
+    @Override
+    public void preDropTable(ObserverContext<PhoenixMetaDataControllerEnvironment> ctx, String tenantId,
+            String tableName, TableName physicalTableName, TableName parentPhysicalTableName, PTableType tableType,
+            List<PTable> indexes) throws IOException {
+
+    }
+
+    @Override
+    public void preAlterTable(ObserverContext<PhoenixMetaDataControllerEnvironment> ctx, String tenantId,
+            String tableName, TableName physicalTableName, TableName parentPhysicalTableName, PTableType type) throws IOException {
+
+    }
+
+    @Override
+    public void preGetSchema(ObserverContext<PhoenixMetaDataControllerEnvironment> ctx, String schemaName)
+            throws IOException {
+
+    }
+
+    @Override
+    public void preCreateSchema(ObserverContext<PhoenixMetaDataControllerEnvironment> ctx, String schemaName)
+            throws IOException {
+
+    }
+
+    @Override
+    public void preDropSchema(ObserverContext<PhoenixMetaDataControllerEnvironment> ctx, String schemaName) throws IOException {
+
+    }
+
+    @Override
+    public void preCreateFunction(ObserverContext<PhoenixMetaDataControllerEnvironment> ctx, String tenantId,
+            String functionName) throws IOException {
+
+    }
+
+    @Override
+    public void preDropFunction(ObserverContext<PhoenixMetaDataControllerEnvironment> ctx, String tenantId, String functionName)
+            throws IOException {}
+
+    @Override
+    public void preGetFunctions(ObserverContext<PhoenixMetaDataControllerEnvironment> ctx, String tenantId, String functionName)
+            throws IOException {
+
+    }
+
+    @Override
+    public void preIndexUpdate(ObserverContext<PhoenixMetaDataControllerEnvironment> ctx, String tenantId,
+            String indexName, TableName physicalTableName, TableName parentPhysicalTableName, PIndexState newState)
+            throws IOException {
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/493a54f9/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
index d05ab79..afbd63f 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
@@ -84,6 +84,7 @@ import static org.apache.phoenix.util.SchemaUtil.getVarCharLength;
 import static org.apache.phoenix.util.SchemaUtil.getVarChars;
 
 import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
 import java.sql.ResultSetMetaData;
 import java.sql.SQLException;
 import java.sql.Statement;
@@ -91,10 +92,12 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.Comparator;
+import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.NavigableMap;
+import java.util.Set;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
@@ -105,6 +108,7 @@ import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValue.Type;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.HTableInterface;
@@ -121,9 +125,12 @@ import org.apache.hadoop.hbase.filter.FilterList;
 import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
 import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.ipc.RpcServer.Call;
+import org.apache.hadoop.hbase.ipc.RpcUtil;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.Region.RowLock;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
+import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.VersionInfo;
@@ -452,7 +459,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
     private static final int DEFAULT_VALUE_INDEX = FUNCTION_ARG_KV_COLUMNS.indexOf(DEFAULT_VALUE_KV);
     private static final int MIN_VALUE_INDEX = FUNCTION_ARG_KV_COLUMNS.indexOf(MIN_VALUE_KV);
     private static final int MAX_VALUE_INDEX = FUNCTION_ARG_KV_COLUMNS.indexOf(MAX_VALUE_KV);
-    
+
     private static PName newPName(byte[] keyBuffer, int keyOffset, int keyLength) {
         if (keyLength <= 0) {
             return null;
@@ -463,6 +470,9 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
 
     private RegionCoprocessorEnvironment env;
 
+    private PhoenixMetaDataCoprocessorHost phoenixAccessCoprocessorHost;
+    private boolean accessCheckEnabled;
+
     /**
      * Stores a reference to the coprocessor environment provided by the
      * {@link org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost} from the region where this
@@ -480,6 +490,10 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         } else {
             throw new CoprocessorException("Must be loaded on a table region!");
         }
+        
+        phoenixAccessCoprocessorHost = new PhoenixMetaDataCoprocessorHost(this.env);
+        this.accessCheckEnabled = env.getConfiguration().getBoolean(QueryServices.PHOENIX_ACLS_ENABLED,
+                QueryServicesOptions.DEFAULT_PHOENIX_ACLS_ENABLED);
         logger.info("Starting Tracing-Metrics Systems");
         // Start the phoenix trace collection
         Tracing.addTraceMetricsSource();
@@ -523,6 +537,9 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 done.run(builder.build());
                 return;
             }
+            getCoprocessorHost().preGetTable(Bytes.toString(tenantId), SchemaUtil.getTableName(schemaName, tableName),
+                    TableName.valueOf(table.getPhysicalName().getBytes()));
+
             builder.setReturnCode(MetaDataProtos.MutationCode.TABLE_ALREADY_EXISTS);
             long disableIndexTimestamp = table.getIndexDisableTimestamp();
             long minNonZerodisableIndexTimestamp = disableIndexTimestamp > 0 ? disableIndexTimestamp : Long.MAX_VALUE;
@@ -554,6 +571,10 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         }
     }
 
+    private PhoenixMetaDataCoprocessorHost getCoprocessorHost() {
+        return phoenixAccessCoprocessorHost;
+    }
+
     private PTable buildTable(byte[] key, ImmutableBytesPtr cacheKey, Region region,
             long clientTimeStamp, int clientVersion) throws IOException, SQLException {
         Scan scan = MetaDataUtil.newTableRowsScan(key, MIN_TABLE_TIMESTAMP, clientTimeStamp);
@@ -1317,12 +1338,14 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
      * @return null if the physical table row information is not present.
      * 
      */
-    private static Mutation getPhysicalTableForView(List<Mutation> tableMetadata, byte[][] parentSchemaTableNames) {
+    private static Mutation getPhysicalTableRowForView(List<Mutation> tableMetadata, byte[][] parentTenantSchemaTableNames, byte[][] physicalSchemaTableNames) {
         int size = tableMetadata.size();
         byte[][] rowKeyMetaData = new byte[3][];
         MetaDataUtil.getTenantIdAndSchemaAndTableName(tableMetadata, rowKeyMetaData);
         Mutation physicalTableRow = null;
+        Mutation parentTableRow = null;
         boolean physicalTableLinkFound = false;
+        boolean parentTableLinkFound = false;
         if (size >= 2) {
             int i = size - 1;
             while (i >= 1) {
@@ -1332,28 +1355,51 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     if (linkType == LinkType.PHYSICAL_TABLE) {
                         physicalTableRow = m;
                         physicalTableLinkFound = true;
-                        break;
                     }
+                    if (linkType == LinkType.PARENT_TABLE) {
+                        parentTableRow=m;
+                        parentTableLinkFound = true;
+                    }
+                }
+                if(physicalTableLinkFound && parentTableLinkFound){
+                    break;
                 }
                 i--;
             }
         }
+        if (!parentTableLinkFound) {
+            parentTenantSchemaTableNames[0] = null;
+            parentTenantSchemaTableNames[1] = null;
+            parentTenantSchemaTableNames[2] = null;
+            
+        }
         if (!physicalTableLinkFound) {
-            parentSchemaTableNames[0] = null;
-            parentSchemaTableNames[1] = null;
-            return null;
+            physicalSchemaTableNames[0] = null;
+            physicalSchemaTableNames[1] = null;
+            physicalSchemaTableNames[2] = null;
+        }
+        if (physicalTableLinkFound) {
+            getSchemaTableNames(physicalTableRow,physicalSchemaTableNames);
+        }
+        if (parentTableLinkFound) {
+            getSchemaTableNames(parentTableRow,parentTenantSchemaTableNames);   
         }
-        rowKeyMetaData = new byte[5][];
-        getVarChars(physicalTableRow.getRow(), 5, rowKeyMetaData);
+        return physicalTableRow;
+    }
+    
+    private static void getSchemaTableNames(Mutation row, byte[][] schemaTableNames) {
+        byte[][] rowKeyMetaData = new byte[5][];
+        getVarChars(row.getRow(), 5, rowKeyMetaData);
+        byte[] tenantId = rowKeyMetaData[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
         byte[] colBytes = rowKeyMetaData[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX];
         byte[] famBytes = rowKeyMetaData[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX];
         if ((colBytes == null || colBytes.length == 0) && (famBytes != null && famBytes.length > 0)) {
             byte[] sName = SchemaUtil.getSchemaNameFromFullName(famBytes).getBytes();
             byte[] tName = SchemaUtil.getTableNameFromFullName(famBytes).getBytes();
-            parentSchemaTableNames[0] = sName;
-            parentSchemaTableNames[1] = tName;
+            schemaTableNames[0]= tenantId;
+            schemaTableNames[1] = sName;
+            schemaTableNames[2] = tName;
         }
-        return physicalTableRow;
     }
     
     @Override
@@ -1370,25 +1416,76 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             byte[] tenantIdBytes = rowKeyMetaData[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
             schemaName = rowKeyMetaData[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX];
             tableName = rowKeyMetaData[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
-
+            boolean isNamespaceMapped = MetaDataUtil.isNameSpaceMapped(tableMetadata, GenericKeyValueBuilder.INSTANCE,
+                    new ImmutableBytesWritable());
+            final IndexType indexType = MetaDataUtil.getIndexType(tableMetadata, GenericKeyValueBuilder.INSTANCE,
+                    new ImmutableBytesWritable());
             byte[] parentSchemaName = null;
             byte[] parentTableName = null;
             PTableType tableType = MetaDataUtil.getTableType(tableMetadata, GenericKeyValueBuilder.INSTANCE, new ImmutableBytesWritable());
             byte[] parentTableKey = null;
             Mutation viewPhysicalTableRow = null;
+            Set<TableName> indexes = new HashSet<TableName>();;
+            byte[] cPhysicalName = SchemaUtil.getPhysicalHBaseTableName(schemaName, tableName, isNamespaceMapped)
+                    .getBytes();
+            byte[] cParentPhysicalName=null;
             if (tableType == PTableType.VIEW) {
-                byte[][] parentSchemaTableNames = new byte[2][];
+                byte[][] parentSchemaTableNames = new byte[3][];
+                byte[][] parentPhysicalSchemaTableNames = new byte[3][];
                 /*
                  * For a view, we lock the base physical table row. For a mapped view, there is 
                  * no link present to the physical table. So the viewPhysicalTableRow is null
                  * in that case.
                  */
-                viewPhysicalTableRow = getPhysicalTableForView(tableMetadata, parentSchemaTableNames);
-                parentSchemaName = parentSchemaTableNames[0];
-                parentTableName = parentSchemaTableNames[1];
-                if (parentTableName != null) {
-                    parentTableKey = SchemaUtil.getTableKey(ByteUtil.EMPTY_BYTE_ARRAY, parentSchemaName, parentTableName);
+                
+                viewPhysicalTableRow = getPhysicalTableRowForView(tableMetadata, parentSchemaTableNames,parentPhysicalSchemaTableNames);
+                long clientTimeStamp = MetaDataUtil.getClientTimeStamp(tableMetadata);
+                if (parentPhysicalSchemaTableNames[2] != null) {
+                    
+                    parentTableKey = SchemaUtil.getTableKey(ByteUtil.EMPTY_BYTE_ARRAY,
+                            parentPhysicalSchemaTableNames[1], parentPhysicalSchemaTableNames[2]);
+                    PTable parentTable = loadTable(env, parentTableKey, new ImmutableBytesPtr(parentTableKey),
+                            clientTimeStamp, clientTimeStamp, clientVersion);
+                    if (parentTable == null) {
+                        builder.setReturnCode(MetaDataProtos.MutationCode.PARENT_TABLE_NOT_FOUND);
+                        builder.setMutationTime(EnvironmentEdgeManager.currentTimeMillis());
+                        done.run(builder.build());
+                        return;
+                    }
+                    cParentPhysicalName = parentTable.getPhysicalName().getBytes();
+                    if (parentSchemaTableNames[2] != null
+                            && Bytes.compareTo(parentSchemaTableNames[2], parentPhysicalSchemaTableNames[2]) != 0) {
+                        // if view is created on view
+                        byte[] parentKey = SchemaUtil.getTableKey(
+                                parentSchemaTableNames[0] == null ? ByteUtil.EMPTY_BYTE_ARRAY : parentSchemaTableNames[0],
+                                parentSchemaTableNames[1], parentSchemaTableNames[2]);
+                        parentTable = loadTable(env, parentKey, new ImmutableBytesPtr(parentKey),
+                                clientTimeStamp, clientTimeStamp, clientVersion);
+                        if (parentTable == null) {
+                            // it could be a global view
+                            parentKey = SchemaUtil.getTableKey(ByteUtil.EMPTY_BYTE_ARRAY,
+                                    parentSchemaTableNames[1], parentSchemaTableNames[2]);
+                            parentTable = loadTable(env, parentKey, new ImmutableBytesPtr(parentKey),
+                                    clientTimeStamp, clientTimeStamp, clientVersion);
+                        }
+                    }
+                    if (parentTable == null) {
+                        builder.setReturnCode(MetaDataProtos.MutationCode.PARENT_TABLE_NOT_FOUND);
+                        builder.setMutationTime(EnvironmentEdgeManager.currentTimeMillis());
+                        done.run(builder.build());
+                        return;
+                    }
+                    for (PTable index : parentTable.getIndexes()) {
+                        indexes.add(TableName.valueOf(index.getPhysicalName().getBytes()));
+                    }
+
+                } else {
+                    // Mapped View
+                    cParentPhysicalName = SchemaUtil.getTableNameAsBytes(schemaName, tableName);
                 }
+                parentSchemaName = parentPhysicalSchemaTableNames[1];
+                parentTableName = parentPhysicalSchemaTableNames[2];
+                    
             } else if (tableType == PTableType.INDEX) {
                 parentSchemaName = schemaName;
                 /* 
@@ -1398,7 +1495,27 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                  */ 
                 parentTableName = MetaDataUtil.getParentTableName(tableMetadata);
                 parentTableKey = SchemaUtil.getTableKey(tenantIdBytes, parentSchemaName, parentTableName);
+                long clientTimeStamp = MetaDataUtil.getClientTimeStamp(tableMetadata);
+                PTable parentTable = loadTable(env, parentTableKey, new ImmutableBytesPtr(parentTableKey),
+                        clientTimeStamp, clientTimeStamp, clientVersion);
+                if (IndexType.LOCAL == indexType) {
+                    cPhysicalName = parentTable.getPhysicalName().getBytes();
+                    cParentPhysicalName=parentTable.getPhysicalName().getBytes();
+                } else if (parentTable.getType() == PTableType.VIEW) {
+                    cPhysicalName = MetaDataUtil.getViewIndexPhysicalName(parentTable.getPhysicalName().getBytes());
+                    cParentPhysicalName = parentTable.getPhysicalName().getBytes();
+                }else{
+                    cParentPhysicalName = SchemaUtil
+                            .getPhysicalHBaseTableName(parentSchemaName, parentTableName, isNamespaceMapped).getBytes();
+                }
             }
+            
+            getCoprocessorHost().preCreateTable(Bytes.toString(tenantIdBytes),
+                    SchemaUtil.getTableName(schemaName, tableName),
+                    (tableType == PTableType.VIEW) ? null : TableName.valueOf(cPhysicalName),
+                    cParentPhysicalName == null ? null : TableName.valueOf(cParentPhysicalName), tableType,
+                    /* TODO: During inital create we may not need the family map */
+                    Collections.<byte[]> emptySet(), indexes);
 
             Region region = env.getRegion();
             List<RowLock> locks = Lists.newArrayList();
@@ -1613,7 +1730,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 // primary and then index table locks are held, in that order). For now, we just don't support
                 // indexing on the system table. This is an issue because of the way we manage batch mutation
                 // in the Indexer.
-                region.mutateRowsWithLocks(tableMetadata, Collections.<byte[]> emptySet(), HConstants.NO_NONCE, HConstants.NO_NONCE);
+                mutateRowsWithLocks(region, tableMetadata, Collections.<byte[]> emptySet(), HConstants.NO_NONCE, HConstants.NO_NONCE);
 
                 // Invalidate the cache - the next getTable call will add it
                 // TODO: consider loading the table that was just created here, patching up the parent table, and updating the cache
@@ -1632,7 +1749,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 done.run(builder.build());
                 return;
             } finally {
-                region.releaseRowLocks(locks);
+                releaseRowLocks(region,locks);
             }
         } catch (Throwable t) {
             logger.error("createTable failed", t);
@@ -1648,16 +1765,6 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 QueryServicesOptions.DEFAULT_MAX_INDEXES_PER_TABLE);
     }
 
-    private static RowLock acquireLock(Region region, byte[] key, List<RowLock> locks)
-        throws IOException {
-        RowLock rowLock = region.getRowLock(key, false);
-        if (rowLock == null) {
-            throw new IOException("Failed to acquire lock on " + Bytes.toStringBinary(key));
-        }
-        locks.add(rowLock);
-        return rowLock;
-    }
-
     private static final byte[] CHILD_TABLE_BYTES = new byte[] {PTable.LinkType.CHILD_TABLE.getSerializedValue()};
 
     
@@ -1846,6 +1953,23 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     parentTableName == null ? lockKey : SchemaUtil.getTableKey(tenantIdBytes,
                         schemaName, tableName);
 
+            
+            PTableType ptableType=PTableType.fromSerializedValue(tableType);
+            long clientTimeStamp = MetaDataUtil.getClientTimeStamp(tableMetadata);
+            byte[] cKey = SchemaUtil.getTableKey(tenantIdBytes, schemaName, tableName);
+            PTable loadedTable = loadTable(env, cKey, new ImmutableBytesPtr(cKey), clientTimeStamp, clientTimeStamp,
+                    request.getClientVersion());
+            if (loadedTable == null) {
+                builder.setReturnCode(MetaDataProtos.MutationCode.TABLE_NOT_FOUND);
+                builder.setMutationTime(EnvironmentEdgeManager.currentTimeMillis());
+                done.run(builder.build());
+                return;
+            }
+            getCoprocessorHost().preDropTable(Bytes.toString(tenantIdBytes),
+                    SchemaUtil.getTableName(schemaName, tableName),
+                    TableName.valueOf(loadedTable.getPhysicalName().getBytes()),
+                    getParentPhysicalTableName(loadedTable), ptableType,loadedTable.getIndexes());
+
             Region region = env.getRegion();
             MetaDataMutationResult result = checkTableKeyInRegion(key, region);
             if (result != null) {
@@ -1870,7 +1994,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 }
                 Cache<ImmutableBytesPtr,PMetaDataEntity> metaDataCache = GlobalCache.getInstance(this.env).getMetaDataCache();
                 // Commit the list of deletion.
-                region.mutateRowsWithLocks(tableMetadata, Collections.<byte[]> emptySet(), HConstants.NO_NONCE,
+                mutateRowsWithLocks(region, tableMetadata, Collections.<byte[]> emptySet(), HConstants.NO_NONCE,
                     HConstants.NO_NONCE);
                 long currentTime = MetaDataUtil.getClientTimeStamp(tableMetadata);
                 for (ImmutableBytesPtr ckey : invalidateList) {
@@ -1883,7 +2007,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 done.run(MetaDataMutationResult.toProto(result));
                 return;
             } finally {
-                region.releaseRowLocks(locks);
+                releaseRowLocks(region,locks);
             }
         } catch (Throwable t) {
           logger.error("dropTable failed", t);
@@ -1891,6 +2015,24 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 ServerUtil.createIOException(SchemaUtil.getTableName(schemaName, tableName), t));
         }
     }
+    
+    protected void releaseRowLocks(Region region, List<RowLock> locks) {
+        if (locks != null) {
+            region.releaseRowLocks(locks);
+        }
+    }
+
+    private RowLock acquireLock(Region region, byte[] lockKey, List<RowLock> locks) throws IOException {
+        //LockManager.RowLock rowLock = lockManager.lockRow(lockKey, rowLockWaitDuration);
+        RowLock rowLock = region.getRowLock(lockKey, false);
+        if (rowLock == null) {
+            throw new IOException("Failed to acquire lock on " + Bytes.toStringBinary(lockKey));
+        }
+        if (locks != null) {
+            locks.add(rowLock);
+        }
+        return rowLock;
+    }
 
     private MetaDataMutationResult doDropTable(byte[] key, byte[] tenantId, byte[] schemaName,
         byte[] tableName, byte[] parentTableName, PTableType tableType, List<Mutation> rowsToDelete,
@@ -2093,18 +2235,15 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                             EnvironmentEdgeManager.currentTimeMillis(), null);
                 }
                 if (table.getTimeStamp() >= clientTimeStamp) {
-                    logger.info("Found newer table as of " + table.getTimeStamp() + " versus client timestamp of " + clientTimeStamp);
+                    logger.info("Found newer table as of " + table.getTimeStamp() + " versus client timestamp of "
+                            + clientTimeStamp);
                     return new MetaDataMutationResult(MutationCode.NEWER_TABLE_FOUND,
                             EnvironmentEdgeManager.currentTimeMillis(), table);
-                } else if (isTableDeleted(table)) {
-                    return new MetaDataMutationResult(MutationCode.TABLE_NOT_FOUND,
-                            EnvironmentEdgeManager.currentTimeMillis(), null);
-                }
-
-                long expectedSeqNum = MetaDataUtil.getSequenceNumber(tableMetadata) - 1; // lookup
-                                                                                         // TABLE_SEQ_NUM
-                                                                                         // in
+                } else if (isTableDeleted(table)) { return new MetaDataMutationResult(MutationCode.TABLE_NOT_FOUND,
+                        EnvironmentEdgeManager.currentTimeMillis(), null); }
+                long expectedSeqNum = MetaDataUtil.getSequenceNumber(tableMetadata) - 1; // lookup TABLE_SEQ_NUM in
                                                                                          // tableMetaData
+
                 if (logger.isDebugEnabled()) {
                     logger.debug("For table " + Bytes.toStringBinary(key) + " expecting seqNum "
                             + expectedSeqNum + " and found seqNum " + table.getSequenceNumber()
@@ -2139,7 +2278,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 if (result != null && result.getMutationCode()!=MutationCode.TABLE_ALREADY_EXISTS) {
                     return result;
                 }
-                region.mutateRowsWithLocks(tableMetadata, Collections.<byte[]> emptySet(), HConstants.NO_NONCE, HConstants.NO_NONCE);
+                mutateRowsWithLocks(region, tableMetadata, Collections.<byte[]> emptySet(), HConstants.NO_NONCE, HConstants.NO_NONCE);
                 // Invalidate from cache
                 for (ImmutableBytesPtr invalidateKey : invalidateList) {
                     metaDataCache.invalidate(invalidateKey);
@@ -2155,7 +2294,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     return new MetaDataMutationResult(MutationCode.TABLE_ALREADY_EXISTS, currentTime, table);
                 }
             } finally {
-                region.releaseRowLocks(locks);
+                releaseRowLocks(region,locks);
             }
         } catch (Throwable t) {
             ServerUtil.throwIOException(SchemaUtil.getTableName(schemaName, tableName), t);
@@ -2971,6 +3110,11 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     PTableType type = table.getType();
                     byte[] tableHeaderRowKey = SchemaUtil.getTableKey(tenantId,
                             schemaName, tableName);
+                    byte[] cPhysicalTableName=table.getPhysicalName().getBytes();
+                    getCoprocessorHost().preAlterTable(Bytes.toString(tenantId),
+                            SchemaUtil.getTableName(schemaName, tableName), TableName.valueOf(cPhysicalTableName),
+                            getParentPhysicalTableName(table),type);
+
                     // Size for worst case - all new columns are PK column
                     List<Mutation> mutationsForAddingColumnsToViews = Lists.newArrayListWithExpectedSize(tableMetaData.size() * ( 1 + table.getIndexes().size()));
                     if (type == PTableType.TABLE || type == PTableType.SYSTEM) {
@@ -3124,10 +3268,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         boolean blockWriteRebuildIndex = env.getConfiguration().getBoolean(QueryServices.INDEX_FAILURE_BLOCK_WRITE, 
                 QueryServicesOptions.DEFAULT_INDEX_FAILURE_BLOCK_WRITE);
         if (!wasLocked) {
-            rowLock = region.getRowLock(key, false);
-            if (rowLock == null) {
-                throw new IOException("Failed to acquire lock on " + Bytes.toStringBinary(key));
-            }
+            rowLock = acquireLock(region, key, null);
         }
         try {
             PTable table = (PTable)metaDataCache.getIfPresent(cacheKey);
@@ -3184,16 +3325,10 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
          * Lock directly on key, though it may be an index table. This will just prevent a table
          * from getting rebuilt too often.
          */
-        List<RowLock> rowLocks = new ArrayList<Region.RowLock>(keys.size());;
+        List<RowLock> rowLocks = new ArrayList<RowLock>(keys.size());;
         try {
-            rowLocks = new ArrayList<Region.RowLock>(keys.size());
             for (int i = 0; i < keys.size(); i++) {
-                Region.RowLock rowLock = region.getRowLock(keys.get(i), false);
-                if (rowLock == null) {
-                    throw new IOException("Failed to acquire lock on "
-                            + Bytes.toStringBinary(keys.get(i)));
-                }
-                rowLocks.add(rowLock);
+                acquireLock(region, keys.get(i), rowLocks);
             }
 
             List<PFunction> functionsAvailable = new ArrayList<PFunction>(keys.size());
@@ -3223,10 +3358,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             if(functionsAvailable.size() == numFunctions) return functionsAvailable;
             return null;
         } finally {
-            for (Region.RowLock lock : rowLocks) {
-                lock.release();
-            }
-            rowLocks.clear();
+            releaseRowLocks(region,rowLocks);
         }
     }
 
@@ -3248,6 +3380,11 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     byte[] schemaName = rowKeyMetaData[SCHEMA_NAME_INDEX];
                     byte[] tableName = rowKeyMetaData[TABLE_NAME_INDEX];
                     boolean deletePKColumn = false;
+                    getCoprocessorHost().preAlterTable(Bytes.toString(tenantId),
+                            SchemaUtil.getTableName(schemaName, tableName),
+                            TableName.valueOf(table.getPhysicalName().getBytes()),
+                            getParentPhysicalTableName(table),table.getType());
+
                     List<Mutation> additionalTableMetaData = Lists.newArrayList();
                     
                     PTableType type = table.getType();
@@ -3480,7 +3617,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             }
             PIndexState newState =
                     PIndexState.fromSerializedValue(newKV.getValueArray()[newKV.getValueOffset()]);
-            RowLock rowLock = region.getRowLock(key, false);
+            RowLock rowLock = acquireLock(region, key, null);
             if (rowLock == null) {
                 throw new IOException("Failed to acquire lock on " + Bytes.toStringBinary(key));
             }
@@ -3502,6 +3639,22 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 Cell currentDisableTimeStamp = currentResult.getColumnLatestCell(TABLE_FAMILY_BYTES, INDEX_DISABLE_TIMESTAMP_BYTES);
                 boolean rowKeyOrderOptimizable = currentResult.getColumnLatestCell(TABLE_FAMILY_BYTES, ROW_KEY_ORDER_OPTIMIZABLE_BYTES) != null;
 
+                //check permission on data table
+                long clientTimeStamp = MetaDataUtil.getClientTimeStamp(tableMetadata);
+                PTable loadedTable = loadTable(env, key, new ImmutableBytesPtr(key), clientTimeStamp, clientTimeStamp,
+                        request.getClientVersion());
+                if (loadedTable == null) {
+                    builder.setReturnCode(MetaDataProtos.MutationCode.TABLE_NOT_FOUND);
+                    builder.setMutationTime(EnvironmentEdgeManager.currentTimeMillis());
+                    done.run(builder.build());
+                    return;
+                }
+                getCoprocessorHost().preIndexUpdate(Bytes.toString(tenantId),
+                        SchemaUtil.getTableName(schemaName, tableName),
+                        TableName.valueOf(loadedTable.getPhysicalName().getBytes()),
+                        getParentPhysicalTableName(loadedTable),
+                        newState);
+
                 PIndexState currentState =
                         PIndexState.fromSerializedValue(currentStateKV.getValueArray()[currentStateKV
                                 .getValueOffset()]);
@@ -3611,7 +3764,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     if (setRowKeyOrderOptimizableCell) {
                         UpgradeUtil.addRowKeyOrderOptimizableCell(tableMetadata, key, timeStamp);
                     }
-                    region.mutateRowsWithLocks(tableMetadata, Collections.<byte[]> emptySet(), HConstants.NO_NONCE,
+                    mutateRowsWithLocks(region, tableMetadata, Collections.<byte[]> emptySet(), HConstants.NO_NONCE,
                         HConstants.NO_NONCE);
                     // Invalidate from cache
                     Cache<ImmutableBytesPtr,PMetaDataEntity> metaDataCache = GlobalCache.getInstance(this.env).getMetaDataCache();
@@ -3772,6 +3925,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         long clientTimeStamp = request.getClientTimestamp();
         List<RowLock> locks = Lists.newArrayList();
         try {
+            getCoprocessorHost().preGetSchema(schemaName);
             acquireLock(region, lockKey, locks);
             // Get as of latest timestamp so we can detect if we have a
             // newer schema that already
@@ -3802,7 +3956,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             done.run(builder.build());
             return;
         } finally {
-            region.releaseRowLocks(locks);
+            releaseRowLocks(region,locks);
         }
     }
 
@@ -3905,7 +4059,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 }
                 // Don't store function info for temporary functions.
                 if(!temporaryFunction) {
-                    region.mutateRowsWithLocks(functionMetaData, Collections.<byte[]> emptySet(), HConstants.NO_NONCE, HConstants.NO_NONCE);
+                    mutateRowsWithLocks(region, functionMetaData, Collections.<byte[]> emptySet(), HConstants.NO_NONCE, HConstants.NO_NONCE);
                 }
 
                 // Invalidate the cache - the next getFunction call will add it
@@ -3919,7 +4073,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 done.run(builder.build());
                 return;
             } finally {
-                region.releaseRowLocks(locks);
+                releaseRowLocks(region,locks);
             }
         } catch (Throwable t) {
           logger.error("createFunction failed", t);
@@ -3958,7 +4112,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     done.run(MetaDataMutationResult.toProto(result));
                     return;
                 }
-                region.mutateRowsWithLocks(functionMetaData, Collections.<byte[]> emptySet(), HConstants.NO_NONCE, HConstants.NO_NONCE);
+                mutateRowsWithLocks(region, functionMetaData, Collections.<byte[]> emptySet(), HConstants.NO_NONCE, HConstants.NO_NONCE);
 
                 Cache<ImmutableBytesPtr,PMetaDataEntity> metaDataCache = GlobalCache.getInstance(this.env).getMetaDataCache();
                 long currentTime = MetaDataUtil.getClientTimeStamp(functionMetaData);
@@ -3971,7 +4125,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 done.run(MetaDataMutationResult.toProto(result));
                 return;
             } finally {
-                region.releaseRowLocks(locks);
+                releaseRowLocks(region,locks);
             }
         } catch (Throwable t) {
           logger.error("dropFunction failed", t);
@@ -4068,7 +4222,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                         return;
                     }
                 }
-                region.mutateRowsWithLocks(schemaMutations, Collections.<byte[]> emptySet(), HConstants.NO_NONCE,
+                mutateRowsWithLocks(region, schemaMutations, Collections.<byte[]> emptySet(), HConstants.NO_NONCE,
                         HConstants.NO_NONCE);
 
                 // Invalidate the cache - the next getSchema call will add it
@@ -4086,7 +4240,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 done.run(builder.build());
                 return;
             } finally {
-                region.releaseRowLocks(locks);
+                releaseRowLocks(region,locks);
             }
         } catch (Throwable t) {
             logger.error("Creating the schema" + schemaName + "failed", t);
@@ -4100,6 +4254,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         try {
             List<Mutation> schemaMetaData = ProtobufUtil.getMutations(request);
             schemaName = request.getSchemaName();
+            getCoprocessorHost().preDropSchema(schemaName);
             byte[] lockKey = SchemaUtil.getSchemaKey(schemaName);
             Region region = env.getRegion();
             MetaDataMutationResult result = checkSchemaKeyInRegion(lockKey, region);
@@ -4117,7 +4272,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     done.run(MetaDataMutationResult.toProto(result));
                     return;
                 }
-                region.mutateRowsWithLocks(schemaMetaData, Collections.<byte[]> emptySet(), HConstants.NO_NONCE,
+                mutateRowsWithLocks(region, schemaMetaData, Collections.<byte[]> emptySet(), HConstants.NO_NONCE,
                         HConstants.NO_NONCE);
                 Cache<ImmutableBytesPtr, PMetaDataEntity> metaDataCache = GlobalCache.getInstance(this.env)
                         .getMetaDataCache();
@@ -4129,7 +4284,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 done.run(MetaDataMutationResult.toProto(result));
                 return;
             } finally {
-                region.releaseRowLocks(locks);
+                releaseRowLocks(region,locks);
             }
         } catch (Throwable t) {
             logger.error("drop schema failed:", t);
@@ -4175,4 +4330,48 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 null);
 
     }
+    
+    private void mutateRowsWithLocks(final Region region, final List<Mutation> mutations, final Set<byte[]> rowsToLock,
+            final long nonceGroup, final long nonce) throws IOException {
+        // we need to mutate SYSTEM.CATALOG with HBase/login user if access is enabled.
+        if (this.accessCheckEnabled) {
+            User.runAsLoginUser(new PrivilegedExceptionAction<Void>() {
+                @Override
+                public Void run() throws Exception {
+                    final Call rpcContext = RpcUtil.getRpcContext();
+                    // Setting RPC context as null so that user can be resetted
+                    try {
+                        RpcUtil.setRpcContext(null);
+                        region.mutateRowsWithLocks(mutations, rowsToLock, nonceGroup, nonce);
+                    } catch (Throwable e) {
+                        throw new IOException(e);
+                    } finally {
+                        // Setting RPC context back to original context of the RPC
+                        RpcUtil.setRpcContext(rpcContext);
+                    }
+                    return null;
+                }
+            });
+        } else {
+            region.mutateRowsWithLocks(mutations, rowsToLock, nonceGroup, nonce);
+        }
+    }
+    
+    private TableName getParentPhysicalTableName(PTable table) {
+        return table
+                .getType() == PTableType.VIEW
+                        ? TableName.valueOf(table.getPhysicalName().getBytes())
+                        : table.getType() == PTableType.INDEX
+                                ? TableName
+                                        .valueOf(SchemaUtil
+                                                .getPhysicalHBaseTableName(table.getParentSchemaName(),
+                                                        table.getParentTableName(), table.isNamespaceMapped())
+                                                .getBytes())
+                                : TableName
+                                        .valueOf(
+                                                SchemaUtil
+                                                        .getPhysicalHBaseTableName(table.getSchemaName(),
+                                                                table.getTableName(), table.isNamespaceMapped())
+                                                        .getBytes());
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/493a54f9/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointObserver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointObserver.java
new file mode 100644
index 0000000..86b8bf1
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointObserver.java
@@ -0,0 +1,68 @@
+/*
+ * 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.coprocessor;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.hadoop.hbase.Coprocessor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.phoenix.coprocessor.PhoenixMetaDataCoprocessorHost.PhoenixMetaDataControllerEnvironment;
+import org.apache.phoenix.schema.PIndexState;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTableType;
+
+public interface MetaDataEndpointObserver extends Coprocessor {
+
+    void preGetTable( ObserverContext<PhoenixMetaDataControllerEnvironment> ctx,  String tenantId,String tableName,
+             TableName physicalTableName) throws IOException;
+
+    void preCreateTable(final ObserverContext<PhoenixMetaDataControllerEnvironment> ctx, final String tenantId,
+            String tableName, TableName physicalTableName, final TableName parentPhysicalTableName,
+            PTableType tableType, final Set<byte[]> familySet, Set<TableName> indexes) throws IOException;
+
+    void preDropTable(final ObserverContext<PhoenixMetaDataControllerEnvironment> ctx, final String tenantId,
+            final String tableName,TableName physicalTableName, TableName parentPhysicalTableName, PTableType tableType, List<PTable> indexes) throws IOException;
+
+    void preAlterTable(final ObserverContext<PhoenixMetaDataControllerEnvironment> ctx, final String tenantId,final String tableName,
+            final TableName physicalTableName,final TableName parentPhysicalTableName, PTableType type) throws IOException;
+
+    void preGetSchema(final ObserverContext<PhoenixMetaDataControllerEnvironment> ctx, final String schemaName)
+            throws IOException;
+
+    void preCreateSchema(final ObserverContext<PhoenixMetaDataControllerEnvironment> ctx, final String schemaName)
+            throws IOException;
+
+    void preDropSchema(final ObserverContext<PhoenixMetaDataControllerEnvironment> ctx, final String schemaName)
+            throws IOException;
+
+    void preCreateFunction(final ObserverContext<PhoenixMetaDataControllerEnvironment> ctx, final String tenantId,
+            final String functionName) throws IOException;
+
+    void preDropFunction(final ObserverContext<PhoenixMetaDataControllerEnvironment> ctx, final String tenantId,
+            final String functionName) throws IOException;
+
+    void preGetFunctions(final ObserverContext<PhoenixMetaDataControllerEnvironment> ctx, final String tenantId,
+            final String functionName) throws IOException;
+
+    void preIndexUpdate(ObserverContext<PhoenixMetaDataControllerEnvironment> ctx, String tenantId,
+            String indexName, TableName physicalTableName, TableName parentPhysicalTableName, PIndexState newState) throws IOException;
+
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/493a54f9/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java
index c816549..af06235 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
@@ -20,6 +20,7 @@ package org.apache.phoenix.coprocessor;
 import static org.apache.phoenix.schema.types.PDataType.TRUE_BYTES;
 
 import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
 import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -50,6 +51,7 @@ import org.apache.hadoop.hbase.filter.CompareFilter;
 import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
+import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
@@ -163,9 +165,18 @@ public class MetaDataRegionObserver extends BaseRegionObserver {
                             SchemaUtil.getPhysicalName(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES, props));
                     statsTable = env.getTable(
                             SchemaUtil.getPhysicalName(PhoenixDatabaseMetaData.SYSTEM_STATS_NAME_BYTES, props));
-                    if (UpgradeUtil.truncateStats(metaTable, statsTable)) {
-                        LOG.info("Stats are successfully truncated for upgrade 4.7!!");
-                    }
+                    final HTableInterface mTable=metaTable;
+                    final HTableInterface sTable=statsTable;
+                    User.runAsLoginUser(new PrivilegedExceptionAction<Void>() {
+                        @Override
+                        public Void run() throws Exception {
+                            if (UpgradeUtil.truncateStats(mTable, sTable)) {
+                                LOG.info("Stats are successfully truncated for upgrade 4.7!!");
+                            }
+                            return null;
+                        }
+                    });
+
                 } catch (Exception exception) {
                     LOG.warn("Exception while truncate stats..,"
                             + " please check and delete stats manually inorder to get proper result with old client!!");


[16/16] phoenix git commit: PHOENIX-4386 Calculate the estimatedSize of MutationState using Map> mutations (addendum)

Posted by ja...@apache.org.
PHOENIX-4386 Calculate the estimatedSize of MutationState using Map<TableRef, Map<ImmutableBytesPtr,RowMutationState>> mutations (addendum)


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

Branch: refs/heads/4.x-HBase-1.2
Commit: 5b493962a437a264bb45f9698223a59f43623373
Parents: 4968e88
Author: Thomas D'Silva <td...@apache.org>
Authored: Tue Nov 21 03:13:53 2017 +0000
Committer: James Taylor <jt...@salesforce.com>
Committed: Sat Dec 16 16:42:54 2017 -0800

----------------------------------------------------------------------
 .../apache/phoenix/execute/PartialCommitIT.java |   5 +-
 .../apache/phoenix/compile/DeleteCompiler.java  |  11 +-
 .../apache/phoenix/compile/UpsertCompiler.java  |   7 +-
 .../apache/phoenix/execute/MutationState.java   | 127 ++++++++++++-------
 .../java/org/apache/phoenix/util/IndexUtil.java |   4 +-
 .../org/apache/phoenix/util/KeyValueUtil.java   |   5 +-
 6 files changed, 98 insertions(+), 61 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/5b493962/phoenix-core/src/it/java/org/apache/phoenix/execute/PartialCommitIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/execute/PartialCommitIT.java b/phoenix-core/src/it/java/org/apache/phoenix/execute/PartialCommitIT.java
index 10fd7f8..e5b57e3 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/execute/PartialCommitIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/execute/PartialCommitIT.java
@@ -33,7 +33,6 @@ import java.sql.SQLException;
 import java.sql.Statement;
 import java.util.Arrays;
 import java.util.Collection;
-import java.util.Collections;
 import java.util.Comparator;
 import java.util.List;
 import java.util.Map;
@@ -52,8 +51,8 @@ import org.apache.hadoop.hbase.coprocessor.SimpleRegionObserver;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.end2end.BaseOwnClusterIT;
+import org.apache.phoenix.execute.MutationState.MultiRowMutationState;
 import org.apache.phoenix.hbase.index.Indexer;
-import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.monitoring.GlobalMetric;
 import org.apache.phoenix.monitoring.MetricType;
@@ -285,7 +284,7 @@ public class PartialCommitIT extends BaseOwnClusterIT {
     private PhoenixConnection getConnectionWithTableOrderPreservingMutationState() throws SQLException {
         Connection con = driver.connect(url, new Properties());
         PhoenixConnection phxCon = new PhoenixConnection(con.unwrap(PhoenixConnection.class));
-        final Map<TableRef,Map<ImmutableBytesPtr,MutationState.RowMutationState>> mutations = Maps.newTreeMap(new TableRefComparator());
+        final Map<TableRef, MultiRowMutationState> mutations = Maps.newTreeMap(new TableRefComparator());
         // passing a null mutation state forces the connection.newMutationState() to be used to create the MutationState
         return new PhoenixConnection(phxCon, null) {
             @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5b493962/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
----------------------------------------------------------------------
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 f9ca300..a06e2ca 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
@@ -26,7 +26,6 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
-import java.util.Map;
 import java.util.Set;
 
 import org.apache.hadoop.hbase.Cell;
@@ -43,6 +42,7 @@ import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.exception.SQLExceptionInfo;
 import org.apache.phoenix.execute.AggregatePlan;
 import org.apache.phoenix.execute.MutationState;
+import org.apache.phoenix.execute.MutationState.MultiRowMutationState;
 import org.apache.phoenix.execute.MutationState.RowMutationState;
 import org.apache.phoenix.filter.SkipScanFilter;
 import org.apache.phoenix.hbase.index.ValueGetter;
@@ -91,7 +91,6 @@ import org.apache.phoenix.util.ScanUtil;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
 import com.sun.istack.NotNull;
 
 public class DeleteCompiler {
@@ -121,14 +120,14 @@ public class DeleteCompiler {
         final int maxSize = services.getProps().getInt(QueryServices.MAX_MUTATION_SIZE_ATTRIB,QueryServicesOptions.DEFAULT_MAX_MUTATION_SIZE);
         final int maxSizeBytes = services.getProps().getInt(QueryServices.MAX_MUTATION_SIZE_BYTES_ATTRIB,QueryServicesOptions.DEFAULT_MAX_MUTATION_SIZE_BYTES);
         final int batchSize = Math.min(connection.getMutateBatchSize(), maxSize);
-        Map<ImmutableBytesPtr,RowMutationState> mutations = Maps.newHashMapWithExpectedSize(batchSize);
-        List<Map<ImmutableBytesPtr,RowMutationState>> indexMutations = null;
+        MultiRowMutationState mutations = new MultiRowMutationState(batchSize);
+        List<MultiRowMutationState> indexMutations = null;
         // If indexTableRef is set, we're deleting the rows from both the index table and
         // the data table through a single query to save executing an additional one.
         if (!otherTableRefs.isEmpty()) {
             indexMutations = Lists.newArrayListWithExpectedSize(otherTableRefs.size());
             for (int i = 0; i < otherTableRefs.size(); i++) {
-                indexMutations.add(Maps.<ImmutableBytesPtr,RowMutationState>newHashMapWithExpectedSize(batchSize));
+                indexMutations.add(new MultiRowMutationState(batchSize));
             }
         }
         List<PColumn> pkColumns = table.getPKColumns();
@@ -644,7 +643,7 @@ public class DeleteCompiler {
             // keys for our ranges
             ScanRanges ranges = context.getScanRanges();
             Iterator<KeyRange> iterator = ranges.getPointLookupKeyIterator();
-            Map<ImmutableBytesPtr,RowMutationState> mutation = Maps.newHashMapWithExpectedSize(ranges.getPointLookupCount());
+            MultiRowMutationState mutation = new MultiRowMutationState(ranges.getPointLookupCount());
             while (iterator.hasNext()) {
                 mutation.put(new ImmutableBytesPtr(iterator.next().getLowerRange()),
                         new RowMutationState(PRow.DELETE_MARKER, 0,

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5b493962/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
index a51fd4c..a81a427 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
@@ -47,6 +47,7 @@ import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.exception.SQLExceptionInfo;
 import org.apache.phoenix.execute.AggregatePlan;
 import org.apache.phoenix.execute.MutationState;
+import org.apache.phoenix.execute.MutationState.MultiRowMutationState;
 import org.apache.phoenix.execute.MutationState.RowMutationState;
 import org.apache.phoenix.execute.MutationState.RowTimestampColInfo;
 import org.apache.phoenix.expression.Determinism;
@@ -116,7 +117,7 @@ import com.google.common.collect.Sets;
 public class UpsertCompiler {
 
     private static void setValues(byte[][] values, int[] pkSlotIndex, int[] columnIndexes,
-            PTable table, Map<ImmutableBytesPtr, RowMutationState> mutation,
+            PTable table, MultiRowMutationState mutation,
             PhoenixStatement statement, boolean useServerTimestamp, IndexMaintainer maintainer,
             byte[][] viewConstants, byte[] onDupKeyBytes, int numSplColumns) throws SQLException {
         long columnValueSize = 0;
@@ -197,7 +198,7 @@ public class UpsertCompiler {
             }
         }
         int rowCount = 0;
-        Map<ImmutableBytesPtr, RowMutationState> mutation = Maps.newHashMapWithExpectedSize(batchSize);
+        MultiRowMutationState mutation = new MultiRowMutationState(batchSize);
         PTable table = tableRef.getTable();
         IndexMaintainer indexMaintainer = null;
         byte[][] viewConstants = null;
@@ -1177,7 +1178,7 @@ public class UpsertCompiler {
                     throw new IllegalStateException();
                 }
             }
-            Map<ImmutableBytesPtr, RowMutationState> mutation = Maps.newHashMapWithExpectedSize(1);
+            MultiRowMutationState mutation = new MultiRowMutationState(1);
             IndexMaintainer indexMaintainer = null;
             byte[][] viewConstants = null;
             if (table.getIndexType() == IndexType.LOCAL) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5b493962/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
index 15e905a..993438e 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
@@ -123,7 +123,7 @@ public class MutationState implements SQLCloseable {
     private final long batchSize;
     private final long batchSizeBytes;
     private long batchCount = 0L;
-    private final Map<TableRef, Map<ImmutableBytesPtr,RowMutationState>> mutations;
+    private final Map<TableRef, MultiRowMutationState> mutations;
     private final Set<String> uncommittedPhysicalNames = Sets.newHashSetWithExpectedSize(10);
 
     private long sizeOffset;
@@ -131,7 +131,7 @@ public class MutationState implements SQLCloseable {
     private long estimatedSize = 0;
     private int[] uncommittedStatementIndexes = EMPTY_STATEMENT_INDEX_ARRAY;
     private boolean isExternalTxContext = false;
-    private Map<TableRef, Map<ImmutableBytesPtr,RowMutationState>> txMutations = Collections.emptyMap();
+    private Map<TableRef, MultiRowMutationState> txMutations = Collections.emptyMap();
 
     final PhoenixTransactionContext phoenixTransactionContext;
 
@@ -159,12 +159,12 @@ public class MutationState implements SQLCloseable {
     }
 
     private MutationState(long maxSize, long maxSizeBytes, PhoenixConnection connection, boolean subTask, PhoenixTransactionContext txContext, long sizeOffset) {
-        this(maxSize, maxSizeBytes, connection, Maps.<TableRef, Map<ImmutableBytesPtr,RowMutationState>>newHashMapWithExpectedSize(5), subTask, txContext);
+        this(maxSize, maxSizeBytes, connection, Maps.<TableRef, MultiRowMutationState>newHashMapWithExpectedSize(5), subTask, txContext);
         this.sizeOffset = sizeOffset;
     }
 
     MutationState(long maxSize, long maxSizeBytes, PhoenixConnection connection,
-            Map<TableRef, Map<ImmutableBytesPtr, RowMutationState>> mutations,
+            Map<TableRef, MultiRowMutationState> mutations,
             boolean subTask, PhoenixTransactionContext txContext) {
         this.maxSize = maxSize;
         this.maxSizeBytes = maxSizeBytes;
@@ -189,7 +189,7 @@ public class MutationState implements SQLCloseable {
         }
     }
 
-    public MutationState(TableRef table, Map<ImmutableBytesPtr,RowMutationState> mutations, long sizeOffset, long maxSize, long maxSizeBytes, PhoenixConnection connection) throws SQLException {
+    public MutationState(TableRef table, MultiRowMutationState mutations, long sizeOffset, long maxSize, long maxSizeBytes, PhoenixConnection connection)  throws SQLException {
         this(maxSize, maxSizeBytes, connection, false, null, sizeOffset);
         if (!mutations.isEmpty()) {
             this.mutations.put(table, mutations);
@@ -350,7 +350,7 @@ public class MutationState implements SQLCloseable {
     }
 
     public static MutationState emptyMutationState(long maxSize, long maxSizeBytes, PhoenixConnection connection) {
-        MutationState state = new MutationState(maxSize, maxSizeBytes, connection, Collections.<TableRef, Map<ImmutableBytesPtr,RowMutationState>>emptyMap(), false, null);
+        MutationState state = new MutationState(maxSize, maxSizeBytes, connection, Collections.<TableRef, MultiRowMutationState>emptyMap(), false, null);
         state.sizeOffset = 0;
         return state;
     }
@@ -372,12 +372,12 @@ public class MutationState implements SQLCloseable {
         return sizeOffset + numRows;
     }
     
-    private void joinMutationState(TableRef tableRef, Map<ImmutableBytesPtr,RowMutationState> srcRows,
-            Map<TableRef, Map<ImmutableBytesPtr, RowMutationState>> dstMutations) {
+    private void joinMutationState(TableRef tableRef, MultiRowMutationState srcRows,
+            Map<TableRef, MultiRowMutationState> dstMutations) {
         PTable table = tableRef.getTable();
         boolean isIndex = table.getType() == PTableType.INDEX;
         boolean incrementRowCount = dstMutations == this.mutations;
-        Map<ImmutableBytesPtr,RowMutationState> existingRows = dstMutations.put(tableRef, srcRows);
+        MultiRowMutationState existingRows = dstMutations.put(tableRef, srcRows);
         if (existingRows != null) { // Rows for that table already exist
             // Loop through new rows and replace existing with new
             for (Map.Entry<ImmutableBytesPtr,RowMutationState> rowEntry : srcRows.entrySet()) {
@@ -389,8 +389,12 @@ public class MutationState implements SQLCloseable {
                         Map<PColumn,byte[]> newRow = rowEntry.getValue().getColumnValues();
                         // if new row is PRow.DELETE_MARKER, it means delete, and we don't need to merge it with existing row. 
                         if (newRow != PRow.DELETE_MARKER) {
+                            // decrement estimated size by the size of the old row
+                            estimatedSize-=existingRowMutationState.calculateEstimatedSize();
                             // Merge existing column values with new column values
                             existingRowMutationState.join(rowEntry.getValue());
+                            // increment estimated size by the size of the new row
+                            estimatedSize+=existingRowMutationState.calculateEstimatedSize();
                             // Now that the existing row has been merged with the new row, replace it back
                             // again (since it was merged with the new one above).
                             existingRows.put(rowEntry.getKey(), existingRowMutationState);
@@ -399,6 +403,8 @@ public class MutationState implements SQLCloseable {
                 } else {
                     if (incrementRowCount && !isIndex) { // Don't count index rows in row count
                         numRows++;
+                        // increment estimated size by the size of the new row
+                        estimatedSize += rowEntry.getValue().calculateEstimatedSize();
                     }
                 }
             }
@@ -406,22 +412,25 @@ public class MutationState implements SQLCloseable {
             dstMutations.put(tableRef, existingRows);
         } else {
             // Size new map at batch size as that's what it'll likely grow to.
-            Map<ImmutableBytesPtr,RowMutationState> newRows = Maps.newHashMapWithExpectedSize(connection.getMutateBatchSize());
+            MultiRowMutationState newRows = new MultiRowMutationState(connection.getMutateBatchSize());
             newRows.putAll(srcRows);
             dstMutations.put(tableRef, newRows);
             if (incrementRowCount && !isIndex) {
                 numRows += srcRows.size();
+                // if we added all the rows from newMutationState we can just increment the
+                // estimatedSize by newMutationState.estimatedSize
+                estimatedSize +=  srcRows.estimatedSize;
             }
         }
     }
     
-    private void joinMutationState(Map<TableRef, Map<ImmutableBytesPtr, RowMutationState>> srcMutations, 
-            Map<TableRef, Map<ImmutableBytesPtr, RowMutationState>> dstMutations) {
+    private void joinMutationState(Map<TableRef, MultiRowMutationState> srcMutations, 
+            Map<TableRef, MultiRowMutationState> dstMutations) {
         // Merge newMutation with this one, keeping state from newMutation for any overlaps
-        for (Map.Entry<TableRef, Map<ImmutableBytesPtr,RowMutationState>> entry : srcMutations.entrySet()) {
+        for (Map.Entry<TableRef, MultiRowMutationState> entry : srcMutations.entrySet()) {
             // Replace existing entries for the table with new entries
             TableRef tableRef = entry.getKey();
-            Map<ImmutableBytesPtr,RowMutationState> srcRows = entry.getValue();
+            MultiRowMutationState srcRows = entry.getValue();
             joinMutationState(tableRef, srcRows, dstMutations);
         }
     }
@@ -439,19 +448,7 @@ public class MutationState implements SQLCloseable {
         phoenixTransactionContext.join(newMutationState.getPhoenixTransactionContext());
 
         this.sizeOffset += newMutationState.sizeOffset;
-        int oldNumRows = this.numRows;
         joinMutationState(newMutationState.mutations, this.mutations);
-        if (newMutationState.numRows>0) {
-            // if we added all the rows from newMutationState we can just increment the
-            // estimatedSize by newMutationState.estimatedSize
-            if (newMutationState.numRows == this.numRows-oldNumRows) {
-                this.estimatedSize +=  newMutationState.estimatedSize;
-            }
-            // we merged the two mutation states so we need to recalculate the size
-            else {
-                this.estimatedSize = KeyValueUtil.getEstimatedRowMutationSize(this.mutations);
-            }
-        }
         if (!newMutationState.txMutations.isEmpty()) {
             if (txMutations.isEmpty()) {
                 txMutations = Maps.newHashMapWithExpectedSize(mutations.size());
@@ -489,7 +486,7 @@ public class MutationState implements SQLCloseable {
         return ptr;
     }
     
-    private Iterator<Pair<PName,List<Mutation>>> addRowMutations(final TableRef tableRef, final Map<ImmutableBytesPtr, RowMutationState> values,
+    private Iterator<Pair<PName,List<Mutation>>> addRowMutations(final TableRef tableRef, final MultiRowMutationState values,
             final long mutationTimestamp, final long serverTimestamp, boolean includeAllIndexes, final boolean sendAll) { 
         final PTable table = tableRef.getTable();
         final Iterator<PTable> indexes = // Only maintain tables with immutable rows through this client-side mechanism
@@ -524,10 +521,10 @@ public class MutationState implements SQLCloseable {
                     // we may also have to include delete mutations for immutable tables if we are not processing all the tables in the mutations map
                     if (!sendAll) {
                         TableRef key = new TableRef(index);
-                        Map<ImmutableBytesPtr, RowMutationState> rowToColumnMap = mutations.remove(key);
-                        if (rowToColumnMap!=null) {
+                        MultiRowMutationState multiRowMutationState = mutations.remove(key);
+                        if (multiRowMutationState!=null) {
                             final List<Mutation> deleteMutations = Lists.newArrayList();
-                            generateMutations(tableRef, mutationTimestamp, serverTimestamp, rowToColumnMap, deleteMutations, null);
+                            generateMutations(tableRef, mutationTimestamp, serverTimestamp, multiRowMutationState, deleteMutations, null);
                             indexMutations.addAll(deleteMutations);
                         }
                     }
@@ -546,14 +543,14 @@ public class MutationState implements SQLCloseable {
     }
 
     private void generateMutations(final TableRef tableRef, final long mutationTimestamp,
-            final long serverTimestamp, final Map<ImmutableBytesPtr, RowMutationState> values,
+            final long serverTimestamp, final MultiRowMutationState values,
             final List<Mutation> mutationList, final List<Mutation> mutationsPertainingToIndex) {
         final PTable table = tableRef.getTable();
         boolean tableWithRowTimestampCol = table.getRowTimestampColPos() != -1;
         Iterator<Map.Entry<ImmutableBytesPtr, RowMutationState>> iterator =
                 values.entrySet().iterator();
         long timestampToUse = mutationTimestamp;
-        Map<ImmutableBytesPtr, RowMutationState> modifiedValues = Maps.newHashMap();
+        MultiRowMutationState modifiedValues = new MultiRowMutationState(16);
         while (iterator.hasNext()) {
             Map.Entry<ImmutableBytesPtr, RowMutationState> rowEntry = iterator.next();
             byte[] onDupKeyBytes = rowEntry.getValue().getOnDupKeyBytes();
@@ -628,7 +625,7 @@ public class MutationState implements SQLCloseable {
     }
     
     public Iterator<Pair<byte[],List<Mutation>>> toMutations(final boolean includeMutableIndexes, final Long tableTimestamp) {
-        final Iterator<Map.Entry<TableRef, Map<ImmutableBytesPtr,RowMutationState>>> iterator = this.mutations.entrySet().iterator();
+        final Iterator<Map.Entry<TableRef, MultiRowMutationState>> iterator = this.mutations.entrySet().iterator();
         if (!iterator.hasNext()) {
             return Collections.emptyIterator();
         }
@@ -636,7 +633,7 @@ public class MutationState implements SQLCloseable {
         final long serverTimestamp = getTableTimestamp(tableTimestamp, scn);
         final long mutationTimestamp = getMutationTimestamp(scn);
         return new Iterator<Pair<byte[],List<Mutation>>>() {
-            private Map.Entry<TableRef, Map<ImmutableBytesPtr,RowMutationState>> current = iterator.next();
+            private Map.Entry<TableRef, MultiRowMutationState> current = iterator.next();
             private Iterator<Pair<byte[],List<Mutation>>> innerIterator = init();
                     
             private Iterator<Pair<byte[],List<Mutation>>> init() {
@@ -700,14 +697,14 @@ public class MutationState implements SQLCloseable {
     private long[] validateAll() throws SQLException {
         int i = 0;
         long[] timeStamps = new long[this.mutations.size()];
-        for (Map.Entry<TableRef, Map<ImmutableBytesPtr,RowMutationState>> entry : mutations.entrySet()) {
+        for (Map.Entry<TableRef, MultiRowMutationState> entry : mutations.entrySet()) {
             TableRef tableRef = entry.getKey();
             timeStamps[i++] = validateAndGetServerTimestamp(tableRef, entry.getValue());
         }
         return timeStamps;
     }
     
-    private long validateAndGetServerTimestamp(TableRef tableRef, Map<ImmutableBytesPtr, RowMutationState> rowKeyToColumnMap) throws SQLException {
+    private long validateAndGetServerTimestamp(TableRef tableRef, MultiRowMutationState rowKeyToColumnMap) throws SQLException {
         Long scn = connection.getSCN();
         MetaDataClient client = new MetaDataClient(connection);
         long serverTimeStamp = tableRef.getTimeStamp();
@@ -919,7 +916,7 @@ public class MutationState implements SQLCloseable {
             sendAll = true;
         }
 
-        Map<ImmutableBytesPtr, RowMutationState> valuesMap;
+        MultiRowMutationState multiRowMutationState;
         Map<TableInfo,List<Mutation>> physicalTableMutationMap = Maps.newLinkedHashMap(); 
         // add tracing for this operation
         try (TraceScope trace = Tracing.startNewSpan(connection, "Committing mutations to tables")) {
@@ -928,16 +925,16 @@ public class MutationState implements SQLCloseable {
             while (tableRefIterator.hasNext()) {
                 // at this point we are going through mutations for each table
                 final TableRef tableRef = tableRefIterator.next();
-                valuesMap = mutations.get(tableRef);
-                if (valuesMap == null || valuesMap.isEmpty()) {
+                multiRowMutationState = mutations.get(tableRef);
+                if (multiRowMutationState == null || multiRowMutationState.isEmpty()) {
                     continue;
                 }
                 // Validate as we go if transactional since we can undo if a problem occurs (which is unlikely)
-                long serverTimestamp = serverTimeStamps == null ? validateAndGetServerTimestamp(tableRef, valuesMap) : serverTimeStamps[i++];
+                long serverTimestamp = serverTimeStamps == null ? validateAndGetServerTimestamp(tableRef, multiRowMutationState) : serverTimeStamps[i++];
                 Long scn = connection.getSCN();
                 long mutationTimestamp = scn == null ? HConstants.LATEST_TIMESTAMP : scn;
                 final PTable table = tableRef.getTable();
-                Iterator<Pair<PName,List<Mutation>>> mutationsIterator = addRowMutations(tableRef, valuesMap, mutationTimestamp, serverTimestamp, false, sendAll);
+                Iterator<Pair<PName,List<Mutation>>> mutationsIterator = addRowMutations(tableRef, multiRowMutationState, mutationTimestamp, serverTimestamp, false, sendAll);
                 // build map from physical table to mutation list
                 boolean isDataTable = true;
                 while (mutationsIterator.hasNext()) {
@@ -955,7 +952,7 @@ public class MutationState implements SQLCloseable {
                 // involved in the transaction since none of them would have been
                 // committed in the event of a failure.
                 if (table.isTransactional()) {
-                    addUncommittedStatementIndexes(valuesMap.values());
+                    addUncommittedStatementIndexes(multiRowMutationState.values());
                     if (txMutations.isEmpty()) {
                         txMutations = Maps.newHashMapWithExpectedSize(mutations.size());
                     }
@@ -964,7 +961,7 @@ public class MutationState implements SQLCloseable {
                     // in the event that we need to replay the commit.
                     // Copy TableRef so we have the original PTable and know when the
                     // indexes have changed.
-                    joinMutationState(new TableRef(tableRef), valuesMap, txMutations);
+                    joinMutationState(new TableRef(tableRef), multiRowMutationState, txMutations);
                 }
             }
             long serverTimestamp = HConstants.LATEST_TIMESTAMP;
@@ -1188,7 +1185,7 @@ public class MutationState implements SQLCloseable {
     }
     
     private int[] getUncommittedStatementIndexes() {
-        for (Map<ImmutableBytesPtr, RowMutationState> rowMutationMap : mutations.values()) {
+        for (MultiRowMutationState rowMutationMap : mutations.values()) {
             addUncommittedStatementIndexes(rowMutationMap.values());
         }
         return uncommittedStatementIndexes;
@@ -1221,7 +1218,7 @@ public class MutationState implements SQLCloseable {
     }
 
     public void commit() throws SQLException {
-        Map<TableRef, Map<ImmutableBytesPtr,RowMutationState>> txMutations = Collections.emptyMap();
+        Map<TableRef, MultiRowMutationState> txMutations = Collections.emptyMap();
         int retryCount = 0;
         do {
             boolean sendSuccessful=false;
@@ -1431,6 +1428,46 @@ public class MutationState implements SQLCloseable {
         }
     }
     
+    public static class MultiRowMutationState {
+        private Map<ImmutableBytesPtr,RowMutationState> rowKeyToRowMutationState;
+        private long estimatedSize;
+        
+        public MultiRowMutationState(int size) {
+            this.rowKeyToRowMutationState = Maps.newHashMapWithExpectedSize(size);
+            this.estimatedSize = 0;
+        }
+        
+        public RowMutationState put(ImmutableBytesPtr ptr, RowMutationState rowMutationState) { 
+            estimatedSize += rowMutationState.calculateEstimatedSize();
+            return rowKeyToRowMutationState.put(ptr, rowMutationState);
+        }
+        
+        public void putAll(MultiRowMutationState other) {
+            estimatedSize += other.estimatedSize;
+            rowKeyToRowMutationState.putAll(other.rowKeyToRowMutationState);
+        }
+        
+        public boolean isEmpty() {
+            return rowKeyToRowMutationState.isEmpty();
+        }
+        
+        public int size() {
+            return rowKeyToRowMutationState.size();
+        }
+        
+        public Set<Entry<ImmutableBytesPtr, RowMutationState>> entrySet() {
+            return rowKeyToRowMutationState.entrySet();
+        }
+        
+        public void clear(){
+            rowKeyToRowMutationState.clear();
+        }
+        
+        public Collection<RowMutationState> values() {
+            return rowKeyToRowMutationState.values();
+        }
+    }
+    
     public static class RowMutationState {
         @Nonnull private Map<PColumn,byte[]> columnValues;
         private int[] statementIndexes;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5b493962/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
index b23ea1b..74f91b4 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
@@ -73,7 +73,7 @@ import org.apache.phoenix.coprocessor.generated.MetaDataProtos.MetaDataService;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.UpdateIndexStateRequest;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.exception.SQLExceptionInfo;
-import org.apache.phoenix.execute.MutationState.RowMutationState;
+import org.apache.phoenix.execute.MutationState.MultiRowMutationState;
 import org.apache.phoenix.execute.TupleProjector;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.expression.KeyValueColumnExpression;
@@ -296,7 +296,7 @@ public class IndexUtil {
     }
     
     public static List<Mutation> generateIndexData(final PTable table, PTable index,
-            final Map<ImmutableBytesPtr, RowMutationState> valuesMap, List<Mutation> dataMutations, final KeyValueBuilder kvBuilder, PhoenixConnection connection)
+            final MultiRowMutationState multiRowMutationState, List<Mutation> dataMutations, final KeyValueBuilder kvBuilder, PhoenixConnection connection)
             throws SQLException {
         try {
         	final ImmutableBytesPtr ptr = new ImmutableBytesPtr();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5b493962/phoenix-core/src/main/java/org/apache/phoenix/util/KeyValueUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/KeyValueUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/KeyValueUtil.java
index 318c9d6..df6a349 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/KeyValueUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/KeyValueUtil.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValue.Type;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.phoenix.execute.MutationState.MultiRowMutationState;
 import org.apache.phoenix.execute.MutationState.RowMutationState;
 import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.hbase.index.util.KeyValueBuilder;
@@ -188,10 +189,10 @@ public class KeyValueUtil {
      * @return estimated row size
      */
     public static long
-            getEstimatedRowMutationSize(Map<TableRef, Map<ImmutableBytesPtr, RowMutationState>> tableMutationMap) {
+            getEstimatedRowMutationSize(Map<TableRef, MultiRowMutationState> tableMutationMap) {
         long size = 0;
         // iterate over table
-        for (Entry<TableRef, Map<ImmutableBytesPtr, RowMutationState>> tableEntry : tableMutationMap.entrySet()) {
+        for (Entry<TableRef, MultiRowMutationState> tableEntry : tableMutationMap.entrySet()) {
             // iterate over rows
             for (Entry<ImmutableBytesPtr, RowMutationState> rowEntry : tableEntry.getValue().entrySet()) {
                 size += calculateRowMutationSize(rowEntry);


[12/16] phoenix git commit: PHOENIX-672 Add GRANT and REVOKE commands using HBase AccessController

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/bd72ce3d/phoenix-core/src/it/java/org/apache/phoenix/end2end/TableDDLPermissionsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TableDDLPermissionsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TableDDLPermissionsIT.java
index 971383b..8666bb8 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TableDDLPermissionsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TableDDLPermissionsIT.java
@@ -16,144 +16,53 @@
  */
 package org.apache.phoenix.end2end;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.io.IOException;
-import java.lang.reflect.UndeclaredThrowableException;
 import java.security.PrivilegedExceptionAction;
 import java.sql.Connection;
-import java.sql.DriverManager;
-import java.sql.PreparedStatement;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.sql.Statement;
-import java.util.Arrays;
-import java.util.Collection;
 import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Set;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.AuthUtil;
-import org.apache.hadoop.hbase.HBaseTestingUtility;
-import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
-import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.security.AccessDeniedException;
 import org.apache.hadoop.hbase.security.access.AccessControlClient;
 import org.apache.hadoop.hbase.security.access.Permission.Action;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.phoenix.exception.PhoenixIOException;
-import org.apache.phoenix.query.QueryServices;
-import org.apache.phoenix.util.MetaDataUtil;
 import org.apache.phoenix.util.SchemaUtil;
-import org.junit.After;
-import org.junit.BeforeClass;
 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;
-
-import com.google.common.collect.Maps;
 
 /**
  * Test that verifies a user can read Phoenix tables with a minimal set of permissions.
  */
 @Category(NeedsOwnMiniClusterTest.class)
-@RunWith(Parameterized.class)
-public class TableDDLPermissionsIT{
-    private static String SUPERUSER;
-
-    private static HBaseTestingUtility testUtil;
-
-    private static final Set<String> PHOENIX_SYSTEM_TABLES = new HashSet<>(Arrays.asList(
-            "SYSTEM.CATALOG", "SYSTEM.SEQUENCE", "SYSTEM.STATS", "SYSTEM.FUNCTION",
-                "SYSTEM.MUTEX"));
-    // PHOENIX-XXXX SYSTEM.MUTEX isn't being created in the SYSTEM namespace as it should be.
-    private static final Set<String> PHOENIX_NAMESPACE_MAPPED_SYSTEM_TABLES = new HashSet<>(
-            Arrays.asList("SYSTEM:CATALOG", "SYSTEM:SEQUENCE", "SYSTEM:STATS", "SYSTEM:FUNCTION",
-                "SYSTEM.MUTEX"));
-    private static final String GROUP_SYSTEM_ACCESS = "group_system_access";
-    final UserGroupInformation superUser = UserGroupInformation.createUserForTesting(SUPERUSER, new String[0]);
-    final UserGroupInformation superUser2 = UserGroupInformation.createUserForTesting("superuser", new String[0]);
-    final UserGroupInformation regularUser = UserGroupInformation.createUserForTesting("user",  new String[0]);
-    final UserGroupInformation groupUser = UserGroupInformation.createUserForTesting("user2", new String[] { GROUP_SYSTEM_ACCESS });
-    final UserGroupInformation unprivilegedUser = UserGroupInformation.createUserForTesting("unprivilegedUser",
-            new String[0]);
-
+public class TableDDLPermissionsIT extends BasePermissionsIT {
 
-    private static final int NUM_RECORDS = 5;
-
-    private boolean isNamespaceMapped;
-
-    public TableDDLPermissionsIT(final boolean isNamespaceMapped) throws Exception {
-        this.isNamespaceMapped = isNamespaceMapped;
-        Map<String, String> clientProps = Maps.newHashMapWithExpectedSize(1);
-        clientProps.put(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, "true");
+    public TableDDLPermissionsIT(boolean isNamespaceMapped) throws Exception {
+        super(isNamespaceMapped);
     }
 
-    private void startNewMiniCluster(Configuration overrideConf) throws Exception{
-        if (null != testUtil) {
-            testUtil.shutdownMiniCluster();
-            testUtil = null;
-        }
-        testUtil = new HBaseTestingUtility();
-
-        Configuration config = testUtil.getConfiguration();
-        
-        config.set("hbase.coprocessor.master.classes",
-                "org.apache.hadoop.hbase.security.access.AccessController");
-        config.set("hbase.coprocessor.region.classes",
-                "org.apache.hadoop.hbase.security.access.AccessController");
-        config.set("hbase.coprocessor.regionserver.classes",
-                "org.apache.hadoop.hbase.security.access.AccessController");
-        config.set("hbase.security.exec.permission.checks", "true");
-        config.set("hbase.security.authorization", "true");
-        config.set("hbase.superuser", SUPERUSER+","+superUser2.getShortUserName());
-        config.set("hbase.regionserver.wal.codec", "org.apache.hadoop.hbase.regionserver.wal.IndexedWALEditCodec");
-        config.set(QueryServices.PHOENIX_ACLS_ENABLED,"true");
-        config.set(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, Boolean.toString(isNamespaceMapped));
-        // Avoid multiple clusters trying to bind the master's info port (16010)
-        config.setInt(HConstants.MASTER_INFO_PORT, -1);
-        
-        if (overrideConf != null) {
-            config.addResource(overrideConf);
-        }
-        testUtil.startMiniCluster(1);
-    }
-    
-    private void grantSystemTableAccess() throws Exception{
+    private void grantSystemTableAccess() throws Exception {
         try (Connection conn = getConnection()) {
             if (isNamespaceMapped) {
-                grantPermissions(regularUser.getShortUserName(), PHOENIX_NAMESPACE_MAPPED_SYSTEM_TABLES, Action.READ,
+                grantPermissions(regularUser1.getShortName(), PHOENIX_NAMESPACE_MAPPED_SYSTEM_TABLES, Action.READ,
                         Action.EXEC);
-                grantPermissions(unprivilegedUser.getShortUserName(), PHOENIX_NAMESPACE_MAPPED_SYSTEM_TABLES,
+                grantPermissions(unprivilegedUser.getShortName(), PHOENIX_NAMESPACE_MAPPED_SYSTEM_TABLES,
                         Action.READ, Action.EXEC);
                 grantPermissions(AuthUtil.toGroupEntry(GROUP_SYSTEM_ACCESS), PHOENIX_NAMESPACE_MAPPED_SYSTEM_TABLES,
                         Action.READ, Action.EXEC);
                 // Local Index requires WRITE permission on SYSTEM.SEQUENCE TABLE.
-                grantPermissions(regularUser.getShortUserName(), Collections.singleton("SYSTEM:SEQUENCE"), Action.WRITE,
+                grantPermissions(regularUser1.getShortName(), Collections.singleton("SYSTEM:SEQUENCE"), Action.WRITE,
                         Action.READ, Action.EXEC);
-                grantPermissions(unprivilegedUser.getShortUserName(), Collections.singleton("SYSTEM:SEQUENCE"), Action.WRITE,
+                grantPermissions(unprivilegedUser.getShortName(), Collections.singleton("SYSTEM:SEQUENCE"), Action.WRITE,
                         Action.READ, Action.EXEC);
                 
             } else {
-                grantPermissions(regularUser.getShortUserName(), PHOENIX_SYSTEM_TABLES, Action.READ, Action.EXEC);
-                grantPermissions(unprivilegedUser.getShortUserName(), PHOENIX_SYSTEM_TABLES, Action.READ, Action.EXEC);
+                grantPermissions(regularUser1.getShortName(), PHOENIX_SYSTEM_TABLES, Action.READ, Action.EXEC);
+                grantPermissions(unprivilegedUser.getShortName(), PHOENIX_SYSTEM_TABLES, Action.READ, Action.EXEC);
                 grantPermissions(AuthUtil.toGroupEntry(GROUP_SYSTEM_ACCESS), PHOENIX_SYSTEM_TABLES, Action.READ, Action.EXEC);
                 // Local Index requires WRITE permission on SYSTEM.SEQUENCE TABLE.
-                grantPermissions(regularUser.getShortUserName(), Collections.singleton("SYSTEM.SEQUENCE"), Action.WRITE,
+                grantPermissions(regularUser1.getShortName(), Collections.singleton("SYSTEM.SEQUENCE"), Action.WRITE,
                         Action.READ, Action.EXEC);
-                grantPermissions(unprivilegedUser.getShortUserName(), Collections.singleton("SYSTEM:SEQUENCE"), Action.WRITE,
+                grantPermissions(unprivilegedUser.getShortName(), Collections.singleton("SYSTEM:SEQUENCE"), Action.WRITE,
                         Action.READ, Action.EXEC);
             }
         } catch (Throwable e) {
@@ -165,40 +74,19 @@ public class TableDDLPermissionsIT{
         }
     }
 
-    @Parameters(name = "isNamespaceMapped={0}") // name is used by failsafe as file name in reports
-    public static Collection<Boolean> data() {
-        return Arrays.asList(true, false);
-    }
-
-    @BeforeClass
-    public static void doSetup() throws Exception {
-        SUPERUSER = System.getProperty("user.name");
-        //setUpTestDriver(new ReadOnlyProps(serverProps.entrySet().iterator()), new ReadOnlyProps(clientProps.entrySet().iterator()));
-    }
-
-    protected static String getUrl() {
-        return "jdbc:phoenix:localhost:" + testUtil.getZkCluster().getClientPort() + ":/hbase";
-    }
-
-    public Connection getConnection() throws SQLException{
-        Properties props = new Properties();
-        props.setProperty(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, Boolean.toString(isNamespaceMapped));
-        return DriverManager.getConnection(getUrl(),props);
-    }
-
     @Test
     public void testSchemaPermissions() throws Throwable{
 
         if (!isNamespaceMapped) { return; }
         try {
-            startNewMiniCluster(null);
+            startNewMiniCluster();
             grantSystemTableAccess();
             final String schemaName = "TEST_SCHEMA_PERMISSION";
-            superUser.doAs(new PrivilegedExceptionAction<Void>() {
+            superUser1.runAs(new PrivilegedExceptionAction<Void>() {
                 @Override
                 public Void run() throws Exception {
                     try {
-                        AccessControlClient.grant(getUtility().getConnection(), regularUser.getShortUserName(),
+                        AccessControlClient.grant(getUtility().getConnection(), regularUser1.getShortName(),
                                 Action.ADMIN);
                     } catch (Throwable e) {
                         if (e instanceof Exception) {
@@ -210,26 +98,20 @@ public class TableDDLPermissionsIT{
                     return null;
                 }
             });
-            verifyAllowed(createSchema(schemaName), regularUser);
+            verifyAllowed(createSchema(schemaName), regularUser1);
             // Unprivileged user cannot drop a schema
-            verifyDenied(dropSchema(schemaName), unprivilegedUser);
-            verifyDenied(createSchema(schemaName), unprivilegedUser);
+            verifyDenied(dropSchema(schemaName), AccessDeniedException.class, unprivilegedUser);
+            verifyDenied(createSchema(schemaName), AccessDeniedException.class, unprivilegedUser);
 
-            verifyAllowed(dropSchema(schemaName), regularUser);
+            verifyAllowed(dropSchema(schemaName), regularUser1);
         } finally {
             revokeAll();
         }
     }
 
     @Test
-    public void testAutomaticGrantDisabled() throws Throwable{
-        testIndexAndView(false);
-    }
-    
-    public void testIndexAndView(boolean isAutomaticGrant) throws Throwable {
-        Configuration conf = new Configuration();
-        conf.set(QueryServices.PHOENIX_AUTOMATIC_GRANT_ENABLED, Boolean.toString(isAutomaticGrant));
-        startNewMiniCluster(conf);
+    public void testAutomaticGrantWithIndexAndView() throws Throwable {
+        startNewMiniCluster();
         final String schema = "TEST_INDEX_VIEW";
         final String tableName = "TABLE_DDL_PERMISSION_IT";
         final String phoenixTableName = schema + "." + tableName;
@@ -244,17 +126,17 @@ public class TableDDLPermissionsIT{
         final String viewIndexName2 = tableName + "_VIDX2";
         grantSystemTableAccess();
         try {
-            superUser.doAs(new PrivilegedExceptionAction<Void>() {
+            superUser1.runAs(new PrivilegedExceptionAction<Void>() {
                 @Override
                 public Void run() throws Exception {
                     try {
-                        verifyAllowed(createSchema(schema), superUser);
+                        verifyAllowed(createSchema(schema), superUser1);
                         if (isNamespaceMapped) {
-                            grantPermissions(regularUser.getShortUserName(), schema, Action.CREATE);
+                            grantPermissions(regularUser1.getShortName(), schema, Action.CREATE);
                             grantPermissions(AuthUtil.toGroupEntry(GROUP_SYSTEM_ACCESS), schema, Action.CREATE);
 
                         } else {
-                            grantPermissions(regularUser.getShortUserName(),
+                            grantPermissions(regularUser1.getShortName(),
                                     NamespaceDescriptor.DEFAULT_NAMESPACE.getName(), Action.CREATE);
                             grantPermissions(AuthUtil.toGroupEntry(GROUP_SYSTEM_ACCESS),
                                     NamespaceDescriptor.DEFAULT_NAMESPACE.getName(), Action.CREATE);
@@ -271,29 +153,29 @@ public class TableDDLPermissionsIT{
                 }
             });
 
-            verifyAllowed(createTable(phoenixTableName), regularUser);
-            verifyAllowed(createIndex(indexName1, phoenixTableName), regularUser);
-            verifyAllowed(createView(viewName1, phoenixTableName), regularUser);
-            verifyAllowed(createLocalIndex(lIndexName1, phoenixTableName), regularUser);
-            verifyAllowed(createIndex(viewIndexName1, viewName1), regularUser);
-            verifyAllowed(createIndex(viewIndexName2, viewName1), regularUser);
-            verifyAllowed(createView(viewName4, viewName1), regularUser);
-            verifyAllowed(readTable(phoenixTableName), regularUser);
-
-            verifyDenied(createIndex(indexName2, phoenixTableName), unprivilegedUser);
-            verifyDenied(createView(viewName2, phoenixTableName), unprivilegedUser);
-            verifyDenied(createView(viewName3, viewName1), unprivilegedUser);
-            verifyDenied(dropView(viewName1), unprivilegedUser);
+            verifyAllowed(createTable(phoenixTableName), regularUser1);
+            verifyAllowed(createIndex(indexName1, phoenixTableName), regularUser1);
+            verifyAllowed(createView(viewName1, phoenixTableName), regularUser1);
+            verifyAllowed(createLocalIndex(lIndexName1, phoenixTableName), regularUser1);
+            verifyAllowed(createIndex(viewIndexName1, viewName1), regularUser1);
+            verifyAllowed(createIndex(viewIndexName2, viewName1), regularUser1);
+            verifyAllowed(createView(viewName4, viewName1), regularUser1);
+            verifyAllowed(readTable(phoenixTableName), regularUser1);
+
+            verifyDenied(createIndex(indexName2, phoenixTableName), AccessDeniedException.class, unprivilegedUser);
+            verifyDenied(createView(viewName2, phoenixTableName),AccessDeniedException.class,  unprivilegedUser);
+            verifyDenied(createView(viewName3, viewName1), AccessDeniedException.class, unprivilegedUser);
+            verifyDenied(dropView(viewName1), AccessDeniedException.class, unprivilegedUser);
             
-            verifyDenied(dropIndex(indexName1, phoenixTableName), unprivilegedUser);
-            verifyDenied(dropTable(phoenixTableName), unprivilegedUser);
-            verifyDenied(rebuildIndex(indexName1, phoenixTableName), unprivilegedUser);
-            verifyDenied(addColumn(phoenixTableName, "val1"), unprivilegedUser);
-            verifyDenied(dropColumn(phoenixTableName, "val"), unprivilegedUser);
-            verifyDenied(addProperties(phoenixTableName, "GUIDE_POSTS_WIDTH", "100"), unprivilegedUser);
+            verifyDenied(dropIndex(indexName1, phoenixTableName), AccessDeniedException.class, unprivilegedUser);
+            verifyDenied(dropTable(phoenixTableName), AccessDeniedException.class, unprivilegedUser);
+            verifyDenied(rebuildIndex(indexName1, phoenixTableName), AccessDeniedException.class, unprivilegedUser);
+            verifyDenied(addColumn(phoenixTableName, "val1"), AccessDeniedException.class, unprivilegedUser);
+            verifyDenied(dropColumn(phoenixTableName, "val"), AccessDeniedException.class, unprivilegedUser);
+            verifyDenied(addProperties(phoenixTableName, "GUIDE_POSTS_WIDTH", "100"), AccessDeniedException.class, unprivilegedUser);
 
             // Granting read permission to unprivileged user, now he should be able to create view but not index
-            grantPermissions(unprivilegedUser.getShortUserName(),
+            grantPermissions(unprivilegedUser.getShortName(),
                     Collections.singleton(
                             SchemaUtil.getPhysicalHBaseTableName(schema, tableName, isNamespaceMapped).getString()),
                     Action.READ, Action.EXEC);
@@ -301,52 +183,18 @@ public class TableDDLPermissionsIT{
                     Collections.singleton(
                             SchemaUtil.getPhysicalHBaseTableName(schema, tableName, isNamespaceMapped).getString()),
                     Action.READ, Action.EXEC);
-            verifyDenied(createIndex(indexName2, phoenixTableName), unprivilegedUser);
-            if (!isAutomaticGrant) {
-                // Automatic grant will read access for all indexes
-                verifyDenied(createView(viewName2, phoenixTableName), unprivilegedUser);
-
-                // Granting read permission to unprivileged user on index so that a new view can read a index as well,
-                // now
-                // he should be able to create view but not index
-                grantPermissions(unprivilegedUser.getShortUserName(),
-                        Collections.singleton(SchemaUtil
-                                .getPhysicalHBaseTableName(schema, indexName1, isNamespaceMapped).getString()),
-                        Action.READ, Action.EXEC);
-                verifyDenied(createView(viewName3, viewName1), unprivilegedUser);
-            }
-            
+            verifyDenied(createIndex(indexName2, phoenixTableName), AccessDeniedException.class, unprivilegedUser);
             verifyAllowed(createView(viewName2, phoenixTableName), unprivilegedUser);
-            
-            if (!isAutomaticGrant) {
-                // Grant access to view index for parent view
-                grantPermissions(unprivilegedUser.getShortUserName(),
-                        Collections.singleton(Bytes.toString(MetaDataUtil.getViewIndexPhysicalName(SchemaUtil
-                                .getPhysicalHBaseTableName(schema, tableName, isNamespaceMapped).getBytes()))),
-                        Action.READ, Action.EXEC);
-            }
             verifyAllowed(createView(viewName3, viewName1), unprivilegedUser);
             
             // Grant create permission in namespace
             if (isNamespaceMapped) {
-                grantPermissions(unprivilegedUser.getShortUserName(), schema, Action.CREATE);
+                grantPermissions(unprivilegedUser.getShortName(), schema, Action.CREATE);
             } else {
-                grantPermissions(unprivilegedUser.getShortUserName(), NamespaceDescriptor.DEFAULT_NAMESPACE.getName(),
+                grantPermissions(unprivilegedUser.getShortName(), NamespaceDescriptor.DEFAULT_NAMESPACE.getName(),
                         Action.CREATE);
             }
-            if (!isAutomaticGrant) {
-                verifyDenied(createIndex(indexName2, phoenixTableName), unprivilegedUser);
-                // Give user of data table access to index table which will be created by unprivilegedUser
-                grantPermissions(regularUser.getShortUserName(),
-                        Collections.singleton(SchemaUtil
-                                .getPhysicalHBaseTableName(schema, indexName2, isNamespaceMapped).getString()),
-                        Action.WRITE);
-                verifyDenied(createIndex(indexName2, phoenixTableName), unprivilegedUser);
-                grantPermissions(regularUser.getShortUserName(),
-                        Collections.singleton(SchemaUtil
-                                .getPhysicalHBaseTableName(schema, indexName2, isNamespaceMapped).getString()),
-                        Action.WRITE, Action.READ, Action.CREATE, Action.EXEC, Action.ADMIN);
-            }
+
             // we should be able to read the data from another index as well to which we have not given any access to
             // this user
             verifyAllowed(createIndex(indexName2, phoenixTableName), unprivilegedUser);
@@ -355,19 +203,19 @@ public class TableDDLPermissionsIT{
             verifyAllowed(rebuildIndex(indexName2, phoenixTableName), unprivilegedUser);
 
             // data table user should be able to read new index
-            verifyAllowed(rebuildIndex(indexName2, phoenixTableName), regularUser);
-            verifyAllowed(readTable(phoenixTableName, indexName2), regularUser);
-
-            verifyAllowed(readTable(phoenixTableName), regularUser);
-            verifyAllowed(rebuildIndex(indexName1, phoenixTableName), regularUser);
-            verifyAllowed(addColumn(phoenixTableName, "val1"), regularUser);
-            verifyAllowed(addProperties(phoenixTableName, "GUIDE_POSTS_WIDTH", "100"), regularUser);
-            verifyAllowed(dropView(viewName1), regularUser);
-            verifyAllowed(dropView(viewName2), regularUser);
-            verifyAllowed(dropColumn(phoenixTableName, "val1"), regularUser);
-            verifyAllowed(dropIndex(indexName2, phoenixTableName), regularUser);
-            verifyAllowed(dropIndex(indexName1, phoenixTableName), regularUser);
-            verifyAllowed(dropTable(phoenixTableName), regularUser);
+            verifyAllowed(rebuildIndex(indexName2, phoenixTableName), regularUser1);
+            verifyAllowed(readTable(phoenixTableName, indexName2), regularUser1);
+
+            verifyAllowed(readTable(phoenixTableName), regularUser1);
+            verifyAllowed(rebuildIndex(indexName1, phoenixTableName), regularUser1);
+            verifyAllowed(addColumn(phoenixTableName, "val1"), regularUser1);
+            verifyAllowed(addProperties(phoenixTableName, "GUIDE_POSTS_WIDTH", "100"), regularUser1);
+            verifyAllowed(dropView(viewName1), regularUser1);
+            verifyAllowed(dropView(viewName2), regularUser1);
+            verifyAllowed(dropColumn(phoenixTableName, "val1"), regularUser1);
+            verifyAllowed(dropIndex(indexName2, phoenixTableName), regularUser1);
+            verifyAllowed(dropIndex(indexName1, phoenixTableName), regularUser1);
+            verifyAllowed(dropTable(phoenixTableName), regularUser1);
 
             // check again with super users
             verifyAllowed(createTable(phoenixTableName), superUser2);
@@ -381,312 +229,5 @@ public class TableDDLPermissionsIT{
             revokeAll();
         }
     }
-    
-    
-    @Test
-    public void testAutomaticGrantEnabled() throws Throwable{
-        testIndexAndView(true);
-    }
-
-    private void revokeAll() throws IOException, Throwable {
-        AccessControlClient.revoke(getUtility().getConnection(), AuthUtil.toGroupEntry(GROUP_SYSTEM_ACCESS),Action.values() );
-        AccessControlClient.revoke(getUtility().getConnection(), regularUser.getShortUserName(),Action.values() );
-        AccessControlClient.revoke(getUtility().getConnection(), unprivilegedUser.getShortUserName(),Action.values() );
-        
-    }
-
-    protected void grantPermissions(String groupEntry, Action... actions) throws IOException, Throwable {
-        AccessControlClient.grant(getUtility().getConnection(), groupEntry, actions);
-    }
-
-    private AccessTestAction dropTable(final String tableName) throws SQLException {
-        return new AccessTestAction() {
-            @Override
-            public Object run() throws Exception {
-                try (Connection conn = getConnection(); Statement stmt = conn.createStatement();) {
-                    assertFalse(stmt.execute("DROP TABLE IF EXISTS " + tableName));
-                }
-                return null;
-            }
-        };
-
-    }
-
-    private AccessTestAction createTable(final String tableName) throws SQLException {
-        return new AccessTestAction() {
-            @Override
-            public Object run() throws Exception {
-        try (Connection conn = getConnection(); Statement stmt = conn.createStatement();) {
-            assertFalse(stmt.execute("CREATE TABLE " + tableName + "(pk INTEGER not null primary key, data VARCHAR,val integer)"));
-            try (PreparedStatement pstmt = conn.prepareStatement("UPSERT INTO " + tableName + " values(?, ?, ?)")) {
-                for (int i = 0; i < NUM_RECORDS; i++) {
-                    pstmt.setInt(1, i);
-                    pstmt.setString(2, Integer.toString(i));
-                    pstmt.setInt(3, i);
-                    assertEquals(1, pstmt.executeUpdate());
-                }
-            }
-            conn.commit();
-        }
-        return null;
-            }
-        };
-    }
-
-    private AccessTestAction readTable(final String tableName) throws SQLException {
-        return readTable(tableName,null);
-    }
-    private AccessTestAction readTable(final String tableName, final String indexName) throws SQLException {
-        return new AccessTestAction() {
-            @Override
-            public Object run() throws Exception {
-                try (Connection conn = getConnection(); Statement stmt = conn.createStatement()) {
-                    ResultSet rs = stmt.executeQuery("SELECT "+(indexName!=null?"/*+ INDEX("+tableName+" "+indexName+")*/":"")+" pk, data,val FROM " + tableName +" where data>='0'");
-                    assertNotNull(rs);
-                    int i = 0;
-                    while (rs.next()) {
-                        assertEquals(i, rs.getInt(1));
-                        assertEquals(Integer.toString(i), rs.getString(2));
-                        assertEquals(i, rs.getInt(3));
-                        i++;
-                    }
-                    assertEquals(NUM_RECORDS, i);
-                }
-                return null;
-                }
-            };
-    }
-
-    public static HBaseTestingUtility getUtility(){
-        return testUtil;
-    }
-
-    private void grantPermissions(String toUser, Set<String> tablesToGrant, Action... actions) throws Throwable {
-        for (String table : tablesToGrant) {
-            AccessControlClient.grant(getUtility().getConnection(), TableName.valueOf(table), toUser, null, null,
-                    actions);
-        }
-    }
-
-    private void grantPermissions(String toUser, String namespace, Action... actions) throws Throwable {
-        AccessControlClient.grant(getUtility().getConnection(), namespace, toUser, actions);
-    }
-    
-
-    private AccessTestAction dropColumn(final String tableName, final String columnName) throws SQLException {
-        return new AccessTestAction() {
-            @Override
-            public Object run() throws Exception {
-                try (Connection conn = getConnection(); Statement stmt = conn.createStatement();) {
-                    assertFalse(stmt.execute("ALTER TABLE " + tableName + " DROP COLUMN "+columnName));
-                }
-                return null;
-            }
-        };
-    }
-
-    private AccessTestAction addColumn(final String tableName, final String columnName) throws SQLException {
-        return new AccessTestAction() {
-            @Override
-            public Object run() throws Exception {
-                try (Connection conn = getConnection(); Statement stmt = conn.createStatement();) {
-                    assertFalse(stmt.execute("ALTER TABLE " + tableName + " ADD "+columnName+" varchar"));
-                }
-                return null;
-            }
-        };
-    }
-
-    private AccessTestAction addProperties(final String tableName, final String property, final String value)
-            throws SQLException {
-        return new AccessTestAction() {
-            @Override
-            public Object run() throws Exception {
-                try (Connection conn = getConnection(); Statement stmt = conn.createStatement();) {
-                    assertFalse(stmt.execute("ALTER TABLE " + tableName + " SET " + property + "=" + value));
-                }
-                return null;
-            }
-        };
-    }
-
-    private AccessTestAction dropView(final String viewName) throws SQLException {
-        return new AccessTestAction() {
-            @Override
-            public Object run() throws Exception {
-                try (Connection conn = getConnection(); Statement stmt = conn.createStatement();) {
-                    assertFalse(stmt.execute("DROP VIEW " + viewName));
-                }
-                return null;
-            }
-        };
-    }
-
-    private AccessTestAction createView(final String viewName, final String dataTable) throws SQLException {
-        return new AccessTestAction() {
-            @Override
-            public Object run() throws Exception {
-                try (Connection conn = getConnection(); Statement stmt = conn.createStatement();) {
-                    assertFalse(stmt.execute("CREATE VIEW " + viewName + " AS SELECT * FROM " + dataTable));
-                }
-                return null;
-            }
-        };
-    }
-
-    private AccessTestAction createIndex(final String indexName, final String dataTable) throws SQLException {
-        return new AccessTestAction() {
-            @Override
-            public Object run() throws Exception {
-
-                try (Connection conn = getConnection(); Statement stmt = conn.createStatement();) {
-                    assertFalse(stmt.execute("CREATE INDEX " + indexName + " on " + dataTable + "(data)"));
-                }
-                return null;
-            }
-        };
-    }
-    
-    private AccessTestAction createLocalIndex(final String indexName, final String dataTable) throws SQLException {
-        return new AccessTestAction() {
-            @Override
-            public Object run() throws Exception {
-
-                try (Connection conn = getConnection(); Statement stmt = conn.createStatement();) {
-                    assertFalse(stmt.execute("CREATE LOCAL INDEX " + indexName + " on " + dataTable + "(data)"));
-                }
-                return null;
-            }
-        };
-    }
-
-    private AccessTestAction dropIndex(final String indexName, final String dataTable) throws SQLException {
-        return new AccessTestAction() {
-            @Override
-            public Object run() throws Exception {
-                try (Connection conn = getConnection(); Statement stmt = conn.createStatement();) {
-                    assertFalse(stmt.execute("DROP INDEX " + indexName + " on " + dataTable));
-                }
-                return null;
-            }
-        };
-    }
-
-    private AccessTestAction createSchema(final String schemaName) throws SQLException {
-        return new AccessTestAction() {
-            @Override
-            public Object run() throws Exception {
-                if (isNamespaceMapped) {
-                    try (Connection conn = getConnection(); Statement stmt = conn.createStatement();) {
-                        assertFalse(stmt.execute("CREATE SCHEMA " + schemaName));
-                    }
-                }
-                return null;
-            }
-        };
-    }
-
-    private AccessTestAction dropSchema(final String schemaName) throws SQLException {
-        return new AccessTestAction() {
-            @Override
-            public Object run() throws Exception {
-                if (isNamespaceMapped) {
-                    try (Connection conn = getConnection(); Statement stmt = conn.createStatement();) {
-                        assertFalse(stmt.execute("DROP SCHEMA " + schemaName));
-                    }
-                }
-                return null;
-            }
-        };
-    }
-
-    private AccessTestAction rebuildIndex(final String indexName, final String dataTable) throws SQLException {
-        return new AccessTestAction() {
-            @Override
-            public Object run() throws Exception {
-                try (Connection conn = getConnection(); Statement stmt = conn.createStatement();) {
-                    assertFalse(stmt.execute("ALTER INDEX " + indexName + " on " + dataTable + " DISABLE"));
-                    assertFalse(stmt.execute("ALTER INDEX " + indexName + " on " + dataTable + " REBUILD"));
-                }
-                return null;
-            }
-        };
-    }
-
-    static interface AccessTestAction extends PrivilegedExceptionAction<Object> { }
-
-    @After
-    public void cleanup() throws Exception {
-        if (null != testUtil) {
-          testUtil.shutdownMiniCluster();
-          testUtil = null;
-        }
-    }
-
-    /** This fails only in case of ADE or empty list for any of the users. */
-    private void verifyAllowed(AccessTestAction action, UserGroupInformation... users) throws Exception {
-      for (UserGroupInformation user : users) {
-        verifyAllowed(user, action);
-      }
-    }
-
-    /** This passes only in case of ADE for all users. */
-    private void verifyDenied(AccessTestAction action, UserGroupInformation... users) throws Exception {
-      for (UserGroupInformation user : users) {
-        verifyDenied(user, action);
-      }
-    }
-
-    /** This fails only in case of ADE or empty list for any of the actions. */
-    private void verifyAllowed(UserGroupInformation user, AccessTestAction... actions) throws Exception {
-      for (AccessTestAction action : actions) {
-        try {
-          Object obj = user.doAs(action);
-          if (obj != null && obj instanceof List<?>) {
-            List<?> results = (List<?>) obj;
-            if (results != null && results.isEmpty()) {
-              fail("Empty non null results from action for user '" + user.getShortUserName() + "'");
-            }
-          }
-        } catch (AccessDeniedException ade) {
-          fail("Expected action to pass for user '" + user.getShortUserName() + "' but was denied");
-        }
-      }
-    }
-
-    /** This passes only in case of ADE for all actions. */
-    private void verifyDenied(UserGroupInformation user, AccessTestAction... actions) throws Exception {
-        for (AccessTestAction action : actions) {
-            try {
-                user.doAs(action);
-                fail("Expected exception was not thrown for user '" + user.getShortUserName() + "'");
-            } catch (IOException e) {
-                fail("Expected exception was not thrown for user '" + user.getShortUserName() + "'");
-            } catch (UndeclaredThrowableException ute) {
-                Throwable ex = ute.getUndeclaredThrowable();
-
-                if (ex instanceof PhoenixIOException) {
-                    if (ex.getCause() instanceof AccessDeniedException) {
-                        // expected result
-                        validateAccessDeniedException((AccessDeniedException) ex.getCause());
-                        return;
-                    }
-                }
-            }catch(RuntimeException ex){
-                // This can occur while accessing tabledescriptors from client by the unprivileged user
-                if (ex.getCause() instanceof AccessDeniedException) {
-                    // expected result
-                    validateAccessDeniedException((AccessDeniedException) ex.getCause());
-                    return;
-                }
-            }
-            fail("Expected exception was not thrown for user '" + user.getShortUserName() + "'");
-        }
-    }
 
-    private void validateAccessDeniedException(AccessDeniedException ade) {
-        String msg = ade.getMessage();
-        assertTrue("Exception contained unexpected message: '" + msg + "'",
-            !msg.contains("is not the scanner owner"));
-    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bd72ce3d/phoenix-core/src/main/antlr3/PhoenixSQL.g
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/antlr3/PhoenixSQL.g b/phoenix-core/src/main/antlr3/PhoenixSQL.g
index 93e0ede..8c9c135 100644
--- a/phoenix-core/src/main/antlr3/PhoenixSQL.g
+++ b/phoenix-core/src/main/antlr3/PhoenixSQL.g
@@ -118,6 +118,7 @@ tokens
     UNION='union';
     FUNCTION='function';
     AS='as';
+    TO='to';
     TEMPORARY='temporary';
     RETURNS='returns';
     USING='using';
@@ -144,6 +145,8 @@ tokens
     DUPLICATE = 'duplicate';
     IGNORE = 'ignore';
     IMMUTABLE = 'immutable';
+    GRANT = 'grant';
+    REVOKE = 'revoke';
 }
 
 
@@ -430,6 +433,8 @@ oneStatement returns [BindableStatement ret]
     |   s=delete_jar_node
     |   s=alter_session_node
     |	s=create_sequence_node
+    |   s=grant_permission_node
+    |   s=revoke_permission_node
     |	s=drop_sequence_node
     |	s=drop_schema_node
     |	s=use_schema_node
@@ -458,6 +463,30 @@ create_schema_node returns [CreateSchemaStatement ret]
         {ret = factory.createSchema(s, ex!=null); }
     ;
 
+// Parse a grant permission statement
+grant_permission_node returns [ChangePermsStatement ret]
+    :   GRANT p=literal (ON ((TABLE)? table=table_name | s=SCHEMA schema=identifier))? TO (g=GROUP)? ug=literal
+        {
+            String permsString = SchemaUtil.normalizeLiteral(p);
+            if (permsString != null && permsString.length() > 5) {
+                throw new RuntimeException("Permissions String length should be less than 5 characters");
+            }
+            $ret = factory.changePermsStatement(permsString, s!=null, table, schema, g!=null, ug, Boolean.TRUE);
+        }
+    ;
+
+// Parse a revoke permission statement
+revoke_permission_node returns [ChangePermsStatement ret]
+    :   REVOKE (p=literal)? (ON ((TABLE)? table=table_name | s=SCHEMA schema=identifier))? FROM (g=GROUP)? ug=literal
+        {
+            String permsString = SchemaUtil.normalizeLiteral(p);
+            if (permsString != null && permsString.length() > 5) {
+                throw new RuntimeException("Permissions String length should be less than 5 characters");
+            }
+            $ret = factory.changePermsStatement(permsString, s!=null, table, schema, g!=null, ug, Boolean.FALSE);
+        }
+    ;
+
 // Parse a create view statement.
 create_view_node returns [CreateTableStatement ret]
     :   CREATE VIEW (IF NOT ex=EXISTS)? t=from_table_name 
@@ -1161,7 +1190,6 @@ BIND_NAME
     : COLON (DIGIT)+
     ;
 
-
 NAME
     :    LETTER (FIELDCHAR)*
     |    '\"' (DBL_QUOTE_CHAR)* '\"'

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bd72ce3d/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/PhoenixAccessController.java
----------------------------------------------------------------------
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 8437b37..a4bc857 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
@@ -75,8 +75,6 @@ public class PhoenixAccessController extends BaseMetaDataEndpointObserver {
     private ArrayList<BaseMasterAndRegionObserver> accessControllers;
     private boolean accessCheckEnabled;
     private UserProvider userProvider;
-    private boolean isAutomaticGrantEnabled;
-    private boolean isStrictMode;
     public static final Log LOG = LogFactory.getLog(PhoenixAccessController.class);
     private static final Log AUDITLOG =
             LogFactory.getLog("SecurityLogger."+PhoenixAccessController.class.getName());
@@ -114,8 +112,6 @@ public class PhoenixAccessController extends BaseMetaDataEndpointObserver {
         Configuration conf = env.getConfiguration();
         this.accessCheckEnabled = conf.getBoolean(QueryServices.PHOENIX_ACLS_ENABLED,
                 QueryServicesOptions.DEFAULT_PHOENIX_ACLS_ENABLED);
-        this.isAutomaticGrantEnabled=conf.getBoolean(QueryServices.PHOENIX_AUTOMATIC_GRANT_ENABLED,
-                QueryServicesOptions.DEFAULT_PHOENIX_AUTOMATIC_GRANT_ENABLED);
         if (!this.accessCheckEnabled) {
             LOG.warn("PhoenixAccessController has been loaded with authorization checks disabled.");
         }
@@ -127,8 +123,6 @@ public class PhoenixAccessController extends BaseMetaDataEndpointObserver {
         }
         // set the user-provider.
         this.userProvider = UserProvider.instantiate(env.getConfiguration());
-        this.isStrictMode = conf.getBoolean(QueryServices.PHOENIX_SECURITY_PERMISSION_STRICT_MODE_ENABLED,
-                QueryServicesOptions.DEFAULT_PHOENIX_SECURITY_PERMISSION_STRICT_MODE_ENABLED);
         // init superusers and add the server principal (if using security)
         // or process owner as default super user.
         Superusers.initialize(env.getConfiguration());
@@ -223,23 +217,12 @@ public class PhoenixAccessController extends BaseMetaDataEndpointObserver {
     public void handleRequireAccessOnDependentTable(String request, String userName, TableName dependentTable,
             String requestTable, Set<Action> requireAccess, Set<Action> accessExists) throws IOException {
 
-        if (!isStrictMode) {
-            AUDITLOG.warn("Strict mode is not enabled, so " + request + " is allowed but User:" + userName
-                    + " will not have following access " + requireAccess + " to the existing dependent physical table "
-                    + dependentTable);
-            return;
-        }
-        if (isAutomaticGrantEnabled) {
-            Set<Action> unionSet = new HashSet<Action>();
-            unionSet.addAll(requireAccess);
-            unionSet.addAll(accessExists);
-            AUDITLOG.info(request + ": Automatically granting access to index table during creation of view:"
-                    + requestTable + authString(userName, dependentTable, requireAccess));
-            grantPermissions(userName, dependentTable.getName(), unionSet.toArray(new Action[0]));
-        } else {
-            throw new AccessDeniedException(
-                    "Insufficient permissions for users of dependent table" + authString(userName, dependentTable, requireAccess));
-        }
+        Set<Action> unionSet = new HashSet<Action>();
+        unionSet.addAll(requireAccess);
+        unionSet.addAll(accessExists);
+        AUDITLOG.info(request + ": Automatically granting access to index table during creation of view:"
+                + requestTable + authString(userName, dependentTable, requireAccess));
+        grantPermissions(userName, dependentTable.getName(), unionSet.toArray(new Action[0]));
     }
     
     private void grantPermissions(final String toUser, final byte[] table, final Action... actions) throws IOException {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bd72ce3d/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java b/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
index e51fd9f..2301c32 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
@@ -229,6 +229,7 @@ public enum SQLExceptionCode {
             return new TableAlreadyExistsException(info.getSchemaName(), info.getTableName());
         }
     }),
+    TABLES_NOT_IN_SYNC(1140, "42M05", "Tables not in sync for some properties."),
 
     // Syntax error
     TYPE_NOT_SUPPORTED_FOR_OPERATOR(1014, "42Y01", "The operator does not support the operand type."),

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bd72ce3d/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
index 174e643..384c8cc 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
@@ -97,6 +97,7 @@ import org.apache.phoenix.parse.AliasedNode;
 import org.apache.phoenix.parse.AlterIndexStatement;
 import org.apache.phoenix.parse.AlterSessionStatement;
 import org.apache.phoenix.parse.BindableStatement;
+import org.apache.phoenix.parse.ChangePermsStatement;
 import org.apache.phoenix.parse.CloseStatement;
 import org.apache.phoenix.parse.ColumnDef;
 import org.apache.phoenix.parse.ColumnName;
@@ -212,8 +213,9 @@ public class PhoenixStatement implements Statement, SQLCloseable {
         QUERY("queried", false),
         DELETE("deleted", true),
         UPSERT("upserted", true),
-        UPGRADE("upgrade", true);
-        
+        UPGRADE("upgrade", true),
+        ADMIN("admin", true);
+
         private final String toString;
         private final boolean isMutation;
         Operation(String toString, boolean isMutation) {
@@ -1153,6 +1155,33 @@ public class PhoenixStatement implements Statement, SQLCloseable {
         }
     }
 
+    private static class ExecutableChangePermsStatement extends ChangePermsStatement implements CompilableStatement {
+
+        public ExecutableChangePermsStatement (String permsString, boolean isSchemaName, TableName tableName,
+                                               String schemaName, boolean isGroupName, LiteralParseNode userOrGroup, boolean isGrantStatement) {
+            super(permsString, isSchemaName, tableName, schemaName, isGroupName, userOrGroup, isGrantStatement);
+        }
+
+        @Override
+        public MutationPlan compilePlan(PhoenixStatement stmt, Sequence.ValueOp seqAction) throws SQLException {
+            final StatementContext context = new StatementContext(stmt);
+
+            return new BaseMutationPlan(context, this.getOperation()) {
+
+                @Override
+                public ExplainPlan getExplainPlan() throws SQLException {
+                    return new ExplainPlan(Collections.singletonList("GRANT PERMISSION"));
+                }
+
+                @Override
+                public MutationState execute() throws SQLException {
+                    MetaDataClient client = new MetaDataClient(getContext().getConnection());
+                    return client.changePermissions(ExecutableChangePermsStatement.this);
+                }
+            };
+        }
+    }
+
     private static class ExecutableDropIndexStatement extends DropIndexStatement implements CompilableStatement {
 
         public ExecutableDropIndexStatement(NamedNode indexName, TableName tableName, boolean ifExists) {
@@ -1558,6 +1587,13 @@ public class PhoenixStatement implements Statement, SQLCloseable {
         public ExecuteUpgradeStatement executeUpgrade() {
             return new ExecutableExecuteUpgradeStatement();
         }
+
+        @Override
+        public ExecutableChangePermsStatement changePermsStatement(String permsString, boolean isSchemaName, TableName tableName,
+                                                         String schemaName, boolean isGroupName, LiteralParseNode userOrGroup, boolean isGrantStatement) {
+            return new ExecutableChangePermsStatement(permsString, isSchemaName, tableName, schemaName, isGroupName, userOrGroup,isGrantStatement);
+        }
+
     }
     
     static class PhoenixStatementParser extends SQLParser {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bd72ce3d/phoenix-core/src/main/java/org/apache/phoenix/parse/ChangePermsStatement.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/ChangePermsStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/ChangePermsStatement.java
new file mode 100644
index 0000000..0eae26f
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/ChangePermsStatement.java
@@ -0,0 +1,102 @@
+/*
+ * 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.parse;
+
+import org.antlr.runtime.RecognitionException;
+import org.apache.hadoop.hbase.AuthUtil;
+import org.apache.hadoop.hbase.security.access.Permission;
+import org.apache.phoenix.exception.PhoenixParserException;
+import org.apache.phoenix.jdbc.PhoenixStatement;
+import org.apache.phoenix.util.SchemaUtil;
+
+import java.util.Arrays;
+
+/**
+ * See PHOENIX-672, Use GRANT/REVOKE statements to assign or remove permissions for a user OR group on a table OR namespace
+ * Permissions are managed by HBase using hbase:acl table, Allowed permissions are RWXCA
+ */
+public class ChangePermsStatement implements BindableStatement {
+
+    private Permission.Action[] permsList;
+    private TableName tableName;
+    private String schemaName;
+    private String name;
+    // Grant/Revoke statements are differentiated based on this boolean
+    private boolean isGrantStatement;
+
+    public ChangePermsStatement(String permsString, boolean isSchemaName,
+                                TableName tableName, String schemaName, boolean isGroupName, LiteralParseNode ugNode, boolean isGrantStatement) {
+        // PHOENIX-672 HBase API doesn't allow to revoke specific permissions, hence this parameter will be ignored here.
+        // To comply with SQL standards, we may support the user given permissions to revoke specific permissions in future.
+        // GRANT permissions statement requires this parameter and the parsing will fail if it is not specified in SQL
+        if(permsString != null) {
+            Permission permission = new Permission(permsString.getBytes());
+            permsList = permission.getActions();
+        }
+        if(isSchemaName) {
+            this.schemaName = SchemaUtil.normalizeIdentifier(schemaName);
+        } else {
+            this.tableName = tableName;
+        }
+        name = SchemaUtil.normalizeLiteral(ugNode);
+        name = isGroupName ? AuthUtil.toGroupEntry(name) : name;
+        this.isGrantStatement = isGrantStatement;
+    }
+
+    public Permission.Action[] getPermsList() {
+        return permsList;
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public TableName getTableName() {
+        return tableName;
+    }
+
+    public String getSchemaName() {
+        return schemaName;
+    }
+
+    public boolean isGrantStatement() {
+        return isGrantStatement;
+    }
+
+    public String toString() {
+        StringBuffer buffer = new StringBuffer();
+        buffer = this.isGrantStatement() ? buffer.append("GRANT ") : buffer.append("REVOKE ");
+        buffer.append("permissions requested for user/group: " + this.getName());
+        if (this.getSchemaName() != null) {
+            buffer.append(" for Schema: " + this.getSchemaName());
+        } else if (this.getTableName() != null) {
+            buffer.append(" for Table: " + this.getTableName());
+        }
+        buffer.append(" Permissions: " + Arrays.toString(this.getPermsList()));
+        return buffer.toString();
+    }
+
+    @Override
+    public int getBindCount() {
+        return 0;
+    }
+
+    @Override
+    public PhoenixStatement.Operation getOperation() {
+        return PhoenixStatement.Operation.ADMIN;
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bd72ce3d/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java
index 0058f38..32c3d8d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java
@@ -25,7 +25,6 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.Collection;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import com.google.common.collect.ArrayListMultimap;
@@ -925,4 +924,10 @@ public class ParseNodeFactory {
     public UseSchemaStatement useSchema(String schemaName) {
         return new UseSchemaStatement(schemaName);
     }
+
+    public ChangePermsStatement changePermsStatement(String permsString, boolean isSchemaName, TableName tableName
+            , String schemaName, boolean isGroupName, LiteralParseNode userOrGroup, boolean isGrantStatement) {
+        return new ChangePermsStatement(permsString, isSchemaName, tableName, schemaName, isGroupName, userOrGroup, isGrantStatement);
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bd72ce3d/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
index 7a255a1..08aadfb 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
@@ -103,6 +103,7 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.NamespaceNotFoundException;
 import org.apache.hadoop.hbase.TableExistsException;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Append;
@@ -190,6 +191,7 @@ import org.apache.phoenix.schema.EmptySequenceCacheException;
 import org.apache.phoenix.schema.FunctionNotFoundException;
 import org.apache.phoenix.schema.MetaDataClient;
 import org.apache.phoenix.schema.MetaDataSplitPolicy;
+import org.apache.phoenix.schema.NewerSchemaAlreadyExistsException;
 import org.apache.phoenix.schema.NewerTableAlreadyExistsException;
 import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PColumnFamily;
@@ -1224,7 +1226,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                             SQLExceptionCode.INCONSISTENT_NAMESPACE_MAPPING_PROPERTIES)
                     .setMessage(
                             "Ensure that config " + QueryServices.IS_NAMESPACE_MAPPING_ENABLED
-                            + " is consitent on client and server.")
+                            + " is consistent on client and server.")
                             .build().buildException(); }
             lowestClusterHBaseVersion = minHBaseVersion;
         } catch (SQLException e) {
@@ -2460,6 +2462,11 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                         logger.warn("Could not check for Phoenix SYSTEM tables, assuming they exist and are properly configured");
                                         checkClientServerCompatibility(SchemaUtil.getPhysicalName(SYSTEM_CATALOG_NAME_BYTES, getProps()).getName());
                                         success = true;
+                                    } else if (!Iterables.isEmpty(Iterables.filter(Throwables.getCausalChain(e), NamespaceNotFoundException.class))) {
+                                        // This exception is only possible if SYSTEM namespace mapping is enabled and SYSTEM namespace is missing
+                                        // It implies that SYSTEM tables are not created and hence we shouldn't provide a connection
+                                        AccessDeniedException ade = new AccessDeniedException("Insufficient permissions to create SYSTEM namespace and SYSTEM Tables");
+                                        initializationException = ServerUtil.parseServerException(ade);
                                     } else {
                                         initializationException = e;
                                     }
@@ -2471,8 +2478,19 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                 // with SYSTEM Namespace. (See PHOENIX-4227 https://issues.apache.org/jira/browse/PHOENIX-4227)
                                 if (SchemaUtil.isNamespaceMappingEnabled(PTableType.SYSTEM,
                                         ConnectionQueryServicesImpl.this.getProps())) {
-                                    metaConnection.createStatement().execute("CREATE SCHEMA IF NOT EXISTS "
-                                            + PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA);
+                                    try {
+                                        metaConnection.createStatement().execute("CREATE SCHEMA IF NOT EXISTS "
+                                                + PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA);
+                                    } catch (NewerSchemaAlreadyExistsException e) {
+                                        // Older clients with appropriate perms may try getting a new connection
+                                        // This results in NewerSchemaAlreadyExistsException, so we can safely ignore it here
+                                    } catch (PhoenixIOException e) {
+                                        if (!Iterables.isEmpty(Iterables.filter(Throwables.getCausalChain(e), AccessDeniedException.class))) {
+                                            // Ignore ADE
+                                        } else {
+                                            throw e;
+                                        }
+                                    }
                                 }
                                 if (!ConnectionQueryServicesImpl.this.upgradeRequired.get()) {
                                     createOtherSystemTables(metaConnection, hBaseAdmin);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bd72ce3d/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
index 7607388..851ba9a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
@@ -149,6 +149,7 @@ public interface QueryConstants {
     public enum JoinType {INNER, LEFT_OUTER}
     public final static String SYSTEM_SCHEMA_NAME = "SYSTEM";
     public final static byte[] SYSTEM_SCHEMA_NAME_BYTES = Bytes.toBytes(SYSTEM_SCHEMA_NAME);
+    public final static String HBASE_DEFAULT_SCHEMA_NAME = "default";
     public final static String PHOENIX_METADATA = "table";
     public final static String OFFSET_ROW_KEY = "_OFFSET_";
     public final static byte[] OFFSET_ROW_KEY_BYTES = Bytes.toBytes(OFFSET_ROW_KEY);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bd72ce3d/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
index b9ed734..59f7385 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
@@ -262,8 +262,6 @@ public interface QueryServices extends SQLCloseable {
     public static final String UPLOAD_BINARY_DATA_TYPE_ENCODING = "phoenix.upload.binaryDataType.encoding";
     // Toggle for server-written updates to SYSTEM.CATALOG
     public static final String PHOENIX_ACLS_ENABLED = "phoenix.acls.enabled";
-    public static final String PHOENIX_AUTOMATIC_GRANT_ENABLED = "phoenix.security.automatic.grant.enabled";
-    public static final String PHOENIX_SECURITY_PERMISSION_STRICT_MODE_ENABLED = "phoenix.security.strict.mode.enabled";
 
     public static final String INDEX_ASYNC_BUILD_ENABLED = "phoenix.index.async.build.enabled";
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bd72ce3d/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
index a586c28..3ceb084 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
@@ -59,13 +59,11 @@ import static org.apache.phoenix.query.QueryServices.MIN_STATS_UPDATE_FREQ_MS_AT
 import static org.apache.phoenix.query.QueryServices.MUTATE_BATCH_SIZE_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.NUM_RETRIES_FOR_SCHEMA_UPDATE_CHECK;
 import static org.apache.phoenix.query.QueryServices.PHOENIX_ACLS_ENABLED;
-import static org.apache.phoenix.query.QueryServices.PHOENIX_AUTOMATIC_GRANT_ENABLED;
 import static org.apache.phoenix.query.QueryServices.PHOENIX_QUERY_SERVER_CLUSTER_BASE_PATH;
 import static org.apache.phoenix.query.QueryServices.PHOENIX_QUERY_SERVER_LOADBALANCER_ENABLED;
 import static org.apache.phoenix.query.QueryServices.PHOENIX_QUERY_SERVER_SERVICE_NAME;
 import static org.apache.phoenix.query.QueryServices.PHOENIX_QUERY_SERVER_ZK_ACL_PASSWORD;
 import static org.apache.phoenix.query.QueryServices.PHOENIX_QUERY_SERVER_ZK_ACL_USERNAME;
-import static org.apache.phoenix.query.QueryServices.PHOENIX_SECURITY_PERMISSION_STRICT_MODE_ENABLED;
 import static org.apache.phoenix.query.QueryServices.QUEUE_SIZE_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.REGIONSERVER_INFO_PORT_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.RENEW_LEASE_ENABLED;
@@ -322,8 +320,6 @@ public class QueryServicesOptions {
     
     //Security defaults
     public static final boolean DEFAULT_PHOENIX_ACLS_ENABLED = false;
-    public static final boolean DEFAULT_PHOENIX_AUTOMATIC_GRANT_ENABLED = false;
-    public static final boolean DEFAULT_PHOENIX_SECURITY_PERMISSION_STRICT_MODE_ENABLED = true;
 
     //default update cache frequency
     public static final int DEFAULT_UPDATE_CACHE_FREQUENCY = 0;
@@ -423,9 +419,7 @@ public class QueryServicesOptions {
             .setIfUnset(STATS_COLLECTION_ENABLED, DEFAULT_STATS_COLLECTION_ENABLED)
             .setIfUnset(USE_STATS_FOR_PARALLELIZATION, DEFAULT_USE_STATS_FOR_PARALLELIZATION)
             .setIfUnset(UPLOAD_BINARY_DATA_TYPE_ENCODING, DEFAULT_UPLOAD_BINARY_DATA_TYPE_ENCODING)
-            .setIfUnset(PHOENIX_ACLS_ENABLED,  DEFAULT_PHOENIX_ACLS_ENABLED)
-            .setIfUnset(PHOENIX_AUTOMATIC_GRANT_ENABLED,  DEFAULT_PHOENIX_AUTOMATIC_GRANT_ENABLED)
-            .setIfUnset(PHOENIX_SECURITY_PERMISSION_STRICT_MODE_ENABLED,  DEFAULT_PHOENIX_SECURITY_PERMISSION_STRICT_MODE_ENABLED);
+            .setIfUnset(PHOENIX_ACLS_ENABLED,  DEFAULT_PHOENIX_ACLS_ENABLED);
         // HBase sets this to 1, so we reset it to something more appropriate.
         // Hopefully HBase will change this, because we can't know if a user set
         // it to 1, so we'll change it.

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bd72ce3d/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
index 338b325..1f76e90 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
@@ -113,6 +113,7 @@ import java.sql.SQLException;
 import java.sql.SQLFeatureNotSupportedException;
 import java.sql.Types;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.BitSet;
 import java.util.Collection;
 import java.util.Collections;
@@ -130,11 +131,16 @@ import java.util.Set;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HConnection;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.security.AccessDeniedException;
+import org.apache.hadoop.hbase.security.access.AccessControlClient;
+import org.apache.hadoop.hbase.security.access.Permission;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.phoenix.compile.ColumnResolver;
@@ -165,6 +171,7 @@ import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.jdbc.PhoenixStatement;
 import org.apache.phoenix.parse.AddColumnStatement;
 import org.apache.phoenix.parse.AlterIndexStatement;
+import org.apache.phoenix.parse.ChangePermsStatement;
 import org.apache.phoenix.parse.CloseStatement;
 import org.apache.phoenix.parse.ColumnDef;
 import org.apache.phoenix.parse.ColumnDefInPkConstraint;
@@ -229,6 +236,7 @@ import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.ReadOnlyProps;
 import org.apache.phoenix.util.ScanUtil;
 import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.ServerUtil;
 import org.apache.phoenix.util.StringUtil;
 import org.apache.phoenix.util.TransactionUtil;
 import org.apache.phoenix.util.UpgradeUtil;
@@ -4168,4 +4176,134 @@ public class MetaDataClient {
         }
         return new MutationState(0, 0, connection);
     }
+
+    /**
+     * GRANT/REVOKE statements use this method to update HBase acl's
+     * Perms can be changed at Schema, Table or User level
+     * @throws SQLException
+     */
+    public MutationState changePermissions(ChangePermsStatement changePermsStatement) throws SQLException {
+
+        logger.info(changePermsStatement.toString());
+
+        try(HBaseAdmin admin = connection.getQueryServices().getAdmin()) {
+            ClusterConnection clusterConnection = (ClusterConnection) admin.getConnection();
+
+            if (changePermsStatement.getSchemaName() != null) {
+                // SYSTEM.CATALOG doesn't have any entry for "default" HBase namespace, hence we will bypass the check
+                if(!changePermsStatement.getSchemaName().equals(QueryConstants.HBASE_DEFAULT_SCHEMA_NAME)) {
+                    FromCompiler.getResolverForSchema(changePermsStatement.getSchemaName(), connection);
+                }
+
+                changePermsOnSchema(clusterConnection, changePermsStatement);
+            } else if (changePermsStatement.getTableName() != null) {
+                PTable inputTable = PhoenixRuntime.getTable(connection,
+                        SchemaUtil.normalizeFullTableName(changePermsStatement.getTableName().toString()));
+                if (!(PTableType.TABLE.equals(inputTable.getType()) || PTableType.SYSTEM.equals(inputTable.getType()))) {
+                    throw new AccessDeniedException("Cannot GRANT or REVOKE permissions on INDEX TABLES or VIEWS");
+                }
+
+                // Changing perms on base table and update the perms for global and view indexes
+                // Views and local indexes are not physical tables and hence update perms is not needed
+                changePermsOnTables(clusterConnection, admin, changePermsStatement, inputTable);
+            } else {
+
+                // User can be given perms at the global level
+                changePermsOnUser(clusterConnection, changePermsStatement);
+            }
+
+        } catch (SQLException e) {
+            // Bubble up the SQL Exception
+            throw e;
+        } catch (Throwable throwable) {
+            // To change perms, the user must have ADMIN perms on that scope, otherwise it throws ADE
+            // Wrap around ADE and other exceptions to PhoenixIOException
+            throw ServerUtil.parseServerException(throwable);
+        }
+
+        return new MutationState(0, 0, connection);
+    }
+
+    private void changePermsOnSchema(ClusterConnection clusterConnection, ChangePermsStatement changePermsStatement) throws Throwable {
+        if(changePermsStatement.isGrantStatement()) {
+            AccessControlClient.grant(clusterConnection, changePermsStatement.getSchemaName(), changePermsStatement.getName(), changePermsStatement.getPermsList());
+        } else {
+            AccessControlClient.revoke(clusterConnection, changePermsStatement.getSchemaName(), changePermsStatement.getName(), Permission.Action.values());
+        }
+    }
+
+    private void changePermsOnTables(ClusterConnection clusterConnection, HBaseAdmin admin, ChangePermsStatement changePermsStatement, PTable inputTable) throws Throwable {
+
+        org.apache.hadoop.hbase.TableName tableName = SchemaUtil.getPhysicalTableName
+                (inputTable.getPhysicalName().getBytes(), inputTable.isNamespaceMapped());
+
+        changePermsOnTable(clusterConnection, changePermsStatement, tableName);
+
+        boolean schemaInconsistency = false;
+        List<PTable> inconsistentTables = null;
+
+        for(PTable indexTable : inputTable.getIndexes()) {
+            // Local Indexes don't correspond to new physical table, they are just stored in separate CF of base table.
+            if(indexTable.getIndexType().equals(IndexType.LOCAL)) {
+                continue;
+            }
+            if (inputTable.isNamespaceMapped() != indexTable.isNamespaceMapped()) {
+                schemaInconsistency = true;
+                if(inconsistentTables == null) {
+                    inconsistentTables = new ArrayList<>();
+                }
+                inconsistentTables.add(indexTable);
+                continue;
+            }
+            logger.info("Updating permissions for Index Table: " +
+                    indexTable.getName() + " Base Table: " + inputTable.getName());
+            tableName = SchemaUtil.getPhysicalTableName(indexTable.getPhysicalName().getBytes(), indexTable.isNamespaceMapped());
+            changePermsOnTable(clusterConnection, changePermsStatement, tableName);
+        }
+
+        if(schemaInconsistency) {
+            for(PTable table : inconsistentTables) {
+                logger.error("Fail to propagate permissions to Index Table: " + table.getName());
+            }
+            throw new TablesNotInSyncException(inputTable.getTableName().getString(),
+                    inconsistentTables.get(0).getTableName().getString(), "Namespace properties");
+        }
+
+        // There will be only a single View Index Table for all the indexes created on views
+        byte[] viewIndexTableBytes = MetaDataUtil.getViewIndexPhysicalName(inputTable.getPhysicalName().getBytes());
+        tableName = org.apache.hadoop.hbase.TableName.valueOf(viewIndexTableBytes);
+        boolean viewIndexTableExists = admin.tableExists(tableName);
+        if(viewIndexTableExists) {
+            logger.info("Updating permissions for View Index Table: " +
+                    Bytes.toString(viewIndexTableBytes) + " Base Table: " + inputTable.getName());
+            changePermsOnTable(clusterConnection, changePermsStatement, tableName);
+        } else {
+            if(inputTable.isMultiTenant()) {
+                logger.error("View Index Table not found for MultiTenant Table: " + inputTable.getName());
+                logger.error("Fail to propagate permissions to view Index Table: " + tableName.getNameAsString());
+                throw new TablesNotInSyncException(inputTable.getTableName().getString(),
+                        Bytes.toString(viewIndexTableBytes), " View Index table should exist for MultiTenant tables");
+            }
+        }
+    }
+
+    private void changePermsOnTable(ClusterConnection clusterConnection, ChangePermsStatement changePermsStatement, org.apache.hadoop.hbase.TableName tableName)
+            throws Throwable {
+        if(changePermsStatement.isGrantStatement()) {
+            AccessControlClient.grant(clusterConnection, tableName, changePermsStatement.getName(),
+                    null, null, changePermsStatement.getPermsList());
+        } else {
+            AccessControlClient.revoke(clusterConnection, tableName, changePermsStatement.getName(),
+                    null, null, Permission.Action.values());
+        }
+    }
+
+    private void changePermsOnUser(ClusterConnection clusterConnection, ChangePermsStatement changePermsStatement)
+            throws Throwable {
+        if(changePermsStatement.isGrantStatement()) {
+            AccessControlClient.grant(clusterConnection, changePermsStatement.getName(), changePermsStatement.getPermsList());
+        } else {
+            AccessControlClient.revoke(clusterConnection, changePermsStatement.getName(), Permission.Action.values());
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bd72ce3d/phoenix-core/src/main/java/org/apache/phoenix/schema/TablesNotInSyncException.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/TablesNotInSyncException.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/TablesNotInSyncException.java
new file mode 100644
index 0000000..e58df71
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/TablesNotInSyncException.java
@@ -0,0 +1,22 @@
+package org.apache.phoenix.schema;
+
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
+
+import java.sql.SQLException;
+
+/**
+ * Exception to raise when multiple tables differ in specified properties
+ * This can happen since Apache Phoenix code doesn't work atomically for many parts
+ * For example, Base table and index tables are inconsistent in namespace mapping
+ * OR View Index table doesn't exist for multi-tenant base table
+ */
+public class TablesNotInSyncException extends SQLException {
+    private static final long serialVersionUID = 1L;
+    private static SQLExceptionCode code = SQLExceptionCode.TABLES_NOT_IN_SYNC;
+
+    public TablesNotInSyncException(String table1, String table2, String diff) {
+        super(new SQLExceptionInfo.Builder(code).setMessage("Table: " + table1 + " and Table: " + table2 + " differ in " + diff).build().toString(), code.getSQLState(), code.getErrorCode());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bd72ce3d/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
index 47b4b43..5b5c3a5 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
@@ -58,6 +58,7 @@ import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
+import org.apache.phoenix.parse.LiteralParseNode;
 import org.apache.phoenix.query.KeyRange;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.query.QueryServices;
@@ -205,7 +206,25 @@ public class SchemaUtil {
         }
         return name.toUpperCase();
     }
-    
+
+    /**
+     * Normalize a Literal. If literal is surrounded by single quotes,
+     * the quotes are trimmed, else full string is returned
+     * @param literal the parsed LiteralParseNode
+     * @return the normalized literal string
+     */
+    public static String normalizeLiteral(LiteralParseNode literal) {
+        if (literal == null) {
+            return null;
+        }
+        String literalString = literal.toString();
+        if (isEnclosedInSingleQuotes(literalString)) {
+            // Trim the single quotes
+            return literalString.substring(1, literalString.length()-1);
+        }
+        return literalString;
+    }
+
     /**
      * Normalizes the fulltableName . Uses {@linkplain normalizeIdentifier}
      * @param fullTableName
@@ -221,6 +240,10 @@ public class SchemaUtil {
         return normalizedTableName + normalizeIdentifier(tableName);
     }
 
+    public static boolean isEnclosedInSingleQuotes(String name) {
+        return name!=null && name.length() > 0 && name.charAt(0)=='\'';
+    }
+
     public static boolean isCaseSensitive(String name) {
         return name!=null && name.length() > 0 && name.charAt(0)=='"';
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/bd72ce3d/phoenix-core/src/test/java/org/apache/phoenix/parse/QueryParserTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/parse/QueryParserTest.java b/phoenix-core/src/test/java/org/apache/phoenix/parse/QueryParserTest.java
index 431f60b..25f59c0 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/parse/QueryParserTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/parse/QueryParserTest.java
@@ -26,6 +26,8 @@ import java.io.IOException;
 import java.io.StringReader;
 import java.sql.SQLException;
 import java.sql.SQLFeatureNotSupportedException;
+import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.List;
 
 import org.apache.hadoop.hbase.util.Pair;
@@ -56,7 +58,7 @@ public class QueryParserTest {
         }
         assertEquals("Expected equality:\n" + sql + "\n" + newSQL, stmt, newStmt);
     }
-    
+
     private void parseQueryThatShouldFail(String sql) throws Exception {
         try {
             parseQuery(sql);
@@ -67,6 +69,48 @@ public class QueryParserTest {
     }
 
     @Test
+    public void testParseGrantQuery() throws Exception {
+
+        String sql0 = "GRANT 'RX' ON SYSTEM.\"SEQUENCE\" TO 'user'";
+        parseQuery(sql0);
+        String sql1 = "GRANT 'RWXCA' ON TABLE some_table0 TO 'user0'";
+        parseQuery(sql1);
+        String sql2 = "GRANT 'RWX' ON some_table1 TO 'user1'";
+        parseQuery(sql2);
+        String sql3 = "GRANT 'CA' ON SCHEMA some_schema2 TO 'user2'";
+        parseQuery(sql3);
+        String sql4 = "GRANT 'RXW' ON some_table3 TO GROUP 'group3'";
+        parseQuery(sql4);
+        String sql5 = "GRANT 'RXW' ON \"some_schema5\".\"some_table5\" TO GROUP 'group5'";
+        parseQuery(sql5);
+        String sql6 = "GRANT 'RWA' TO 'user6'";
+        parseQuery(sql6);
+        String sql7 = "GRANT 'A' TO GROUP 'group7'";
+        parseQuery(sql7);
+        String sql8 = "GRANT 'ARXRRRRR' TO GROUP 'group8'";
+        parseQueryThatShouldFail(sql8);
+    }
+
+    @Test
+    public void testParseRevokeQuery() throws Exception {
+
+        String sql0 = "REVOKE ON SCHEMA SYSTEM FROM 'user0'";
+        parseQuery(sql0);
+        String sql1 = "REVOKE ON SYSTEM.\"SEQUENCE\" FROM 'user1'";
+        parseQuery(sql1);
+        String sql2 = "REVOKE ON TABLE some_table2 FROM GROUP 'group2'";
+        parseQuery(sql2);
+        String sql3 = "REVOKE ON some_table3 FROM GROUP 'group2'";
+        parseQuery(sql3);
+        String sql4 = "REVOKE FROM 'user4'";
+        parseQuery(sql4);
+        String sql5 = "REVOKE FROM GROUP 'group5'";
+        parseQuery(sql5);
+        String sql6 = "REVOKE 'RRWWXAAA' FROM GROUP 'group6'";
+        parseQueryThatShouldFail(sql6);
+    }
+
+    @Test
     public void testParsePreQuery0() throws Exception {
         String sql = ((
             "select a from b\n" +


[11/16] phoenix git commit: PHOENIX-4322 DESC primary key column with variable length does not work in SkipScanFilter

Posted by ja...@apache.org.
PHOENIX-4322 DESC primary key column with variable length does not work in SkipScanFilter


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

Branch: refs/heads/4.x-HBase-1.2
Commit: 7e61234047b3107cd0b69bae860118c31e2e9927
Parents: 597764c
Author: maryannxue <ma...@gmail.com>
Authored: Sun Nov 5 02:37:55 2017 +0000
Committer: James Taylor <jt...@salesforce.com>
Committed: Sat Dec 16 16:42:54 2017 -0800

----------------------------------------------------------------------
 .../it/java/org/apache/phoenix/end2end/SortOrderIT.java  | 11 ++++++++++-
 .../expression/RowValueConstructorExpression.java        |  4 ++--
 2 files changed, 12 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/7e612340/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortOrderIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortOrderIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortOrderIT.java
index 655dbb1..3f749c1 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortOrderIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SortOrderIT.java
@@ -167,7 +167,16 @@ public class SortOrderIT extends ParallelStatsDisabledIT {
         runQueryTest(ddl, upsert("oid", "code"), insertedRows, new Object[][]{{"o2", 2}}, new WhereCondition("oid", "IN", "('o2')"),
             table);
     }
-    
+
+    @Test
+    public void inDescCompositePK3() throws Exception {
+        String table = generateUniqueName();
+        String ddl = "CREATE table " + table + " (oid VARCHAR NOT NULL, code VARCHAR NOT NULL constraint pk primary key (oid DESC, code DESC))";
+        Object[][] insertedRows = new Object[][]{{"o1", "1"}, {"o2", "2"}, {"o3", "3"}};
+        runQueryTest(ddl, upsert("oid", "code"), insertedRows, new Object[][]{{"o2", "2"}, {"o1", "1"}}, new WhereCondition("(oid, code)", "IN", "(('o2', '2'), ('o1', '1'))"),
+                table);
+    }
+
     @Test
     public void likeDescCompositePK1() throws Exception {
         String table = generateUniqueName();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7e612340/phoenix-core/src/main/java/org/apache/phoenix/expression/RowValueConstructorExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/RowValueConstructorExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/RowValueConstructorExpression.java
index 15f6e3e..9bb7234 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/RowValueConstructorExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/RowValueConstructorExpression.java
@@ -199,8 +199,8 @@ public class RowValueConstructorExpression extends BaseCompoundExpression {
                     // as otherwise we need it to ensure sort order is correct
                     for (int k = expressionCount -1 ; 
                             k >=0 &&  getChildren().get(k).getDataType() != null 
-                                  && !getChildren().get(k).getDataType().isFixedWidth() 
-                                  && outputBytes[outputSize-1] == QueryConstants.SEPARATOR_BYTE ; k--) {
+                                  && !getChildren().get(k).getDataType().isFixedWidth()
+                                  && outputBytes[outputSize-1] == SchemaUtil.getSeparatorByte(true, false, getChildren().get(k)) ; k--) {
                         outputSize--;
                     }
                     ptr.set(outputBytes, 0, outputSize);