You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by td...@apache.org on 2018/01/19 18:03:09 UTC

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

Repository: phoenix
Updated Branches:
  refs/heads/system-catalog ae359fec1 -> 15d4dace4


http://git-wip-us.apache.org/repos/asf/phoenix/blob/217867c7/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/217867c7/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/217867c7/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 cc2c6b3..3c69941 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;
@@ -160,12 +162,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())
@@ -209,55 +211,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/217867c7/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..f8f8501 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);
             }
@@ -2528,7 +2529,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;
             }
@@ -2545,8 +2547,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/217867c7/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/217867c7/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/217867c7/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/217867c7/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/217867c7/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());
+    }
+    
+
 }


[09/50] [abbrv] phoenix git commit: PHOENIX-4379 Upgrade code to create CHILD links should only create the links for views and not for indexes

Posted by td...@apache.org.
PHOENIX-4379 Upgrade code to create CHILD links should only create the links for views and not for indexes


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

Branch: refs/heads/system-catalog
Commit: 2053905683409225ffdc1c0ae4fc6c759604a80d
Parents: b2d5b4d
Author: Thomas D'Silva <td...@apache.org>
Authored: Tue Nov 14 15:28:01 2017 -0800
Committer: Thomas D'Silva <td...@apache.org>
Committed: Tue Nov 14 15:28:01 2017 -0800

----------------------------------------------------------------------
 .../org/apache/phoenix/end2end/UpgradeIT.java   | 73 ++++++++++++++++++++
 .../org/apache/phoenix/util/UpgradeUtil.java    |  1 +
 2 files changed, 74 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/20539056/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpgradeIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpgradeIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpgradeIT.java
index 4cb4642..b71dd7c 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpgradeIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpgradeIT.java
@@ -39,12 +39,14 @@ import java.sql.SQLException;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.Properties;
+import java.util.Set;
 import java.util.concurrent.Callable;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.FutureTask;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import org.apache.curator.shaded.com.google.common.collect.Sets;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.HTableInterface;
@@ -67,6 +69,7 @@ import org.apache.phoenix.schema.PMetaData;
 import org.apache.phoenix.schema.PName;
 import org.apache.phoenix.schema.PNameFactory;
 import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTable.LinkType;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.util.MetaDataUtil;
 import org.apache.phoenix.util.PhoenixRuntime;
@@ -838,4 +841,74 @@ public class UpgradeIT extends ParallelStatsDisabledIT {
         return DriverManager.getConnection(getUrl());
     }
     
+    @Test
+    public void testAddParentChildLinks() throws Exception {
+        String schema = "S_" + generateUniqueName();
+        String table1 = "T_" + generateUniqueName();
+        String table2 = "T_" + generateUniqueName();
+        String tableName = SchemaUtil.getTableName(schema, table1);
+        String multiTenantTableName = SchemaUtil.getTableName(schema, table2);
+        String viewName1 = "VIEW_" + generateUniqueName();
+        String viewIndexName1 = "VIDX_" + generateUniqueName();
+        String viewName2 = "VIEW_" + generateUniqueName();
+        String viewIndexName2 = "VIDX_" + generateUniqueName();
+        try (Connection conn = getConnection(false, null);
+                Connection tenantConn = getConnection(true, "tenant1");
+                Connection metaConn = getConnection(false, null)) {
+            // create a non multi-tenant and multi-tenant table
+            conn.createStatement()
+                    .execute("CREATE TABLE IF NOT EXISTS " + tableName + " ("
+                            + " TENANT_ID CHAR(15) NOT NULL, " + " PK1 integer NOT NULL, "
+                            + "PK2 bigint NOT NULL, " + "V1 VARCHAR, " + "V2 VARCHAR "
+                            + " CONSTRAINT NAME_PK PRIMARY KEY (TENANT_ID, PK1, PK2))");
+            conn.createStatement()
+                    .execute("CREATE TABLE IF NOT EXISTS " + multiTenantTableName + " ("
+                            + " TENANT_ID CHAR(15) NOT NULL, " + " PK1 integer NOT NULL, "
+                            + "PK2 bigint NOT NULL, " + "V1 VARCHAR, " + "V2 VARCHAR "
+                            + " CONSTRAINT NAME_PK PRIMARY KEY (TENANT_ID, PK1, PK2)"
+                            + " ) MULTI_TENANT= true");
+            // create tenant and global view
+            conn.createStatement().execute(
+                "CREATE VIEW " + viewName1 + " (col VARCHAR) AS SELECT * FROM " + tableName);
+            tenantConn.createStatement().execute("CREATE VIEW " + viewName2
+                    + "(col VARCHAR) AS SELECT * FROM " + multiTenantTableName);
+            // create index on the above views
+            conn.createStatement()
+                    .execute("create index " + viewIndexName1 + "  on " + viewName1 + "(col)");
+            tenantConn.createStatement()
+                    .execute("create index " + viewIndexName2 + " on " + viewName2 + "(col)");
+
+            // query all parent -> child links
+            Set<String> expectedChildLinkSet = getChildLinks(conn);
+
+            // delete all the child links
+            conn.createStatement().execute("DELETE FROM SYSTEM.CATALOG WHERE LINK_TYPE = "
+                    + LinkType.CHILD_TABLE.getSerializedValue());
+
+            // re-create them by running the upgrade code
+            PhoenixConnection phxMetaConn = metaConn.unwrap(PhoenixConnection.class);
+            phxMetaConn.setRunningUpgrade(true);
+            UpgradeUtil.addParentToChildLinks(phxMetaConn);
+            Set<String> actualChildLinkSet = getChildLinks(conn);
+
+            assertEquals("Unexpected child links", expectedChildLinkSet, actualChildLinkSet);
+        }
+    }
+
+    private Set<String> getChildLinks(Connection conn) throws SQLException {
+        ResultSet rs =
+                conn.createStatement().executeQuery(
+                    "SELECT TENANT_ID, TABLE_SCHEM, TABLE_NAME, COLUMN_NAME, COLUMN_FAMILY FROM SYSTEM.CATALOG WHERE LINK_TYPE = "
+                            + LinkType.CHILD_TABLE.getSerializedValue());
+        Set<String> childLinkSet = Sets.newHashSet();
+        while (rs.next()) {
+            String key =
+                    rs.getString("TENANT_ID") + " " + rs.getString("TABLE_SCHEM") + " "
+                            + rs.getString("TABLE_NAME") + " " + rs.getString("COLUMN_NAME") + " "
+                            + rs.getString("COLUMN_FAMILY");
+            childLinkSet.add(key);
+        }
+        return childLinkSet;
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/20539056/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java
index f5825b4..4488e86 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java
@@ -1110,6 +1110,7 @@ public class UpgradeUtil {
                                         "       4 AS LINK_TYPE " + 
                                         "FROM SYSTEM.CATALOG(PARENT_TENANT_ID VARCHAR)" + 
                                         "WHERE LINK_TYPE = 2 " +
+                                        "AND TABLE_TYPE IS NULL " +
                                         "AND (TENANT_ID, TABLE_SCHEM, TABLE_NAME) NOT IN (   " +
                                         "       SELECT TENANT_ID, " +
                                         "              TABLE_SCHEM, " +


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

Posted by td...@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/1c3387d0
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/1c3387d0
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/1c3387d0

Branch: refs/heads/system-catalog
Commit: 1c3387d0eb2fcec2423dda029aa65ca66f547416
Parents: ee728a4
Author: aertoria <ca...@gmail.com>
Authored: Mon Nov 27 11:58:01 2017 -0800
Committer: aertoria <ca...@gmail.com>
Committed: Thu Dec 7 11:51:17 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/1c3387d0/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/1c3387d0/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/1c3387d0/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/1c3387d0/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/1c3387d0/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/1c3387d0/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/1c3387d0/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/1c3387d0/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/1c3387d0/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/1c3387d0/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/1c3387d0/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/1c3387d0/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;
+        }
+    }
 }


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

Posted by td...@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/217867c7
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/217867c7
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/217867c7

Branch: refs/heads/system-catalog
Commit: 217867c78108b29d991794726c01c1eefb49b828
Parents: 4a1f0df
Author: Ankit Singhal <an...@gmail.com>
Authored: Thu Nov 9 11:27:11 2017 +0530
Committer: Ankit Singhal <an...@gmail.com>
Committed: Thu Nov 9 11:27:11 2017 +0530

----------------------------------------------------------------------
 .../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/217867c7/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/217867c7/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/217867c7/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/217867c7/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/217867c7/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/217867c7/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!!");


[41/50] [abbrv] phoenix git commit: PHOENIX-4488 Cache config parameters for MetaDataEndPointImpl during initialization

Posted by td...@apache.org.
PHOENIX-4488 Cache config parameters for MetaDataEndPointImpl during initialization


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

Branch: refs/heads/system-catalog
Commit: 1636f6182ae89e9c4b97c877aa919a6edac5bbc2
Parents: 3469384
Author: James Taylor <jt...@salesforce.com>
Authored: Fri Dec 22 11:36:44 2017 -0800
Committer: James Taylor <jt...@salesforce.com>
Committed: Tue Dec 26 13:21:09 2017 -0800

----------------------------------------------------------------------
 .../coprocessor/MetaDataEndpointImplTest.java   | 44 --------------------
 .../coprocessor/MetaDataEndpointImpl.java       | 30 ++++++-------
 2 files changed, 16 insertions(+), 58 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/1636f618/phoenix-core/src/it/java/org/apache/phoenix/coprocessor/MetaDataEndpointImplTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/coprocessor/MetaDataEndpointImplTest.java b/phoenix-core/src/it/java/org/apache/phoenix/coprocessor/MetaDataEndpointImplTest.java
deleted file mode 100644
index 2c558d8..0000000
--- a/phoenix-core/src/it/java/org/apache/phoenix/coprocessor/MetaDataEndpointImplTest.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * http://www.apache.org/licenses/LICENSE-2.0
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.phoenix.coprocessor;
-
-import com.google.common.collect.Lists;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.phoenix.query.QueryServices;
-import org.apache.phoenix.schema.PTable;
-import org.apache.phoenix.schema.PTableType;
-import org.junit.Test;
-
-import java.util.List;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-public class MetaDataEndpointImplTest {
-
-    @Test
-    public void testExceededIndexQuota() throws Exception {
-        PTable parentTable = mock(PTable.class);
-        List<PTable> indexes = Lists.newArrayList(mock(PTable.class), mock(PTable.class));
-        when(parentTable.getIndexes()).thenReturn(indexes);
-        Configuration configuration = new Configuration();
-        assertFalse(MetaDataEndpointImpl.execeededIndexQuota(PTableType.INDEX, parentTable, configuration));
-        configuration.setInt(QueryServices.MAX_INDEXES_PER_TABLE, 1);
-        assertTrue(MetaDataEndpointImpl.execeededIndexQuota(PTableType.INDEX, parentTable, configuration));
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/1636f618/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 afbd63f..7ad9028 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
@@ -242,7 +242,6 @@ import org.apache.phoenix.util.UpgradeUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.annotations.VisibleForTesting;
 import com.google.common.cache.Cache;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
@@ -472,6 +471,10 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
 
     private PhoenixMetaDataCoprocessorHost phoenixAccessCoprocessorHost;
     private boolean accessCheckEnabled;
+    private boolean blockWriteRebuildIndex;
+    private int maxIndexesPerTable;
+    private boolean isTablesMappingEnabled;
+
 
     /**
      * Stores a reference to the coprocessor environment provided by the
@@ -492,8 +495,16 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         }
         
         phoenixAccessCoprocessorHost = new PhoenixMetaDataCoprocessorHost(this.env);
-        this.accessCheckEnabled = env.getConfiguration().getBoolean(QueryServices.PHOENIX_ACLS_ENABLED,
+        Configuration config = env.getConfiguration();
+        this.accessCheckEnabled = config.getBoolean(QueryServices.PHOENIX_ACLS_ENABLED,
                 QueryServicesOptions.DEFAULT_PHOENIX_ACLS_ENABLED);
+        this.blockWriteRebuildIndex  = config.getBoolean(QueryServices.INDEX_FAILURE_BLOCK_WRITE,
+                QueryServicesOptions.DEFAULT_INDEX_FAILURE_BLOCK_WRITE);
+        this.maxIndexesPerTable = config.getInt(QueryServices.MAX_INDEXES_PER_TABLE,
+                    QueryServicesOptions.DEFAULT_MAX_INDEXES_PER_TABLE);
+        this.isTablesMappingEnabled = SchemaUtil.isNamespaceMappingEnabled(PTableType.TABLE,
+                new ReadOnlyProps(config.iterator()));
+
         logger.info("Starting Tracing-Metrics Systems");
         // Start the phoenix trace collection
         Tracing.addTraceMetricsSource();
@@ -583,8 +594,6 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             PTable oldTable = (PTable)metaDataCache.getIfPresent(cacheKey);
             long tableTimeStamp = oldTable == null ? MIN_TABLE_TIMESTAMP-1 : oldTable.getTimeStamp();
             PTable newTable;
-            boolean blockWriteRebuildIndex = env.getConfiguration().getBoolean(QueryServices.INDEX_FAILURE_BLOCK_WRITE, 
-                    QueryServicesOptions.DEFAULT_INDEX_FAILURE_BLOCK_WRITE);
             newTable = getTable(scanner, clientTimeStamp, tableTimeStamp, clientVersion);
             if (newTable == null) {
                 return null;
@@ -1551,7 +1560,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                             return;
                         }
                         // make sure we haven't gone over our threshold for indexes on this table.
-                        if (execeededIndexQuota(tableType, parentTable, env.getConfiguration())) {
+                        if (execeededIndexQuota(tableType, parentTable)) {
                             builder.setReturnCode(MetaDataProtos.MutationCode.TOO_MANY_INDEXES);
                             builder.setMutationTime(EnvironmentEdgeManager.currentTimeMillis());
                             done.run(builder.build());
@@ -1758,11 +1767,8 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         }
     }
 
-    @VisibleForTesting
-    static boolean execeededIndexQuota(PTableType tableType, PTable parentTable, Configuration configuration) {
-        return PTableType.INDEX == tableType && parentTable.getIndexes().size() >= configuration
-            .getInt(QueryServices.MAX_INDEXES_PER_TABLE,
-                QueryServicesOptions.DEFAULT_MAX_INDEXES_PER_TABLE);
+    private boolean execeededIndexQuota(PTableType tableType, PTable parentTable) {
+        return PTableType.INDEX == tableType && parentTable.getIndexes().size() >= maxIndexesPerTable;
     }
 
     private static final byte[] CHILD_TABLE_BYTES = new byte[] {PTable.LinkType.CHILD_TABLE.getSerializedValue()};
@@ -3265,8 +3271,6 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
          * from getting rebuilt too often.
          */
         final boolean wasLocked = (rowLock != null);
-        boolean blockWriteRebuildIndex = env.getConfiguration().getBoolean(QueryServices.INDEX_FAILURE_BLOCK_WRITE, 
-                QueryServicesOptions.DEFAULT_INDEX_FAILURE_BLOCK_WRITE);
         if (!wasLocked) {
             rowLock = acquireLock(region, key, null);
         }
@@ -3558,8 +3562,6 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
 
         GetVersionResponse.Builder builder = GetVersionResponse.newBuilder();
         Configuration config = env.getConfiguration();
-        boolean isTablesMappingEnabled = SchemaUtil.isNamespaceMappingEnabled(PTableType.TABLE,
-                new ReadOnlyProps(config.iterator()));
         if (isTablesMappingEnabled
                 && PhoenixDatabaseMetaData.MIN_NAMESPACE_MAPPED_PHOENIX_VERSION > request.getClientVersion()) {
             logger.error("Old client is not compatible when" + " system tables are upgraded to map to namespace");


[38/50] [abbrv] phoenix git commit: PHOENIX-4468 Looking up a parent index table of a child view from a different client fails (addendum)

Posted by td...@apache.org.
PHOENIX-4468 Looking up a parent index table of a child view from a different client fails (addendum)


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

Branch: refs/heads/system-catalog
Commit: 9355a4d262d31d8d65e1467bcc351bb99760e11d
Parents: c935f57
Author: Thomas D'Silva <td...@apache.org>
Authored: Wed Dec 20 15:47:19 2017 -0800
Committer: Thomas D'Silva <td...@apache.org>
Committed: Wed Dec 20 15:48:58 2017 -0800

----------------------------------------------------------------------
 phoenix-core/src/main/antlr3/PhoenixSQL.g                          | 1 -
 .../src/main/java/org/apache/phoenix/schema/PTableImpl.java        | 2 +-
 2 files changed, 1 insertion(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/9355a4d2/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 5e64984..87153cd 100644
--- a/phoenix-core/src/main/antlr3/PhoenixSQL.g
+++ b/phoenix-core/src/main/antlr3/PhoenixSQL.g
@@ -1114,7 +1114,6 @@ from_table_name returns [TableName ret]
 table_identifier returns [String ret]
     :   c=identifier {
            if (c.contains(QueryConstants.NAMESPACE_SEPARATOR) ) { throw new RuntimeException("Table or schema name cannot contain colon"); }
-           if (c.contains(QueryConstants.CHILD_VIEW_INDEX_NAME_SEPARATOR) ) { throw new RuntimeException("Table or schema name cannot contain hash"); }
            $ret = c;
     }
     ;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/9355a4d2/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
index 7df2458..9525127 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
@@ -267,7 +267,7 @@ public class PTableImpl implements PTable {
     
     public static PTableImpl makePTable(PTable table, PName tableName, long timeStamp, List<PName> physicalNames, List<PTable> indexes, String viewStatement, long updateCacheFrequency, PName tenantId) throws SQLException {
             return new PTableImpl(
-                    tenantId, PNameImpl.EMPTY_NAME, tableName, table.getType(), table.getIndexState(), timeStamp,
+                    tenantId, table.getSchemaName(), tableName, table.getType(), table.getIndexState(), timeStamp,
                     table.getSequenceNumber(), table.getPKName(), table.getBucketNum(), getColumnsToClone(table), table.getParentSchemaName(), table.getParentTableName(),
                     indexes, table.isImmutableRows(), physicalNames, table.getDefaultFamilyName(), viewStatement,
                     table.isWALDisabled(), table.isMultiTenant(), table.getStoreNulls(), table.getViewType(), table.getViewIndexId(), table.getIndexType(),


[11/50] [abbrv] phoenix git commit: PHOENIX-4361: Remove redundant argument in separateAndValidateProperties in CQSI

Posted by td...@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/ddd3ef28
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/ddd3ef28
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/ddd3ef28

Branch: refs/heads/system-catalog
Commit: ddd3ef28d36f963b295cef74db372f13040ea22a
Parents: ef3bce1
Author: Chinmay Kulkarni <ch...@gmail.com>
Authored: Wed Nov 15 18:31:20 2017 -0800
Committer: aertoria <ca...@gmail.com>
Committed: Fri Nov 17 11:26:00 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/ddd3ef28/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 f8f8501..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
@@ -1722,7 +1722,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) {
@@ -1940,7 +1940,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();


[49/50] [abbrv] phoenix git commit: fix test failure.

Posted by td...@apache.org.
fix test failure.


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

Branch: refs/heads/system-catalog
Commit: 41f81423322381fedd240302255f9cbb7d42d41d
Parents: 338c650
Author: Thomas D'Silva <td...@apache.org>
Authored: Tue Jan 2 13:04:19 2018 -0800
Committer: Thomas D'Silva <td...@apache.org>
Committed: Tue Jan 2 13:04:56 2018 -0800

----------------------------------------------------------------------
 .../java/org/apache/phoenix/end2end/BasePermissionsIT.java   | 4 ++--
 .../org/apache/phoenix/end2end/TableDDLPermissionsIT.java    | 8 ++++----
 2 files changed, 6 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/41f81423/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
index 9d7ef1b..9354b39 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BasePermissionsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BasePermissionsIT.java
@@ -642,12 +642,12 @@ public class BasePermissionsIT extends BaseTest {
         };
     }
 
-    AccessTestAction dropView(final String viewName) throws SQLException {
+    AccessTestAction dropView(final String viewName, final boolean cascade) 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));
+                    assertFalse(stmt.execute("DROP VIEW " + viewName + ( cascade ? " CASCADE" : "" ) ));
                 }
                 return null;
             }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/41f81423/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 8666bb8..1c0c83f 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
@@ -165,7 +165,7 @@ public class TableDDLPermissionsIT extends BasePermissionsIT {
             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(dropView(viewName1, true), AccessDeniedException.class, unprivilegedUser);
             
             verifyDenied(dropIndex(indexName1, phoenixTableName), AccessDeniedException.class, unprivilegedUser);
             verifyDenied(dropTable(phoenixTableName), AccessDeniedException.class, unprivilegedUser);
@@ -210,8 +210,8 @@ public class TableDDLPermissionsIT extends BasePermissionsIT {
             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(dropView(viewName1, true), regularUser1);
+            verifyAllowed(dropView(viewName2, false), regularUser1);
             verifyAllowed(dropColumn(phoenixTableName, "val1"), regularUser1);
             verifyAllowed(dropIndex(indexName2, phoenixTableName), regularUser1);
             verifyAllowed(dropIndex(indexName1, phoenixTableName), regularUser1);
@@ -222,7 +222,7 @@ public class TableDDLPermissionsIT extends BasePermissionsIT {
             verifyAllowed(createIndex(indexName1, phoenixTableName), superUser2);
             verifyAllowed(createView(viewName1, phoenixTableName), superUser2);
             verifyAllowed(readTable(phoenixTableName), superUser2);
-            verifyAllowed(dropView(viewName1), superUser2);
+            verifyAllowed(dropView(viewName1, false), superUser2);
             verifyAllowed(dropTable(phoenixTableName), superUser2);
 
         } finally {


[36/50] [abbrv] phoenix git commit: PHOENIX-4460 High GC / RS shutdown when we use select query with IN clause using 4.10 phoenix client on 4.13 phoenix server

Posted by td...@apache.org.
PHOENIX-4460 High GC / RS shutdown when we use select query with IN clause using 4.10 phoenix client on 4.13 phoenix server


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

Branch: refs/heads/system-catalog
Commit: 9d8be0e9214ba3680a81c399c5da316c1b91c99b
Parents: 5cb02da
Author: Thomas D'Silva <td...@apache.org>
Authored: Thu Dec 14 15:41:42 2017 -0800
Committer: Thomas D'Silva <td...@apache.org>
Committed: Mon Dec 18 20:28:15 2017 -0800

----------------------------------------------------------------------
 .../apache/phoenix/coprocessor/BaseScannerRegionObserver.java   | 5 ++++-
 1 file changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/9d8be0e9/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseScannerRegionObserver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseScannerRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseScannerRegionObserver.java
index 519e419..316a286 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseScannerRegionObserver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseScannerRegionObserver.java
@@ -366,7 +366,10 @@ abstract public class BaseScannerRegionObserver extends BaseRegionObserver {
       if (scan.isRaw() || ScanInfoUtil.isKeepDeletedCells(store.getScanInfo()) || scan.getTimeRange().getMax() == HConstants.LATEST_TIMESTAMP || TransactionUtil.isTransactionalTimestamp(scan.getTimeRange().getMax())) {
         return s;
       }
-
+      
+      if (s!=null) {
+          s.close();
+      }
       ScanInfo scanInfo = ScanInfoUtil.cloneScanInfoWithKeepDeletedCells(store.getScanInfo());
       return new StoreScanner(store, scanInfo, scan, targetCols,
           c.getEnvironment().getRegion().getReadpoint(scan.getIsolationLevel()));


[28/50] [abbrv] phoenix git commit: PHOENIX-4397 Incorrect query results when with stats are disabled on a salted table

Posted by td...@apache.org.
PHOENIX-4397 Incorrect query results when with stats are disabled on a salted table


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

Branch: refs/heads/system-catalog
Commit: 72bc8190272480fae17475398ef492a3071a3a44
Parents: d6e61af
Author: Samarth Jain <sa...@apache.org>
Authored: Fri Dec 8 14:40:22 2017 -0800
Committer: Samarth Jain <sa...@apache.org>
Committed: Fri Dec 8 14:40:22 2017 -0800

----------------------------------------------------------------------
 .../end2end/ExplainPlanWithStatsEnabledIT.java  | 36 ++++++++++++++++++++
 .../phoenix/iterate/BaseResultIterators.java    |  6 +---
 2 files changed, 37 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/72bc8190/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
index bfc6819..49efa97 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
@@ -1068,4 +1068,40 @@ public class ExplainPlanWithStatsEnabledIT extends ParallelStatsEnabledIT {
                     .getQueryPlan().getScans().get(0).size());
         }
     }
+
+    @Test
+    public void testQueryingWithUseStatsForParallelizationOnOff() throws SQLException {
+        testUseStatsForParallelizationOnSaltedTable(true, true);
+        testUseStatsForParallelizationOnSaltedTable(true, false);
+        testUseStatsForParallelizationOnSaltedTable(false, true);
+        testUseStatsForParallelizationOnSaltedTable(false, false);
+    }
+
+    private void testUseStatsForParallelizationOnSaltedTable(boolean useStatsFlag, boolean salted)
+            throws SQLException {
+        String tableName = generateUniqueName();
+        Connection conn = DriverManager.getConnection(getUrl());
+        conn.createStatement().execute(
+            "create table " + tableName + "(k varchar not null primary key, v varchar) "
+                    + (salted ? " SALT_BUCKETS=2," : "") + " USE_STATS_FOR_PARALLELIZATION="
+                    + useStatsFlag);
+        conn.createStatement().execute("upsert into " + tableName + " values ('1', 'B')");
+        conn.createStatement().execute("upsert into " + tableName + " values ('2', 'A')");
+        conn.commit();
+        String query = "SELECT V FROM " + tableName + " ORDER BY V";
+        ResultSet rs = conn.createStatement().executeQuery(query);
+        assertTrue(rs.next());
+        assertEquals("A", rs.getString(1));
+        assertTrue(rs.next());
+        assertEquals("B", rs.getString(1));
+
+        // Collect stats and make sure query still works correctly
+        conn.createStatement().execute("UPDATE STATISTICS " + tableName);
+        rs = conn.createStatement().executeQuery(query);
+        assertTrue(rs.next());
+        assertEquals("A", rs.getString(1));
+        assertTrue(rs.next());
+        assertEquals("B", rs.getString(1));
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/72bc8190/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java
index eb09813..bd67fa8 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java
@@ -607,11 +607,7 @@ public abstract class BaseResultIterators extends ExplainTable implements Result
                 estimate.bytesEstimate += gpsBytes;
             }
         }
-        if (startNewScan && !scans.isEmpty() && useStatsForParallelization) {
-            /*
-             * Note that even if region boundary was crossed, if we are not using stats for
-             * parallelization, nothing gets added to the parallel scans.
-             */
+        if (startNewScan && !scans.isEmpty()) {
             parallelScans.add(scans);
             scans = Lists.newArrayListWithExpectedSize(1);
         }


[47/50] [abbrv] phoenix git commit: Merge remote-tracking branch 'upstream/master' into PHOENIX-3534

Posted by td...@apache.org.
Merge remote-tracking branch 'upstream/master' into PHOENIX-3534


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

Branch: refs/heads/system-catalog
Commit: cb47eaecfd098e22c903ae8b3125b9e3f2508593
Parents: 7b4ccaf f714287
Author: Thomas D'Silva <td...@apache.org>
Authored: Thu Dec 28 23:30:35 2017 -0800
Committer: Thomas D'Silva <td...@apache.org>
Committed: Thu Dec 28 23:30:35 2017 -0800

----------------------------------------------------------------------
 NOTICE                                          |    2 +
 bin/argparse-1.4.0/argparse.py                  | 2392 ++++++++++++++++++
 bin/daemon.py                                   |   16 +-
 bin/queryserver.py                              |    2 +-
 bin/sqlline-thin.py                             |    9 +-
 bin/sqlline.py                                  |   14 +-
 phoenix-core/pom.xml                            |    1 +
 ...ReplayWithIndexWritesAndCompressedWALIT.java |    2 +
 .../apache/phoenix/end2end/AlterTableIT.java    |    2 +-
 .../phoenix/end2end/BasePermissionsIT.java      |  754 ++++++
 .../phoenix/end2end/ChangePermissionsIT.java    |  270 ++
 .../phoenix/end2end/CostBasedDecisionIT.java    |  466 ++++
 .../apache/phoenix/end2end/CreateSchemaIT.java  |   64 +-
 .../phoenix/end2end/DefaultColumnValueIT.java   |    9 +-
 .../end2end/ExplainPlanWithStatsEnabledIT.java  |   38 +-
 .../org/apache/phoenix/end2end/IndexToolIT.java |   40 +
 .../MigrateSystemTablesToSystemNamespaceIT.java |    8 +
 .../apache/phoenix/end2end/MutationStateIT.java |  161 ++
 .../apache/phoenix/end2end/PhoenixDriverIT.java |  160 ++
 .../org/apache/phoenix/end2end/QueryMoreIT.java |   42 -
 .../org/apache/phoenix/end2end/SequenceIT.java  |   29 -
 .../org/apache/phoenix/end2end/SortOrderIT.java |   11 +-
 .../apache/phoenix/end2end/StoreNullsIT.java    |    6 +-
 .../apache/phoenix/end2end/SystemCatalogIT.java |   96 +
 .../end2end/SystemTablePermissionsIT.java       |  226 +-
 .../phoenix/end2end/TableDDLPermissionsIT.java  |  583 +----
 .../org/apache/phoenix/end2end/UpgradeIT.java   |   73 +
 .../phoenix/end2end/UpsertBigValuesIT.java      |   64 +-
 .../phoenix/end2end/index/DropColumnIT.java     |   19 +-
 .../phoenix/end2end/index/IndexMetadataIT.java  |   55 +
 .../phoenix/end2end/join/HashJoinMoreIT.java    |    5 +
 .../apache/phoenix/execute/PartialCommitIT.java |    5 +-
 phoenix-core/src/main/antlr3/PhoenixSQL.g       |   37 +-
 .../phoenix/compile/BaseMutationPlan.java       |    5 +
 .../phoenix/compile/DelegateMutationPlan.java   |    5 +
 .../apache/phoenix/compile/DeleteCompiler.java  |  558 ++--
 .../apache/phoenix/compile/JoinCompiler.java    |   19 +-
 .../phoenix/compile/ListJarsQueryPlan.java      |    6 +
 .../apache/phoenix/compile/MutationPlan.java    |    5 +-
 .../phoenix/compile/ProjectionCompiler.java     |    6 +-
 .../apache/phoenix/compile/QueryCompiler.java   |    6 +-
 .../org/apache/phoenix/compile/QueryPlan.java   |    5 +-
 .../org/apache/phoenix/compile/ScanRanges.java  |   11 +-
 .../apache/phoenix/compile/TraceQueryPlan.java  |    6 +
 .../apache/phoenix/compile/UpsertCompiler.java  |  687 ++---
 .../apache/phoenix/compile/WhereOptimizer.java  |    5 -
 .../coprocessor/BaseScannerRegionObserver.java  |    5 +-
 .../coprocessor/MetaDataEndpointImpl.java       |   30 +-
 .../coprocessor/PhoenixAccessController.java    |   29 +-
 .../phoenix/exception/SQLExceptionCode.java     |    1 +
 .../apache/phoenix/execute/AggregatePlan.java   |   30 +-
 .../apache/phoenix/execute/BaseQueryPlan.java   |   38 +-
 .../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    |   88 +-
 .../execute/LiteralResultIterationPlan.java     |    6 +
 .../apache/phoenix/execute/MutationState.java   |  154 +-
 .../org/apache/phoenix/execute/ScanPlan.java    |   25 +
 .../phoenix/execute/SortMergeJoinPlan.java      |   81 +-
 .../org/apache/phoenix/execute/UnionPlan.java   |   63 +-
 .../RowValueConstructorExpression.java          |    4 +-
 .../expression/SingleCellColumnExpression.java  |   35 +-
 .../phoenix/hbase/index/write/IndexWriter.java  |    3 +-
 .../apache/phoenix/index/IndexMaintainer.java   |    2 +-
 .../phoenix/index/PhoenixIndexBuilder.java      |    5 -
 .../phoenix/iterate/BaseResultIterators.java    |    6 +-
 .../NonAggregateRegionScannerFactory.java       |    2 +-
 .../apache/phoenix/jdbc/PhoenixStatement.java   |   74 +-
 .../phoenix/mapreduce/PhoenixOutputFormat.java  |   13 +-
 .../phoenix/mapreduce/PhoenixRecordWriter.java  |    8 +-
 .../phoenix/mapreduce/util/ConnectionUtil.java  |   23 +-
 .../java/org/apache/phoenix/optimize/Cost.java  |  123 +
 .../apache/phoenix/optimize/QueryOptimizer.java |   30 +-
 .../phoenix/parse/AddColumnStatement.java       |    2 +-
 .../phoenix/parse/AlterIndexStatement.java      |   14 +
 .../phoenix/parse/ChangePermsStatement.java     |  102 +
 .../phoenix/parse/CreateSchemaStatement.java    |    2 +-
 .../apache/phoenix/parse/ParseNodeFactory.java  |   13 +-
 .../phoenix/query/ConnectionQueryServices.java  |    2 +
 .../query/ConnectionQueryServicesImpl.java      |   78 +-
 .../query/ConnectionlessQueryServicesImpl.java  |    7 +
 .../query/DelegateConnectionQueryServices.java  |    8 +-
 .../org/apache/phoenix/query/QueryServices.java |    5 +-
 .../phoenix/query/QueryServicesOptions.java     |   12 +-
 .../org/apache/phoenix/schema/ColumnRef.java    |    4 +-
 .../apache/phoenix/schema/MetaDataClient.java   |  714 ++++--
 .../phoenix/schema/MetaDataSplitPolicy.java     |   23 +-
 .../java/org/apache/phoenix/schema/PTable.java  |   21 +-
 .../schema/TablesNotInSyncException.java        |   22 +
 .../phoenix/schema/types/PArrayDataType.java    |   18 +-
 .../schema/types/PArrayDataTypeDecoder.java     |   79 +-
 .../schema/types/PArrayDataTypeEncoder.java     |   10 +-
 .../java/org/apache/phoenix/util/CostUtil.java  |   90 +
 .../java/org/apache/phoenix/util/IndexUtil.java |    7 +-
 .../org/apache/phoenix/util/KeyValueUtil.java   |   55 +-
 .../org/apache/phoenix/util/PhoenixRuntime.java |   32 +-
 .../org/apache/phoenix/util/PropertiesUtil.java |    9 +-
 .../org/apache/phoenix/util/SchemaUtil.java     |   30 +-
 .../org/apache/phoenix/util/UpgradeUtil.java    |    1 +
 .../apache/phoenix/parse/QueryParserTest.java   |   59 +-
 .../query/ParallelIteratorsSplitTest.java       |    6 +
 .../schema/ImmutableStorageSchemeTest.java      |  241 +-
 .../java/org/apache/phoenix/pig/BasePigIT.java  |    4 +
 .../apache/phoenix/pig/PhoenixHBaseStorage.java |   12 +-
 phoenix-queryserver-client/pom.xml              |   16 +-
 phoenix-server/pom.xml                          |    3 +
 pom.xml                                         |    3 +-
 109 files changed, 7730 insertions(+), 1921 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/cb47eaec/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/cb47eaec/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/cb47eaec/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/cb47eaec/phoenix-core/src/it/java/org/apache/phoenix/end2end/TableDDLPermissionsIT.java
----------------------------------------------------------------------
diff --cc phoenix-core/src/it/java/org/apache/phoenix/end2end/TableDDLPermissionsIT.java
index 0130f35,8666bb8..dc55400
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TableDDLPermissionsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TableDDLPermissionsIT.java
@@@ -353,23 -201,21 +201,23 @@@ public class TableDDLPermissionsIT exte
              verifyAllowed(readTable(phoenixTableName, indexName1), unprivilegedUser);
              verifyAllowed(readTable(phoenixTableName, indexName2), unprivilegedUser);
              verifyAllowed(rebuildIndex(indexName2, phoenixTableName), unprivilegedUser);
 +            verifyAllowed(dropView(viewName3), regularUser);
 +            verifyAllowed(dropView(viewName4), regularUser);
  
              // 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);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/cb47eaec/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpgradeIT.java
----------------------------------------------------------------------
diff --cc phoenix-core/src/it/java/org/apache/phoenix/end2end/UpgradeIT.java
index bf5bd94,b71dd7c..ac0cbd3
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpgradeIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpgradeIT.java
@@@ -33,7 -36,10 +33,8 @@@ import java.sql.DriverManager
  import java.sql.PreparedStatement;
  import java.sql.ResultSet;
  import java.sql.SQLException;
 -import java.util.Arrays;
 -import java.util.Collections;
  import java.util.Properties;
+ import java.util.Set;
  import java.util.concurrent.Callable;
  import java.util.concurrent.CountDownLatch;
  import java.util.concurrent.FutureTask;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/cb47eaec/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/DropColumnIT.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/cb47eaec/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexMetadataIT.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/cb47eaec/phoenix-core/src/it/java/org/apache/phoenix/execute/PartialCommitIT.java
----------------------------------------------------------------------
diff --cc phoenix-core/src/it/java/org/apache/phoenix/execute/PartialCommitIT.java
index 2199b93,e5b57e3..938ffe6
--- a/phoenix-core/src/it/java/org/apache/phoenix/execute/PartialCommitIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/execute/PartialCommitIT.java
@@@ -285,9 -284,9 +284,9 @@@ public class PartialCommitIT extends Ba
      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) {
 +        return new PhoenixConnection(phxCon, (MutationState)null) {
              @Override
              protected MutationState newMutationState(int maxSize, int maxSizeBytes) {
                  return new MutationState(maxSize, maxSizeBytes, this, mutations, false, null);

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

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

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

http://git-wip-us.apache.org/repos/asf/phoenix/blob/cb47eaec/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
----------------------------------------------------------------------
diff --cc phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
index cb08b2f,47ad7cf..6277b78
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
@@@ -238,9 -242,7 +238,8 @@@ import org.apache.phoenix.util.UpgradeU
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
- import com.google.common.annotations.VisibleForTesting;
  import com.google.common.cache.Cache;
 +import com.google.common.collect.Iterables;
  import com.google.common.collect.Lists;
  import com.google.common.collect.Maps;
  import com.google.protobuf.ByteString;
@@@ -2045,30 -1767,93 +2054,27 @@@ public class MetaDataEndpointImpl exten
          }
      }
  
-     @VisibleForTesting
-     static boolean execeededIndexQuota(PTableType tableType, PTable parentTable, Configuration configuration) {
-         return PTableType.INDEX == tableType && parentTable.getIndexes().size() >= configuration
-             .getInt(QueryServices.MAX_INDEXES_PER_TABLE,
-                 QueryServicesOptions.DEFAULT_MAX_INDEXES_PER_TABLE);
+     private boolean execeededIndexQuota(PTableType tableType, PTable parentTable) {
+         return PTableType.INDEX == tableType && parentTable.getIndexes().size() >= maxIndexesPerTable;
      }
  
 -    private static final byte[] CHILD_TABLE_BYTES = new byte[] {PTable.LinkType.CHILD_TABLE.getSerializedValue()};
 -
 -    
 -    private void findAllChildViews(Region region, byte[] tenantId, PTable table,
 -            TableViewFinder result, long clientTimeStamp, int clientVersion) throws IOException, SQLException {
 -        TableViewFinder currResult = findChildViews(region, tenantId, table, clientVersion);
 -        result.addResult(currResult);
 -        for (ViewInfo viewInfo : currResult.getViewInfoList()) {
 -            byte[] viewtenantId = viewInfo.getTenantId();
 -            byte[] viewSchema = viewInfo.getSchemaName();
 -            byte[] viewTable = viewInfo.getViewName();
 -            byte[] tableKey = SchemaUtil.getTableKey(viewtenantId, viewSchema, viewTable);
 -            ImmutableBytesPtr cacheKey = new ImmutableBytesPtr(tableKey);
 -            PTable view = loadTable(env, tableKey, cacheKey, clientTimeStamp, clientTimeStamp, clientVersion);
 -            findAllChildViews(region, viewtenantId, view, result, clientTimeStamp, clientVersion);
 -        }
 -    }
 -        
 -    // TODO remove this in 4.13 release 
 -    @Deprecated
 -    private TableViewFinder findChildViews_deprecated(Region region, byte[] tenantId, PTable table, byte[] linkTypeBytes) throws IOException {
 -        byte[] schemaName = table.getSchemaName().getBytes();
 -        byte[] tableName = table.getTableName().getBytes();
 -        boolean isMultiTenant = table.isMultiTenant();
 -        Scan scan = new Scan();
 -        // If the table is multi-tenant, we need to check across all tenant_ids,
 -        // so we can't constrain the row key. Otherwise, any views would have
 -        // the same tenantId.
 -        if (!isMultiTenant) {
 -            byte[] startRow = ByteUtil.concat(tenantId, QueryConstants.SEPARATOR_BYTE_ARRAY);
 -            byte[] stopRow = ByteUtil.nextKey(startRow);
 -            scan.setStartRow(startRow);
 -            scan.setStopRow(stopRow);
 -        }
 -        SingleColumnValueFilter linkFilter = new SingleColumnValueFilter(TABLE_FAMILY_BYTES, LINK_TYPE_BYTES, CompareOp.EQUAL, linkTypeBytes);
 -        SingleColumnValueFilter tableTypeFilter = new SingleColumnValueFilter(TABLE_FAMILY_BYTES, TABLE_TYPE_BYTES,
 -                CompareOp.EQUAL, PTableType.VIEW.getSerializedValue().getBytes());
 -        tableTypeFilter.setFilterIfMissing(false);
 -        linkFilter.setFilterIfMissing(true);
 -        byte[] suffix = ByteUtil.concat(QueryConstants.SEPARATOR_BYTE_ARRAY, SchemaUtil
 -                .getPhysicalHBaseTableName(schemaName, tableName, table.isNamespaceMapped())
 -                .getBytes());
 -        SuffixFilter rowFilter = new SuffixFilter(suffix);
 -        FilterList filter = new FilterList(linkFilter,tableTypeFilter,rowFilter);
 -        scan.setFilter(filter);
 -        scan.addColumn(TABLE_FAMILY_BYTES, LINK_TYPE_BYTES);
 -        scan.addColumn(TABLE_FAMILY_BYTES, TABLE_TYPE_BYTES);
 -        scan.addColumn(TABLE_FAMILY_BYTES, TABLE_SEQ_NUM_BYTES);
 -        
 -        // Original region-only scanner modified due to PHOENIX-1208
 -        // RegionScanner scanner = region.getScanner(scan);
 -        // The following *should* work, but doesn't due to HBASE-11837
 -        // TableName systemCatalogTableName = region.getTableDesc().getTableName();
 -        // HTableInterface hTable = env.getTable(systemCatalogTableName);
 -        // These deprecated calls work around the issue
 -        try (HTableInterface hTable = ServerUtil.getHTableForCoprocessorScan(env,
 -            region.getTableDesc().getTableName().getName())) {
 -            boolean allViewsInCurrentRegion = true;
 -            int numOfChildViews = 0;
 -            List<ViewInfo> viewInfoList = Lists.newArrayList();
 -            try (ResultScanner scanner = hTable.getScanner(scan)) {
 -                for (Result result = scanner.next(); (result != null); result = scanner.next()) {
 -                    numOfChildViews++;
 -                    ImmutableBytesWritable ptr = new ImmutableBytesWritable();
 -                    ResultTuple resultTuple = new ResultTuple(result);
 -                    resultTuple.getKey(ptr);
 -                    byte[] key = ptr.copyBytes();
 -                    if (checkTableKeyInRegion(key, region) != null) {
 -                        allViewsInCurrentRegion = false;
 -                    }
 -                    byte[][] rowKeyMetaData = new byte[3][];
 -                    getVarChars(result.getRow(), 3, rowKeyMetaData);
 -                    byte[] viewTenantId = rowKeyMetaData[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
 -                    byte[] viewSchemaName = rowKeyMetaData[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX];
 -                    byte[] viewName = rowKeyMetaData[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
 -                    viewInfoList.add(new ViewInfo(viewTenantId, viewSchemaName, viewName));
 -                }
 -                TableViewFinder tableViewFinderResult = new TableViewFinder(viewInfoList);
 -                if (numOfChildViews > 0 && !allViewsInCurrentRegion) {
 -                    tableViewFinderResult.setAllViewsNotInSingleRegion();
 -                }
 -                return tableViewFinderResult;
 -            }
 +    private void findAncestorViewsOfIndex(byte[] tenantId, byte[] schemaName, byte[] indexName, TableViewFinderResult result) throws IOException {
 +        HTableInterface hTable = env.getTable(SchemaUtil
 +                .getPhysicalTableName(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES, env.getConfiguration()));
 +        try {
 +            TableViewFinderResult currentResult = ViewFinder.findParentViewofIndex(hTable, tenantId, schemaName, indexName);
 +//            currentResult.addResult(ViewFinder.findBaseTable(hTable, tenantId, schemaName, indexName));
 +//            if ( currentResult.getResults().size()!=1 ) {
 +//                throw new RuntimeException("View index should have exactly one parent");
 +//            }
 +            if (currentResult.getResults().size()==1) {
 +            	result.addResult(currentResult);
 +            	TableInfo tableInfo = currentResult.getResults().get(0);
 +            	findAncestorViews(tableInfo.getTenantId(), tableInfo.getSchemaName(), tableInfo.getTableName(), result);
 +            }
 +            // else this is an index on a regular table and so we don't need to combine columns
 +        } finally {
 +            hTable.close();
          }
      }
      

http://git-wip-us.apache.org/repos/asf/phoenix/blob/cb47eaec/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
----------------------------------------------------------------------

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

http://git-wip-us.apache.org/repos/asf/phoenix/blob/cb47eaec/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/cb47eaec/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/cb47eaec/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/cb47eaec/phoenix-core/src/main/java/org/apache/phoenix/schema/PTable.java
----------------------------------------------------------------------

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

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

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


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

Posted by td...@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/355ee522
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/355ee522
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/355ee522

Branch: refs/heads/system-catalog
Commit: 355ee522c1d4ff07cf9fbb0a9a01e43e3f702730
Parents: d46d4e5
Author: Thomas D'Silva <td...@apache.org>
Authored: Tue Nov 28 18:37:55 2017 -0800
Committer: Thomas D'Silva <td...@apache.org>
Committed: Tue Nov 28 18:37:55 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/355ee522/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/355ee522/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/355ee522/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/355ee522/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 7462baa..b5a55b8 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/355ee522/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/355ee522/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);


[25/50] [abbrv] phoenix git commit: PHOENIX-4439 QueryServer pid file name doesn't comply the usual schema we are using in hadoop ecosystem

Posted by td...@apache.org.
PHOENIX-4439 QueryServer pid file name doesn't comply the usual schema we are using in hadoop ecosystem


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

Branch: refs/heads/system-catalog
Commit: ee728a4d19c004ad456b24cd228fb2351362472d
Parents: 25359a9
Author: Sergey Soldatov <ss...@apache.org>
Authored: Wed Dec 6 11:23:48 2017 -0800
Committer: Sergey Soldatov <ss...@apache.org>
Committed: Wed Dec 6 14:03:26 2017 -0800

----------------------------------------------------------------------
 bin/queryserver.py | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/ee728a4d/bin/queryserver.py
----------------------------------------------------------------------
diff --git a/bin/queryserver.py b/bin/queryserver.py
index fefe0a5..711bcc4 100755
--- a/bin/queryserver.py
+++ b/bin/queryserver.py
@@ -76,7 +76,7 @@ hadoop_config_path = phoenix_utils.hadoop_conf
 hadoop_classpath = phoenix_utils.hadoop_classpath
 
 # TODO: add windows support
-phoenix_file_basename = '%s-queryserver' % getpass.getuser()
+phoenix_file_basename = 'phoenix-%s-queryserver' % getpass.getuser()
 phoenix_log_file = '%s.log' % phoenix_file_basename
 phoenix_out_file = '%s.out' % phoenix_file_basename
 phoenix_pid_file = '%s.pid' % phoenix_file_basename


[10/50] [abbrv] phoenix git commit: PHOENIX-4381 Calculate the estimatedSize of MutationState incrementally

Posted by td...@apache.org.
PHOENIX-4381 Calculate the estimatedSize of MutationState incrementally


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

Branch: refs/heads/system-catalog
Commit: ef3bce18fe7373b66136d933cc364001dff2c3f8
Parents: 2053905
Author: Thomas D'Silva <td...@apache.org>
Authored: Wed Nov 15 18:54:04 2017 -0800
Committer: Thomas D'Silva <td...@apache.org>
Committed: Wed Nov 15 20:59:53 2017 -0800

----------------------------------------------------------------------
 .../apache/phoenix/execute/MutationState.java   | 15 ++++-
 .../org/apache/phoenix/util/KeyValueUtil.java   | 65 +++++++++-----------
 2 files changed, 43 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/ef3bce18/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 1f47a33..0cdb010 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
@@ -127,6 +127,7 @@ public class MutationState implements SQLCloseable {
 
     private long sizeOffset;
     private int numRows = 0;
+    private long estimatedSize = 0;
     private int[] uncommittedStatementIndexes = EMPTY_STATEMENT_INDEX_ARRAY;
     private boolean isExternalTxContext = false;
     private Map<TableRef, Map<ImmutableBytesPtr,RowMutationState>> txMutations = Collections.emptyMap();
@@ -193,6 +194,7 @@ public class MutationState implements SQLCloseable {
             this.mutations.put(table, mutations);
         }
         this.numRows = mutations.size();
+        this.estimatedSize = KeyValueUtil.getEstimatedRowSize(table, mutations);
         throwIfTooBig();
     }
 
@@ -354,7 +356,6 @@ public class MutationState implements SQLCloseable {
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.MAX_MUTATION_SIZE_EXCEEDED).build()
                     .buildException();
         }
-        long estimatedSize = KeyValueUtil.getEstimatedRowSize(mutations);
         if (estimatedSize > maxSizeBytes) {
             resetState();
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.MAX_MUTATION_SIZE_BYTES_EXCEEDED)
@@ -433,7 +434,12 @@ public class MutationState implements SQLCloseable {
         phoenixTransactionContext.join(newMutationState.getPhoenixTransactionContext());
 
         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 (!newMutationState.txMutations.isEmpty()) {
             if (txMutations.isEmpty()) {
                 txMutations = Maps.newHashMapWithExpectedSize(mutations.size());
@@ -968,6 +974,8 @@ 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();
@@ -1004,8 +1012,8 @@ public class MutationState implements SQLCloseable {
                         for (List<Mutation> mutationBatch : mutationBatchList) {
                             hTable.batch(mutationBatch);
                             batchCount++;
+                            if (logger.isDebugEnabled()) logger.debug("Sent batch of " + mutationBatch.size() + " for " + Bytes.toString(htableName));
                         }
-                        if (logger.isDebugEnabled()) logger.debug("Sent batch of " + numMutations + " for " + Bytes.toString(htableName));
                         child.stop();
                         child.stop();
                         shouldRetry = false;
@@ -1015,6 +1023,8 @@ public class MutationState implements SQLCloseable {
                         
                         if (tableInfo.isDataTable()) {
                             numRows -= numMutations;
+                            // decrement estimated size by the fraction of rows we sent to hbase
+                            estimatedSize -= ((double)numMutations/startNumRows)*startEstimatedSize;
                         }
                         // Remove batches as we process them
                         mutations.remove(origTableRef);
@@ -1180,6 +1190,7 @@ public class MutationState implements SQLCloseable {
 
     private void resetState() {
         numRows = 0;
+        estimatedSize = 0;
         this.mutations.clear();
         resetTransactionalState();
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ef3bce18/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 4234df5..2dfe1b9 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
@@ -192,46 +192,41 @@ public class KeyValueUtil {
      * @return estimated row size
      */
     public static long
-            getEstimatedRowSize(Map<TableRef, Map<ImmutableBytesPtr, RowMutationState>> mutations) {
+            getEstimatedRowSize(TableRef tableRef, Map<ImmutableBytesPtr, RowMutationState> mutations) {
         long size = 0;
-        // iterate over tables
-        for (Entry<TableRef, Map<ImmutableBytesPtr, RowMutationState>> tableEntry : mutations
-                .entrySet()) {
-            PTable table = tableEntry.getKey().getTable();
-            // iterate over rows
-            for (Entry<ImmutableBytesPtr, RowMutationState> rowEntry : tableEntry.getValue()
-                    .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
+        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 +=
-                            PArrayDataTypeEncoder.getEstimatedByteSize(table, rowLength,
-                                colValueMap);
-                    break;
+                            KeyValue.getKeyValueDataStructureSize(rowLength,
+                                pColumn.getFamilyName().getBytes().length,
+                                pColumn.getColumnQualifierBytes().length,
+                                colValueEntry.getValue().length);
                 }
-                // count the empty key value
-                Pair<byte[], byte[]> emptyKeyValueInfo =
-                        EncodedColumnsUtil.getEmptyKeyValueInfo(table);
+                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 +=
-                        KeyValue.getKeyValueDataStructureSize(rowLength,
-                            SchemaUtil.getEmptyColumnFamilyPtr(table).getLength(),
-                            emptyKeyValueInfo.getFirst().length,
-                            emptyKeyValueInfo.getSecond().length);
+                        PArrayDataTypeEncoder.getEstimatedByteSize(table, rowLength,
+                            colValueMap);
+                break;
             }
+            // 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;
     }


[45/50] [abbrv] phoenix git commit: PHOENIX-4473 Exception when Adding new columns to base table and view diverge (addendum)

Posted by td...@apache.org.
PHOENIX-4473 Exception when Adding new columns to base table and view diverge (addendum)


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

Branch: refs/heads/system-catalog
Commit: f7142879f33cae236e0530a8ed4eeaad1542d66a
Parents: ae21f87
Author: Thomas D'Silva <td...@apache.org>
Authored: Thu Dec 28 15:34:22 2017 -0800
Committer: Thomas D'Silva <td...@apache.org>
Committed: Thu Dec 28 15:35:53 2017 -0800

----------------------------------------------------------------------
 .../java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/f7142879/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 18b1ee5..47ad7cf 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
@@ -3077,7 +3077,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                         PhoenixDatabaseMetaData.ORDINAL_POSITION_BYTES, ordinalPositionBytes);
             
             // New PK columns have to be nullable after the first DDL
-            byte[] isNullableBytes = PBoolean.INSTANCE.toBytes(ResultSetMetaData.columnNullable);
+            byte[] isNullableBytes = PInteger.INSTANCE.toBytes(ResultSetMetaData.columnNullable);
             indexColumnDefinitionPut.add(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES,
                         PhoenixDatabaseMetaData.NULLABLE_BYTES, isNullableBytes);
             


[44/50] [abbrv] phoenix git commit: PHOENIX-4473 Exception when Adding new columns to base table and view diverge (Ankit Singhal)

Posted by td...@apache.org.
PHOENIX-4473 Exception when Adding new columns to base table and view diverge (Ankit Singhal)


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

Branch: refs/heads/system-catalog
Commit: ae21f87338de5c80efd8e89b256415faad0e00a3
Parents: edc9d12
Author: Thomas D'Silva <td...@apache.org>
Authored: Thu Dec 28 11:22:26 2017 -0800
Committer: Thomas D'Silva <td...@apache.org>
Committed: Thu Dec 28 11:22:53 2017 -0800

----------------------------------------------------------------------
 .../java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/ae21f873/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 7ad9028..18b1ee5 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
@@ -3077,7 +3077,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                         PhoenixDatabaseMetaData.ORDINAL_POSITION_BYTES, ordinalPositionBytes);
             
             // New PK columns have to be nullable after the first DDL
-            byte[] isNullableBytes = PBoolean.INSTANCE.toBytes(true);
+            byte[] isNullableBytes = PBoolean.INSTANCE.toBytes(ResultSetMetaData.columnNullable);
             indexColumnDefinitionPut.add(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES,
                         PhoenixDatabaseMetaData.NULLABLE_BYTES, isNullableBytes);
             


[46/50] [abbrv] phoenix git commit: use htable to process mutations that aren't on the current region server

Posted by td...@apache.org.
use htable to process mutations that aren't on the current region server


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

Branch: refs/heads/system-catalog
Commit: 7b4ccaf7798956a75c0ff4df542ded5259172690
Parents: 39c0d1d
Author: Thomas D'Silva <td...@apache.org>
Authored: Thu Dec 28 20:12:29 2017 -0800
Committer: Thomas D'Silva <td...@apache.org>
Committed: Thu Dec 28 20:12:29 2017 -0800

----------------------------------------------------------------------
 .../end2end/ExplainPlanWithStatsEnabledIT.java  |   2 +-
 .../phoenix/end2end/TableDDLPermissionsIT.java  |   2 +-
 .../coprocessor/MetaDataEndpointImpl.java       | 208 +++++++++++++------
 .../apache/phoenix/coprocessor/ViewFinder.java  |   9 +-
 .../coprocessor/MetaDataEndpointImplTest.java   |   4 +-
 5 files changed, 155 insertions(+), 70 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/7b4ccaf7/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
index bfc6819..bcf8cd8 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
@@ -604,7 +604,7 @@ public class ExplainPlanWithStatsEnabledIT extends ParallelStatsEnabledIT {
             assertTrue(info.getEstimateInfoTs() > 0);
 
             conn.createStatement()
-                    .execute("ALTER TABLE " + viewName + " SET USE_STATS_FOR_PARALLELIZATION=true");
+                    .execute("ALTER VIEW " + viewName + " SET USE_STATS_FOR_PARALLELIZATION=true");
             sql = "SELECT COUNT(*) FROM " + viewName;
             // query the view
             rs = conn.createStatement().executeQuery(sql);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7b4ccaf7/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 2e78cce..0130f35 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
@@ -167,7 +167,7 @@ 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*/);
+        return Arrays.asList(true, false);
     }
 
     @BeforeClass

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7b4ccaf7/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 b9dbc20..cb08b2f 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
@@ -55,6 +55,7 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.MULTI_TENANT_BYTES
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.NULLABLE_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.NUM_ARGS_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ORDINAL_POSITION_BYTES;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.PARENT_TENANT_ID_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.PK_NAME_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.RETURN_TYPE_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SALT_BUCKETS_BYTES;
@@ -118,6 +119,7 @@ import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorException;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorService;
@@ -2246,6 +2248,81 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
 			}
 		}
 	}
+	
+	public class ScannerAdaptor {
+		private boolean useHTable;
+		private RegionScanner regionScanner;
+		private ResultScanner resultScanner;
+		private List<Cell> results;
+		Result result;
+		
+		public ScannerAdaptor(Scan scan, Region region, boolean useHTable) throws IOException {
+			this.useHTable = useHTable;
+			this.results = Lists.newArrayList();
+			if (useHTable) {
+				HTableInterface hTable = env.getTable(SchemaUtil
+		                .getPhysicalTableName(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES, env.getConfiguration()));
+				resultScanner = hTable.getScanner(scan);
+			} else {
+				regionScanner = region.getScanner(scan);
+			}
+		}
+		
+		public boolean next() throws IOException {
+			if (useHTable) {
+				result = resultScanner.next();
+				if (result==null) {
+					return false;
+				}
+			} else {
+				results.clear();
+				regionScanner.next(results);
+				if (results.isEmpty()) {
+					return false;
+				}
+			}
+			return true;
+		}
+		
+		public Delete getRowDelete(long clientTimestamp) {
+			if (useHTable) {
+				return  new Delete(result.getRow(), clientTimestamp);
+			} else {
+				Cell cell = results.get(0);
+				Delete delete = new Delete(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(), clientTimestamp);
+				return delete;
+			}
+		}
+		
+		public Cell getLinkType() {
+			if (useHTable) {
+				return result.getColumnLatest(TABLE_FAMILY_BYTES, LINK_TYPE_BYTES);
+			} else {
+				Cell kv = results.get(LINK_TYPE_INDEX);
+				if (Bytes.compareTo(kv.getQualifierArray(), kv.getQualifierOffset(), kv.getQualifierLength(),
+						LINK_TYPE_BYTES, 0, LINK_TYPE_BYTES.length) == 0) {
+					return kv;
+				}
+                return null;
+			}
+		}
+		
+		public Cell getParentTenantId() {
+			if (useHTable) {
+				return result.getColumnLatest(TABLE_FAMILY_BYTES, PARENT_TENANT_ID_BYTES);
+			} else {
+				return MetaDataUtil.getCell(results, PARENT_TENANT_ID_BYTES);
+			}
+		}
+		
+		public void close() throws IOException {
+			if (useHTable) {
+				resultScanner.close();
+			} else {
+				regionScanner.close();
+			}
+		}
+	}
 
 	private MetaDataMutationResult doDropTable(byte[] key, byte[] tenantId, byte[] schemaName, byte[] tableName,
 			byte[] parentTableName, PTableType tableType, List<Mutation> catalogMutations,
@@ -2289,80 +2366,43 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         // use the table to generate the Delete markers.
         Scan scan = MetaDataUtil.newTableRowsScan(key, MIN_TABLE_TIMESTAMP, clientTimeStamp);
         List<byte[]> indexNames = Lists.newArrayList();
-        List<Cell> results = Lists.newArrayList();
-        try (RegionScanner scanner = region.getScanner(scan);) {
-            scanner.next(results);
-            if (results.isEmpty()) { // Should not be possible
+        RegionScanner scanner = region.getScanner(scan);
+        boolean regionHasRow = region.getRegionInfo().containsRow(key);
+        ScannerAdaptor scannerAdaptor = new ScannerAdaptor(scan, region, !regionHasRow);
+        try {
+        	if (!scannerAdaptor.next()) {
                 return new MetaDataMutationResult(MutationCode.TABLE_NOT_FOUND,
                         EnvironmentEdgeManager.currentTimeMillis(), null);
             }
 
-            if (tableType == PTableType.TABLE || tableType == PTableType.SYSTEM) {
-                // Handle any child views that exist
-                TableViewFinderResult tableViewFinderResult = new TableViewFinderResult();
-                findAllChildViews(tenantId, table.getSchemaName().getBytes(), table.getTableName().getBytes(), tableViewFinderResult);
-                if (tableViewFinderResult.hasViews()) {
-                    if (isCascade) {
-                        // Recursively delete views adding the mutations to delete child views to rowsToDelete
-                        for (TableInfo tableInfo : tableViewFinderResult.getResults()) {
-                            byte[] viewTenantId = tableInfo.getTenantId();
-                            byte[] viewSchemaName = tableInfo.getSchemaName();
-                            byte[] viewName = tableInfo.getTableName();
-                            byte[] viewKey = tableInfo.getRowKeyPrefix();
-                            Delete delete = new Delete(tableInfo.getRowKeyPrefix(), clientTimeStamp);
-                            catalogMutations.add(delete);
-                            MetaDataMutationResult result = doDropTable(viewKey, viewTenantId, viewSchemaName,
-                                    viewName, null, PTableType.VIEW, catalogMutations, childLinkMutations, invalidateList,
-                                    tableNamesToDelete, sharedTablesToDelete, false, clientVersion);
-                            if (result.getMutationCode() != MutationCode.TABLE_ALREADY_EXISTS) {
-                                return result;
-                            }
-                        }
-                    } else {
-                        // DROP without CASCADE on tables with child views is not permitted
-                        return new MetaDataMutationResult(MutationCode.UNALLOWED_TABLE_MUTATION,
-                                EnvironmentEdgeManager.currentTimeMillis(), null);
-                    }
-                }
-            }
-
-            // Add to list of HTables to delete, unless it's a view or its a shared index
-            if (tableType != PTableType.VIEW && table.getViewIndexId()==null) {
-                tableNamesToDelete.add(table.getPhysicalName().getBytes());
-            }
-            else {
-                sharedTablesToDelete.add(new SharedTableState(table));
-            }
-            invalidateList.add(cacheKey);
-            byte[][] rowKeyMetaData = new byte[5][];
+			MetaDataMutationResult result = processChildViews(tenantId, catalogMutations, childLinkMutations,
+					invalidateList, tableNamesToDelete, sharedTablesToDelete, isCascade, clientVersion, clientTimeStamp,
+					cacheKey, table);
+			 if (result.getMutationCode() != MutationCode.TABLE_ALREADY_EXISTS) {
+                 return result;
+             }
             do {
-                Cell kv = results.get(LINK_TYPE_INDEX);
-                int nColumns = getVarChars(kv.getRowArray(), kv.getRowOffset(), kv.getRowLength(), 0, rowKeyMetaData);
-                if (nColumns == 5
-                        && rowKeyMetaData[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX].length > 0
-                        && Bytes.compareTo(kv.getQualifierArray(), kv.getQualifierOffset(), kv.getQualifierLength(),
-                                LINK_TYPE_BYTES, 0, LINK_TYPE_BYTES.length) == 0) {
-                        LinkType linkType = LinkType.fromSerializedValue(kv.getValueArray()[kv.getValueOffset()]);
-                        if (rowKeyMetaData[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX].length == 0 && linkType == LinkType.INDEX_TABLE) {
+            	Cell linkTypeCell = scannerAdaptor.getLinkType();
+                if (linkTypeCell!=null) {
+                        LinkType linkType = LinkType.fromSerializedValue(linkTypeCell.getValueArray()[linkTypeCell.getValueOffset()]);
+                        if (linkType == LinkType.INDEX_TABLE) {
+                        	byte[][] rowKeyMetaData = new byte[5][];
+                            getVarChars(linkTypeCell.getRowArray(), linkTypeCell.getRowOffset(), linkTypeCell.getRowLength(), 0, rowKeyMetaData);
                             indexNames.add(rowKeyMetaData[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX]);
-                        } else if (linkType == LinkType.PARENT_TABLE || linkType == LinkType.PHYSICAL_TABLE) {
+                        } else if (table.getType() == PTableType.VIEW && (linkType == LinkType.PARENT_TABLE || linkType == LinkType.PHYSICAL_TABLE)) {
                             // delete parent->child link for views
-                            Cell parentTenantIdCell = MetaDataUtil.getCell(results, PhoenixDatabaseMetaData.PARENT_TENANT_ID_BYTES);
+                            Cell parentTenantIdCell = scannerAdaptor.getParentTenantId();
                             PName parentTenantId = parentTenantIdCell!=null ? PNameFactory.newName(parentTenantIdCell.getValueArray(), parentTenantIdCell.getValueOffset(), parentTenantIdCell.getValueLength()) : null;
                             byte[] linkKey = MetaDataUtil.getChildLinkKey(parentTenantId, table.getParentSchemaName(), table.getParentTableName(), table.getTenantId(), table.getName());
                             Delete linkDelete = new Delete(linkKey, clientTimeStamp);
                             childLinkMutations.add(linkDelete);
                         }
                 }
-                // FIXME: Remove when unintentionally deprecated method is fixed (HBASE-7870).
-                // FIXME: the version of the Delete constructor without the lock args was introduced
-                // in 0.94.4, thus if we try to use it here we can no longer use the 0.94.2 version
-                // of the client.
-                Delete delete = new Delete(kv.getRowArray(), kv.getRowOffset(), kv.getRowLength(), clientTimeStamp);
-                catalogMutations.add(delete);
-                results.clear();
-                scanner.next(results);
-            } while (!results.isEmpty());
+                catalogMutations.add(scannerAdaptor.getRowDelete(clientTimeStamp));
+            } while (scannerAdaptor.next());
+        }
+        finally {
+        	scanner.close();
         }
 
         // Recursively delete indexes
@@ -2386,6 +2426,52 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 EnvironmentEdgeManager.currentTimeMillis(), table, tableNamesToDelete);
     }
 
+	private MetaDataMutationResult processChildViews(byte[] tenantId, List<Mutation> catalogMutations,
+			List<Mutation> childLinkMutations, List<ImmutableBytesPtr> invalidateList, List<byte[]> tableNamesToDelete,
+			List<SharedTableState> sharedTablesToDelete, boolean isCascade, int clientVersion, long clientTimeStamp,
+			ImmutableBytesPtr cacheKey, PTable table) throws IOException, SQLException {
+		PTableType tableType = table.getType();
+		if (tableType == PTableType.VIEW || tableType == PTableType.TABLE || tableType == PTableType.SYSTEM) {
+		    // Handle any child views that exist
+		    TableViewFinderResult tableViewFinderResult = new TableViewFinderResult();
+		    findAllChildViews(tenantId, table.getSchemaName().getBytes(), table.getTableName().getBytes(), tableViewFinderResult);
+		    if (tableViewFinderResult.hasViews()) {
+		        if (isCascade) {
+		            // Recursively delete views adding the mutations to delete child views to rowsToDelete
+		            for (TableInfo tableInfo : tableViewFinderResult.getResults()) {
+		                byte[] viewTenantId = tableInfo.getTenantId();
+		                byte[] viewSchemaName = tableInfo.getSchemaName();
+		                byte[] viewName = tableInfo.getTableName();
+		                byte[] viewKey = tableInfo.getRowKeyPrefix();
+		                Delete delete = new Delete(tableInfo.getRowKeyPrefix(), clientTimeStamp);
+		                catalogMutations.add(delete);
+		                MetaDataMutationResult result = doDropTable(viewKey, viewTenantId, viewSchemaName,
+		                        viewName, null, PTableType.VIEW, catalogMutations, childLinkMutations, invalidateList,
+		                        tableNamesToDelete, sharedTablesToDelete, isCascade, clientVersion);
+		                if (result.getMutationCode() != MutationCode.TABLE_ALREADY_EXISTS) {
+		                    return result;
+		                }
+		            }
+		        } else {
+		            // DROP without CASCADE on tables with child views is not permitted
+		            return new MetaDataMutationResult(MutationCode.UNALLOWED_TABLE_MUTATION,
+		                    EnvironmentEdgeManager.currentTimeMillis(), null);
+		        }
+		    }
+		}
+
+		// Add to list of HTables to delete, unless it's a view or its a shared index
+		if (tableType != PTableType.VIEW && table.getViewIndexId()==null) {
+		    tableNamesToDelete.add(table.getPhysicalName().getBytes());
+		}
+		else {
+		    sharedTablesToDelete.add(new SharedTableState(table));
+		}
+		invalidateList.add(cacheKey);
+		return new MetaDataMutationResult(MutationCode.TABLE_ALREADY_EXISTS,
+                EnvironmentEdgeManager.currentTimeMillis(), null);
+	}
+
 
     private static interface ColumnMutator {
         MetaDataMutationResult updateMutation(PTable table, byte[][] rowKeyMetaData,

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7b4ccaf7/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ViewFinder.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ViewFinder.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ViewFinder.java
index cde7a01..7389437 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ViewFinder.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ViewFinder.java
@@ -40,6 +40,7 @@ import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.tuple.ResultTuple;
 import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.MetaDataUtil;
 import org.apache.phoenix.util.SchemaUtil;
 
 import com.google.common.collect.LinkedHashMultimap;
@@ -81,12 +82,8 @@ class ViewFinder {
         if (linkType==PTable.LinkType.INDEX_TABLE || linkType==PTable.LinkType.EXCLUDED_COLUMN) {
             throw new IllegalArgumentException("findAllRelatives does not support link type "+linkType);
         }
-        Scan scan = new Scan();
-        byte[] startRow = ByteUtil.concat(SchemaUtil.getTableKey(tenantId, schema, table), SEPARATOR_BYTE_ARRAY);
-        byte[] stopRow = ByteUtil.nextKey(startRow);
-        scan.setStartRow(startRow);
-        scan.setStopRow(stopRow);
-        scan.setTimeRange(0, timestamp);
+        byte[] key = SchemaUtil.getTableKey(tenantId, schema, table);
+		Scan scan = MetaDataUtil.newTableRowsScan(key, MetaDataProtocol.MIN_TABLE_TIMESTAMP, timestamp);
         SingleColumnValueFilter linkFilter =
             new SingleColumnValueFilter(TABLE_FAMILY_BYTES, LINK_TYPE_BYTES, CompareFilter.CompareOp.EQUAL,
                 linkType.getSerializedValueAsByteArray());

http://git-wip-us.apache.org/repos/asf/phoenix/blob/7b4ccaf7/phoenix-core/src/test/java/org/apache/phoenix/coprocessor/MetaDataEndpointImplTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/coprocessor/MetaDataEndpointImplTest.java b/phoenix-core/src/test/java/org/apache/phoenix/coprocessor/MetaDataEndpointImplTest.java
index fbde4b7..4803839 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/coprocessor/MetaDataEndpointImplTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/coprocessor/MetaDataEndpointImplTest.java
@@ -234,7 +234,9 @@ public class MetaDataEndpointImplTest extends ParallelStatsDisabledIT {
         // now lets drop the parent table
         conn.createStatement().execute("DROP TABLE " + baseTable + " CASCADE");
 
-        // the grand child should no longer exist
+        // the tables should no longer exist
+        PhoenixRuntime.getTableNoCache(conn, baseTable);
+        PhoenixRuntime.getTableNoCache(conn, child);
         PhoenixRuntime.getTableNoCache(conn, grandChild);
     }
 


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

Posted by td...@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/d77c237b
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/d77c237b
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/d77c237b

Branch: refs/heads/system-catalog
Commit: d77c237b560900671c3a9c58f6f2398342655e8a
Parents: 6807dac
Author: maryannxue <ma...@gmail.com>
Authored: Tue Dec 5 10:59:41 2017 -0800
Committer: maryannxue <ma...@gmail.com>
Committed: Tue Dec 5 10:59:41 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/d77c237b/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/d77c237b/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);


[37/50] [abbrv] phoenix git commit: PHOENIX-4468 Looking up a parent index table of a child view from a different client fails

Posted by td...@apache.org.
PHOENIX-4468 Looking up a parent index table of a child view from a different client fails


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

Branch: refs/heads/system-catalog
Commit: c935f57f5e7afb42ff8b62a4712ad7d8ffed17cc
Parents: 9d8be0e
Author: Thomas D'Silva <td...@apache.org>
Authored: Mon Dec 18 11:48:39 2017 -0800
Committer: Thomas D'Silva <td...@apache.org>
Committed: Wed Dec 20 10:41:12 2017 -0800

----------------------------------------------------------------------
 .../apache/phoenix/end2end/PhoenixDriverIT.java | 160 +++++++++++++++++++
 .../org/apache/phoenix/end2end/SequenceIT.java  |  29 ----
 phoenix-core/src/main/antlr3/PhoenixSQL.g       |   1 +
 .../apache/phoenix/schema/MetaDataClient.java   |   4 +-
 .../org/apache/phoenix/schema/PTableImpl.java   |   2 +-
 .../org/apache/phoenix/util/PhoenixRuntime.java |  32 +++-
 6 files changed, 189 insertions(+), 39 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/c935f57f/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixDriverIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixDriverIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixDriverIT.java
new file mode 100644
index 0000000..216653c
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixDriverIT.java
@@ -0,0 +1,160 @@
+/*
+ * 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.assertTrue;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.Properties;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.phoenix.jdbc.PhoenixDriver;
+import org.apache.phoenix.jdbc.PhoenixStatement;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.query.QueryServicesOptions;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.apache.phoenix.util.SchemaUtil;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class PhoenixDriverIT extends BaseUniqueNamesOwnClusterIT {
+    
+    private static HBaseTestingUtility hbaseTestUtil;
+    private static String zkQuorum;
+    
+    @BeforeClass
+    public static void setUp() throws Exception {
+        Configuration conf = HBaseConfiguration.create();
+        hbaseTestUtil = new HBaseTestingUtility(conf);
+        setUpConfigForMiniCluster(conf);
+        conf.set(QueryServices.EXTRA_JDBC_ARGUMENTS_ATTRIB, QueryServicesOptions.DEFAULT_EXTRA_JDBC_ARGUMENTS);
+        hbaseTestUtil.startMiniCluster();
+        // establish url and quorum. Need to use PhoenixDriver and not PhoenixTestDriver
+        zkQuorum = "localhost:" + hbaseTestUtil.getZkCluster().getClientPort();
+        url = PhoenixRuntime.JDBC_PROTOCOL + PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR + zkQuorum;
+        DriverManager.registerDriver(PhoenixDriver.INSTANCE);
+    }
+    
+    public Connection createConnection(boolean isMultiTenant, boolean isDifferentClient) throws SQLException {
+        Properties props = new Properties();
+        props.setProperty(QueryServices.RETURN_SEQUENCE_VALUES_ATTRIB, "false");
+        // force the use of ConnectionQueryServicesImpl instead of ConnectionQueryServicesTestImpl
+        props.put(QueryServices.EXTRA_JDBC_ARGUMENTS_ATTRIB,
+            QueryServicesOptions.DEFAULT_EXTRA_JDBC_ARGUMENTS);
+        if (isMultiTenant)
+            props.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, "tenant1");
+        StringBuilder sb = new StringBuilder(url);
+        if (isMultiTenant)
+            sb.append(PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR + "Client2");
+        return DriverManager.getConnection(sb.toString(), props);
+    }
+    
+    @Test
+    public void testReturnAllSequencesNotCalledForNoOpenConnections() throws Exception {
+        String schemaName = "S";
+        String sequenceNameWithoutSchema = generateUniqueSequenceName();
+        String sequenceName = SchemaUtil.getTableName(schemaName, sequenceNameWithoutSchema);
+        
+        Connection conn = createConnection(false, false);
+        conn.createStatement().execute("CREATE SEQUENCE " + sequenceName + " START WITH 3 INCREMENT BY 2 CACHE 5");
+        
+        String query = "SELECT NEXT VALUE FOR " + sequenceName ;
+        ResultSet rs = conn.prepareStatement(query).executeQuery();
+        assertTrue(rs.next());
+        assertEquals(3, rs.getInt(1));
+        assertFalse(rs.next());
+        rs = conn.prepareStatement(query).executeQuery();
+        assertTrue(rs.next());
+        assertEquals(5, rs.getInt(1));
+        assertFalse(rs.next());
+        conn.close();
+        
+        conn = createConnection(false, false);
+        // verify that calling close() does not return sequence values back to the server
+        query = "SELECT CURRENT_VALUE FROM \"SYSTEM\".\"SEQUENCE\" WHERE SEQUENCE_SCHEMA=? AND SEQUENCE_NAME=?";
+        PreparedStatement preparedStatement = conn.prepareStatement(query);
+        preparedStatement.setString(1, schemaName);
+        preparedStatement.setString(2, sequenceNameWithoutSchema);
+        rs = preparedStatement.executeQuery();
+        assertTrue(rs.next());
+        assertEquals(13, rs.getInt(1));
+        assertFalse(rs.next());
+        conn.close();
+    }
+    
+    @Test
+    public void testViewParentIndexLookupMutipleClients() throws Exception {
+        helpTestViewParentIndexLookupMutipleClients(false);
+    }
+    
+    @Test
+    public void testMulitTenantViewParentIndexLookupMutipleClients() throws Exception {
+        helpTestViewParentIndexLookupMutipleClients(true);
+    }
+    
+    public void helpTestViewParentIndexLookupMutipleClients(boolean isMultiTenant) throws Exception {
+        final String baseTableName = generateUniqueName();
+        final String baseTableIndexName = generateUniqueName();
+        final String viewName = generateUniqueName();
+        try (Connection globalConn = createConnection(false, false);
+                Connection conn1 = createConnection(isMultiTenant, false);
+                Connection conn2 = createConnection(isMultiTenant, false)) {
+            // create base table
+            String baseTableDdl = "CREATE TABLE " + baseTableName + " (" +
+                    ( isMultiTenant ? "TENANT_ID VARCHAR(1) NOT NULL," : "") +
+                    "PK CHAR(1) NOT NULL," +
+                    "V1 CHAR(1)," +
+                    "V2 CHAR(1)," +
+                    "V3 CHAR(1)" + 
+                    "CONSTRAINT pk PRIMARY KEY (" + (isMultiTenant ? "TENANT_ID," : "") + " pk))";
+            globalConn.createStatement().execute(baseTableDdl);
+            
+            // create index on parent view
+            globalConn.createStatement().execute("CREATE INDEX " + baseTableIndexName + " ON " + baseTableName + " (V2) INCLUDE (v1, V3)");
+            
+            // create a view on the base table
+            String viewDDL = "CREATE VIEW " + viewName + " AS SELECT * FROM " + baseTableName + " WHERE V1 = 'X'";
+            conn1.createStatement().execute(viewDDL);
+            conn1.commit();
+
+            // ensure we can use parent table index
+            String sql = "SELECT V3 FROM " + viewName +" WHERE V2 = '3'";
+            PhoenixStatement stmt = conn1.createStatement().unwrap(PhoenixStatement.class);
+            stmt.executeQuery(sql);
+            PTable indexTable = stmt.getQueryPlan().getTableRef().getTable();
+            String tableName = indexTable.getName().getString();
+            String expectedTableName = baseTableIndexName + QueryConstants.CHILD_VIEW_INDEX_NAME_SEPARATOR + viewName;
+            assertEquals("Parent Index table is not used ", expectedTableName, tableName);
+            
+            // verify that we can look up the index using PhoenixRuntime from a different client
+            PTable table = PhoenixRuntime.getTable(conn2, tableName);
+            assertEquals(indexTable, table);
+        }
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c935f57f/phoenix-core/src/it/java/org/apache/phoenix/end2end/SequenceIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SequenceIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SequenceIT.java
index 3014e45..9b870e1 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SequenceIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SequenceIT.java
@@ -1358,35 +1358,6 @@ public class SequenceIT extends ParallelStatsDisabledIT {
         assertEquals(5, rs.getInt(2));        
     }
     
-    @Test
-    public void testReturnAllSequencesNotCalledForNoOpenConnections() throws Exception {
-        String sequenceName = generateSequenceNameWithSchema();
-        String sequenceNameWithoutSchema = getNameWithoutSchema(sequenceName);
-        String schemaName = getSchemaName(sequenceName);
-        
-        conn.createStatement().execute("CREATE SEQUENCE " + sequenceName + " START WITH 3 INCREMENT BY 2 CACHE 5");
-        
-        String query = "SELECT NEXT VALUE FOR " + sequenceName ;
-        ResultSet rs = conn.prepareStatement(query).executeQuery();
-        assertTrue(rs.next());
-        assertEquals(3, rs.getInt(1));
-        assertFalse(rs.next());
-        rs = conn.prepareStatement(query).executeQuery();
-        assertTrue(rs.next());
-        assertEquals(5, rs.getInt(1));
-        assertFalse(rs.next());
-        
-        // verify that calling close() does not return sequence values back to the server
-        query = "SELECT CURRENT_VALUE FROM \"SYSTEM\".\"SEQUENCE\" WHERE SEQUENCE_SCHEMA=? AND SEQUENCE_NAME=?";
-        PreparedStatement preparedStatement = conn.prepareStatement(query);
-        preparedStatement.setString(1, schemaName);
-        preparedStatement.setString(2, sequenceNameWithoutSchema);
-        rs = preparedStatement.executeQuery();
-        assertTrue(rs.next());
-        assertEquals(13, rs.getInt(1));
-        assertFalse(rs.next());
-    }
-    
     private static String getSchemaName(String tableName) {
     	return tableName.substring(0, tableName.indexOf("."));
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c935f57f/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 87153cd..5e64984 100644
--- a/phoenix-core/src/main/antlr3/PhoenixSQL.g
+++ b/phoenix-core/src/main/antlr3/PhoenixSQL.g
@@ -1114,6 +1114,7 @@ from_table_name returns [TableName ret]
 table_identifier returns [String ret]
     :   c=identifier {
            if (c.contains(QueryConstants.NAMESPACE_SEPARATOR) ) { throw new RuntimeException("Table or schema name cannot contain colon"); }
+           if (c.contains(QueryConstants.CHILD_VIEW_INDEX_NAME_SEPARATOR) ) { throw new RuntimeException("Table or schema name cannot contain hash"); }
            $ret = c;
     }
     ;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c935f57f/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 5ec5ac3..403cbfe 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
@@ -854,8 +854,8 @@ public class MetaDataClient {
             if (containsAllReqdCols) {
                 // Tack on view statement to index to get proper filtering for view
                 String viewStatement = IndexUtil.rewriteViewStatement(connection, index, parentTable, view.getViewStatement());
-                PName modifiedIndexName = PNameFactory.newName(index.getSchemaName().getString() + QueryConstants.CHILD_VIEW_INDEX_NAME_SEPARATOR
-                        + index.getName().getString() + QueryConstants.CHILD_VIEW_INDEX_NAME_SEPARATOR + view.getName().getString());
+                PName modifiedIndexName = PNameFactory.newName(index.getName().getString() 
+                    + QueryConstants.CHILD_VIEW_INDEX_NAME_SEPARATOR + view.getName().getString());
                 // add the index table with a new name so that it does not conflict with the existing index table
                 // also set update cache frequency to never since the renamed index is not present on the server
                 indexesToAdd.add(PTableImpl.makePTable(index, modifiedIndexName, viewStatement, Long.MAX_VALUE, view.getTenantId()));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c935f57f/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
index 9525127..7df2458 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
@@ -267,7 +267,7 @@ public class PTableImpl implements PTable {
     
     public static PTableImpl makePTable(PTable table, PName tableName, long timeStamp, List<PName> physicalNames, List<PTable> indexes, String viewStatement, long updateCacheFrequency, PName tenantId) throws SQLException {
             return new PTableImpl(
-                    tenantId, table.getSchemaName(), tableName, table.getType(), table.getIndexState(), timeStamp,
+                    tenantId, PNameImpl.EMPTY_NAME, tableName, table.getType(), table.getIndexState(), timeStamp,
                     table.getSequenceNumber(), table.getPKName(), table.getBucketNum(), getColumnsToClone(table), table.getParentSchemaName(), table.getParentTableName(),
                     indexes, table.isImmutableRows(), physicalNames, table.getDefaultFamilyName(), viewStatement,
                     table.isWALDisabled(), table.isMultiTenant(), table.getStoreNulls(), table.getViewType(), table.getViewIndexId(), table.getIndexType(),

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c935f57f/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java b/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java
index 16ef206..31ab194 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java
@@ -430,8 +430,15 @@ public class PhoenixRuntime {
         return result.getTable();
 
     }
+    
     /**
-     * 
+     * Returns the table if it is found in the connection metadata cache. If the metadata of this
+     * table has changed since it was put in the cache these changes will not necessarily be
+     * reflected in the returned table. If the table is not found, makes a call to the server to
+     * fetch the latest metadata of the table. This is different than how a table is resolved when
+     * it is referenced from a query (a call is made to the server to fetch the latest metadata of the table
+     * depending on the UPDATE_CACHE_FREQUENCY property)
+     * See https://issues.apache.org/jira/browse/PHOENIX-4475
      * @param conn
      * @param name requires a pre-normalized table name or a pre-normalized schema and table name
      * @return
@@ -443,13 +450,24 @@ public class PhoenixRuntime {
         try {
             table = pconn.getTable(new PTableKey(pconn.getTenantId(), name));
         } catch (TableNotFoundException e) {
-            String schemaName = SchemaUtil.getSchemaNameFromFullName(name);
-            String tableName = SchemaUtil.getTableNameFromFullName(name);
-            MetaDataMutationResult result = new MetaDataClient(pconn).updateCache(schemaName, tableName);
-            if (result.getMutationCode() != MutationCode.TABLE_ALREADY_EXISTS) {
-                throw e;
+            // parent indexes on child view metadata rows are not present on the server
+            if (name.contains(QueryConstants.CHILD_VIEW_INDEX_NAME_SEPARATOR)) {
+                String viewName =
+                        SchemaUtil.getTableNameFromFullName(name,
+                            QueryConstants.CHILD_VIEW_INDEX_NAME_SEPARATOR);
+                // resolve the view which should also load any parent indexes
+                getTable(conn, viewName);
+                table = pconn.getTable(new PTableKey(pconn.getTenantId(), name));
+            } else {
+                String schemaName = SchemaUtil.getSchemaNameFromFullName(name);
+                String tableName = SchemaUtil.getTableNameFromFullName(name);
+                MetaDataMutationResult result =
+                        new MetaDataClient(pconn).updateCache(schemaName, tableName);
+                if (result.getMutationCode() != MutationCode.TABLE_ALREADY_EXISTS) {
+                    throw e;
+                }
+                table = result.getTable();
             }
-            table = result.getTable();
         }
         return table;
     }


[06/50] [abbrv] phoenix git commit: Merge remote-tracking branch 'upstream/master' into PHOENIX-3534

Posted by td...@apache.org.
Merge remote-tracking branch 'upstream/master' into PHOENIX-3534


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

Branch: refs/heads/system-catalog
Commit: 39c0d1d41ceca41bfb0257de6d18aab59f1c9566
Parents: 758efd9 2a8e1c7
Author: Thomas D'Silva <td...@apache.org>
Authored: Mon Nov 13 08:22:34 2017 -0800
Committer: Thomas D'Silva <td...@apache.org>
Committed: Mon Nov 13 08:22:34 2017 -0800

----------------------------------------------------------------------
 LICENSE                                         |  43 +-
 dev/make_rc.sh                                  |  26 +-
 dev/make_rc_on_mac.sh                           | 121 ----
 dev/release_files/LICENSE                       |   2 +
 phoenix-assembly/pom.xml                        |   2 +-
 phoenix-client/pom.xml                          |   2 +-
 phoenix-core/pom.xml                            |   6 +-
 .../phoenix/end2end/CollationKeyFunctionIT.java | 181 +++++
 .../org/apache/phoenix/end2end/DeleteIT.java    |  96 ++-
 .../end2end/ExplainPlanWithStatsEnabledIT.java  | 146 +++-
 .../phoenix/end2end/TableDDLPermissionsIT.java  | 694 +++++++++++++++++++
 phoenix-core/src/main/antlr3/PhoenixSQL.g       |  15 +-
 .../org/apache/hadoop/hbase/ipc/RpcUtil.java    |  32 +
 .../apache/phoenix/compile/DeleteCompiler.java  |   5 +-
 .../BaseMetaDataEndpointObserver.java           | 111 +++
 .../coprocessor/MetaDataEndpointImpl.java       | 358 +++++++---
 .../coprocessor/MetaDataEndpointObserver.java   |  68 ++
 .../phoenix/coprocessor/MetaDataProtocol.java   |   8 +-
 .../coprocessor/MetaDataRegionObserver.java     |  17 +-
 .../coprocessor/PhoenixAccessController.java    | 628 +++++++++++++++++
 .../PhoenixMetaDataCoprocessorHost.java         | 236 +++++++
 .../apache/phoenix/coprocessor/ViewFinder.java  |   5 +-
 .../phoenix/expression/ExpressionType.java      |   4 +-
 .../function/CollationKeyFunction.java          | 199 ++++++
 .../index/PhoenixIndexFailurePolicy.java        | 109 +--
 .../phoenix/iterate/BaseResultIterators.java    |  42 +-
 .../apache/phoenix/jdbc/PhoenixConnection.java  |   3 +
 .../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 +
 .../apache/phoenix/util/VarBinaryFormatter.java |  52 ++
 .../function/CollationKeyFunctionTest.java      | 243 +++++++
 .../apache/phoenix/parse/QueryParserTest.java   |  21 +
 phoenix-flume/pom.xml                           |   2 +-
 phoenix-hive/pom.xml                            |   2 +-
 phoenix-kafka/pom.xml                           |   2 +-
 phoenix-load-balancer/pom.xml                   |   2 +-
 phoenix-pherf/pom.xml                           |   2 +-
 phoenix-pig/pom.xml                             |   2 +-
 phoenix-queryserver-client/pom.xml              |   2 +-
 phoenix-queryserver/pom.xml                     |   2 +-
 phoenix-server/pom.xml                          |   3 +-
 phoenix-spark/pom.xml                           |   2 +-
 phoenix-tracing-webapp/pom.xml                  |   2 +-
 pom.xml                                         |   7 +-
 48 files changed, 3253 insertions(+), 357 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/39c0d1d4/phoenix-core/src/it/java/org/apache/phoenix/end2end/TableDDLPermissionsIT.java
----------------------------------------------------------------------
diff --cc phoenix-core/src/it/java/org/apache/phoenix/end2end/TableDDLPermissionsIT.java
index 0000000,971383b..2e78cce
mode 000000,100644..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
@@@ -1,0 -1,692 +1,694 @@@
+ /*
+  * 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);
++        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);
++            verifyAllowed(dropView(viewName3), regularUser);
++            verifyAllowed(dropView(viewName4), regularUser);
+ 
+             // 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/39c0d1d4/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
----------------------------------------------------------------------
diff --cc phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
index 9f7a629,afbd63f..b9dbc20
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
@@@ -89,13 -92,12 +90,15 @@@ 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.LinkedList;
  import java.util.List;
 +import java.util.ListIterator;
  import java.util.Map;
 +import java.util.Map.Entry;
  import java.util.NavigableMap;
+ import java.util.Set;
  
  import org.apache.hadoop.conf.Configuration;
  import org.apache.hadoop.hbase.Cell;
@@@ -119,8 -120,13 +122,10 @@@ import org.apache.hadoop.hbase.client.S
  import org.apache.hadoop.hbase.coprocessor.CoprocessorException;
  import org.apache.hadoop.hbase.coprocessor.CoprocessorService;
  import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 -import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
 -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;
@@@ -554,209 -570,11 +570,213 @@@ public class MetaDataEndpointImpl exten
                  ServerUtil.createIOException(SchemaUtil.getTableName(schemaName, tableName), t));
          }
      }
 +    
 +	private Pair<PTable, MetaDataProtos.MutationCode> combineColumns(PTable table, byte[] tenantId, byte[] schemaName,
 +			byte[] tableName, long timestamp, int clientVersion) throws SQLException, IOException {
 +		// combine columns for view and view indexes
 +		boolean hasIndexId = table.getViewIndexId() != null;
 +		if (table.getType() != PTableType.VIEW && !hasIndexId) {
 +			return new Pair<PTable, MetaDataProtos.MutationCode>(table, MetaDataProtos.MutationCode.TABLE_ALREADY_EXISTS);
 +		}
 +		boolean isDiverged = isDivergedView(table);
 +		// here you combine columns from the parent tables
 +		// the logic is as follows, if the PColumn is in the EXCLUDED_COLUMNS
 +		// remove it,
 +		// otherwise priority of keeping duplicate columns is child -> parent
 +		List<byte[]> ancestorList = Lists.newArrayList();
 +		TableViewFinderResult viewFinderResult = new TableViewFinderResult();
 +		if (PTableType.VIEW == table.getType()) {
 +			findAncestorViews(tenantId, schemaName, tableName, viewFinderResult);
 +		} else { // is a view index
 +			findAncestorViewsOfIndex(tenantId, schemaName, tableName, viewFinderResult);
 +		}
 +		if (viewFinderResult.getResults().isEmpty()) {
 +			// no need to combine columns for local indexes on regular tables
 +			return new Pair<PTable, MetaDataProtos.MutationCode>(table, MetaDataProtos.MutationCode.TABLE_ALREADY_EXISTS);
 +		}
 +		for (TableInfo viewInfo : viewFinderResult.getResults()) {
 +			ancestorList.add(viewInfo.getRowKeyPrefix());
 +		}
 +		List<PColumn> allColumns = Lists.newArrayList();
 +		List<PColumn> excludedColumns = Lists.newArrayList();
 +		// add my own columns first in reverse order
 +		List<PColumn> myColumns = table.getColumns();
 +		for (int i = myColumns.size() - 1; i >= 0; i--) {
 +			PColumn pColumn = myColumns.get(i);
 +			if (pColumn.isExcluded()) {
 +				excludedColumns.add(pColumn);
 +			} else if (!pColumn.equals(SaltingUtil.SALTING_COLUMN)) { // skip salted column as it will be added from the base table columns
 +				allColumns.add(pColumn);
 +			}
 +		}
 +		// index columns that have been dropped in the parent table
 +		boolean isSalted = table.getBucketNum() != null;
 +		int indexPosOffset = (isSalted ? 1 : 0) + (table.isMultiTenant() ? 1 : 0) + 1;
 +		// map from indexed expression to list of data columns that have been dropped
 +		Map<PColumn, List<String>> droppedColMap = Maps.newHashMapWithExpectedSize(table.getColumns().size());
 +		if (hasIndexId) {
 +			ColumnNameTrackingExpressionCompiler expressionCompiler = new ColumnNameTrackingExpressionCompiler();
 +	        for (int i = indexPosOffset; i < table.getPKColumns().size(); i++) {
 +	            PColumn indexColumn = table.getPKColumns().get(i);
 +	            try {
 +	                expressionCompiler.reset();
 +	                String expressionStr = IndexUtil.getIndexColumnExpressionStr(indexColumn);
 +	                ParseNode parseNode  = SQLParser.parseCondition(expressionStr);
 +	                parseNode.accept(expressionCompiler);
 +	                droppedColMap.put(indexColumn, Lists.newArrayList(expressionCompiler.getDataColumnNames()));
 +	            } catch (SQLException e) {
 +	                throw new RuntimeException(e); // Impossible
 +	            }
 +	        }
 +		}
 +		// now go up from child to parent all the way to the base table:
 +		PTable baseTable = null;
 +		long maxTableTimestamp = -1;
 +		int numPKCols = table.getPKColumns().size();
 +		for (int i = 0; i < ancestorList.size(); i++) {
 +			byte[] tableInQuestion = ancestorList.get(i);
 +			PTable pTable = this.doGetTable(tableInQuestion, timestamp, clientVersion);
 +			if (pTable == null) {
 +				String tableNameLink = Bytes.toString(tableInQuestion);
 +				throw new TableNotFoundException("ERROR COMBINING COLUMNS FOR: " + tableNameLink);
 +			} else {
 +				// if it has an index id only combine columns for view indexes
 +				// (and not local indexes on regular tables)
 +				if (i == 0 && hasIndexId && pTable.getType() != PTableType.VIEW) {
 +					return new Pair<PTable, MetaDataProtos.MutationCode>(table, MetaDataProtos.MutationCode.TABLE_ALREADY_EXISTS);
 +				}
 +				if (TABLE.equals(pTable.getType())) {
 +					baseTable = pTable;
 +				}
 +				maxTableTimestamp = Math.max(maxTableTimestamp, pTable.getTimeStamp());
 +				if (hasIndexId) {
 +					// add all pk columns of parent tables to indexes
 +					for (PColumn column : pTable.getPKColumns()) {
 +						if (column.isExcluded()) {
 +							continue;
 +						}
 +						column = IndexUtil.getIndexPKColumn(++numPKCols, column);
 +						int existingColumnIndex = allColumns.indexOf(column);
 +						if (existingColumnIndex == -1) {
 +							allColumns.add(0, column);
 +						}
 +						// TODO should we just generate columnsToAdd here (since
 +						// it doesnt need to be reversed)
 +					}
 +					for (int j = 0; j < pTable.getColumns().size(); j++) {
 +						PColumn tableColumn = pTable.getColumns().get(j);
 +						if (tableColumn.isExcluded()) {
 +							continue;
 +						}
 +						String dataColumnName = tableColumn.getName().getString();
 +						// remove from list of dropped columns since it
 +						// still exists
 +						for (Entry<PColumn, List<String>> entry : droppedColMap.entrySet()) {
 +							entry.getValue().remove(dataColumnName);
 +						}
 +					}
 +				} else {
 +					List<PColumn> someTablesColumns = PTableImpl.getColumnsToClone(pTable);
 +					if (someTablesColumns != null) {
 +						for (int j = someTablesColumns.size() - 1; j >= 0; j--) {
 +							PColumn column = someTablesColumns.get(j);
 +							// For diverged views we always include pk columns
 +							// of the base table. We have to include these pk
 +							// columns to be able to support adding pk columns
 +							// to the diverged view
 +							// We only include regular columns that were created
 +							// before the view diverged
 +							if (isDiverged && column.getFamilyName()!=null && column.getTimestamp() > table.getTimeStamp()) {
 +								continue;
 +							}
 +							// need to check if this column is in the list of excluded (dropped) columns of the view
 +							int existingIndex = excludedColumns.indexOf(column);
 +							if (existingIndex != -1) {
 +								// if it is, only exclude the column if was
 +								// created before the column was dropped in the
 +								// view in order to handle the case where a base
 +								// table column is dropped in a view, then
 +								// dropped in the base table and then added back
 +								// to the base table
 +								if (column.getTimestamp() <= excludedColumns.get(existingIndex).getTimestamp()) {
 +									continue;
 +								}
 +							}
 +							if (column.isExcluded()) {
 +								excludedColumns.add(column);
 +							} else {
 +								int existingColumnIndex = allColumns.indexOf(column);
 +								if (existingColumnIndex != -1) {
 +									// TODO ask james about this
 +									// we always keep the parent table column so
 +									// that we can handle
 +									// the case when you add a column that
 +									// already exists in a view to the base
 +									// table
 +									PColumn existingColumn = allColumns.get(existingColumnIndex);
 +									// if (column.getTimestamp() <
 +									// existingColumn.getTimestamp()) {
 +									allColumns.remove(existingColumnIndex);
 +									allColumns.add(column);
 +									// }
 +								} else {
 +									allColumns.add(column);
 +								}
 +							}
 +						}
 +					}
 +				}
 +			}
 +		}
 +		for (Entry<PColumn, List<String>> entry : droppedColMap.entrySet()) {
 +			if (!entry.getValue().isEmpty()) {
 +				PColumn indexColumnToBeDropped = entry.getKey();
 +				if (SchemaUtil.isPKColumn(indexColumnToBeDropped)) {
 +					// if an indexed column was dropped in an ancestor then we
 +					// cannot use this index an more
 +					// TODO figure out a way to actually drop this view index
 +					return new Pair<PTable, MetaDataProtos.MutationCode>(null, MetaDataProtos.MutationCode.TABLE_NOT_FOUND);
 +				} else {
 +					allColumns.remove(indexColumnToBeDropped);
 +				}
 +			}
 +		}
 +		// lets remove the excluded columns first if the timestamp is newer than
 +		// the added column
 +		for (PColumn excludedColumn : excludedColumns) {
 +			int index = allColumns.indexOf(excludedColumn);
 +			if (index != -1) {
 +				if (allColumns.get(index).getTimestamp() <= excludedColumn.getTimestamp()) {
 +					allColumns.remove(excludedColumn);
 +				}
 +			}
 +		}
 +		List<PColumn> columnsToAdd = Lists.newArrayList();
 +		int position = isSalted ? 1 : 0;
 +		for (int i = allColumns.size() - 1; i >= 0; i--) {
 +			PColumn column = allColumns.get(i);
 +			if (table.getColumns().contains(column)) {
 +				// for views this column is not derived from an ancestor
 +				columnsToAdd.add(new PColumnImpl(column, position));
 +			} else {
 +				columnsToAdd.add(new PColumnImpl(column, true, position));
 +			}
 +			position++;
 +		}
 +		// need to have the columns in the PTable to use the WhereCompiler
 +		// unfortunately so this needs to be done
 +		// twice....
 +		// TODO set the view properties correctly instead of just setting them
 +		// same as the base table
 +		int baseTableColumnCount = isDiverged ? QueryConstants.DIVERGED_VIEW_BASE_COLUMN_COUNT : columnsToAdd.size() - myColumns.size();
 +		PTableImpl pTable = PTableImpl.makePTable(table, baseTable, columnsToAdd, maxTableTimestamp, baseTableColumnCount);
 +		return WhereConstantParser.addViewInfoToPColumnsIfNeeded(pTable);
 +	}
  
+     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);
@@@ -1529,11 -1334,11 +1549,11 @@@
      }
  
      /**
 -     * 
 +     *
       * @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);
@@@ -1566,12 -1396,12 +1611,12 @@@
          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
      public void createTable(RpcController controller, CreateTableRequest request,
              RpcCallback<MetaDataResponse> done) {
@@@ -1586,65 -1416,22 +1631,74 @@@
              byte[] tenantIdBytes = rowKeyMetaData[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
              schemaName = rowKeyMetaData[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX];
              tableName = rowKeyMetaData[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
 +            // no need to run OrpanCleaner (which cleans up orphaned views) while creating SYSTEM tables  env.getTable
 +            if (Bytes.compareTo(schemaName,PhoenixDatabaseMetaData.SYSTEM_SCHEMA_NAME_BYTES)!=0) {
 +	            HTableInterface systemCatalog = null;
 +	            try {
 +	            	// can't use SchemaUtil.getPhysicalTableName on server side as we don't know whether 
 +	            	// the system tables have been migrated to the system namespaces
 +	            	TableName systemCatalogTableName = env.getRegion().getTableDesc().getTableName();
 +	                systemCatalog = env.getTable(systemCatalogTableName);
 +	                OrphanCleaner.reapOrphans(systemCatalog, tenantIdBytes, schemaName, tableName);
 +	            } finally {
 +	                if (systemCatalog != null) {
 +	                    systemCatalog.close();
 +	                }
 +	            }
 +            }
+             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());
 +            ViewType viewType = MetaDataUtil.getViewType(tableMetadata, GenericKeyValueBuilder.INSTANCE, new ImmutableBytesWritable());
 +
 +            // Here we are passed the parent's columns to add to a view, PHOENIX-3534 allows for a splittable
 +            // System.Catalog thus we only store the columns that are new to the view, not the parents columns,
 +            // thus here we remove everything that is ORDINAL.POSITION <= baseColumnCount and update the
 +            // ORDINAL.POSITIONS to be shifted accordingly.
 +            if (PTableType.VIEW.equals(tableType) && !ViewType.MAPPED.equals(viewType)) {
 +            	boolean isSalted = MetaDataUtil.getSaltBuckets(tableMetadata, GenericKeyValueBuilder.INSTANCE, new ImmutableBytesWritable()) > 0;
 +				int baseColumnCount = MetaDataUtil.getBaseColumnCount(tableMetadata) - (isSalted ? 1 : 0);
 +                if (baseColumnCount > 0) {
 +                    Iterator<Mutation> mutationIterator = tableMetadata.iterator();
 +                    while (mutationIterator.hasNext()) {
 +                        Mutation mutation = mutationIterator.next();
 +                        // if not null and ordinal position < base column count remove this mutation
 +                        ImmutableBytesWritable ptr = new ImmutableBytesWritable();
 +                        MetaDataUtil.getMutationValue(mutation, PhoenixDatabaseMetaData.ORDINAL_POSITION_BYTES,
 +                            GenericKeyValueBuilder.INSTANCE, ptr);
 +                        if (MetaDataUtil.getMutationValue(mutation, PhoenixDatabaseMetaData.ORDINAL_POSITION_BYTES,
 +                            GenericKeyValueBuilder.INSTANCE, ptr)) {
 +                            int ordinalValue = PInteger.INSTANCE.getCodec().decodeInt(ptr, SortOrder.ASC);
 +                            if (ordinalValue <= baseColumnCount) {
 +                                mutationIterator.remove();
 +                            } else {
 +                                if (mutation instanceof Put) {
 +                                    byte[] ordinalPositionBytes = new byte[PInteger.INSTANCE.getByteSize()];
 +                                    int newOrdinalValue = ordinalValue - baseColumnCount;
 +                                    PInteger.INSTANCE.getCodec()
 +                                        .encodeInt(newOrdinalValue, ordinalPositionBytes, 0);
 +                                    byte[] family = Iterables.getOnlyElement(mutation.getFamilyCellMap().keySet());
 +                                    MetaDataUtil.mutatePutValue((Put) mutation, family, PhoenixDatabaseMetaData.ORDINAL_POSITION_BYTES, ordinalPositionBytes);
 +                                }
 +                            }
 +                        }
 +                    }
 +                }
 +            }
 +
              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
@@@ -1662,10 -1492,30 +1759,30 @@@
                   * For an index we lock the parent table's row which could be a physical table or a view.
                   * If the parent table is a physical table, then the tenantIdBytes is empty because
                   * we allow creating an index with a tenant connection only if the parent table is a view.
 -                 */ 
 +                 */
                  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();
@@@ -1933,57 -1765,89 +2050,23 @@@
                  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));
 -    private static final byte[] CHILD_TABLE_BYTES = new byte[] {PTable.LinkType.CHILD_TABLE.getSerializedValue()};
 -
 -    
 -    private void findAllChildViews(Region region, byte[] tenantId, PTable table,
 -            TableViewFinder result, long clientTimeStamp, int clientVersion) throws IOException, SQLException {
 -        TableViewFinder currResult = findChildViews(region, tenantId, table, clientVersion);
 -        result.addResult(currResult);
 -        for (ViewInfo viewInfo : currResult.getViewInfoList()) {
 -            byte[] viewtenantId = viewInfo.getTenantId();
 -            byte[] viewSchema = viewInfo.getSchemaName();
 -            byte[] viewTable = viewInfo.getViewName();
 -            byte[] tableKey = SchemaUtil.getTableKey(viewtenantId, viewSchema, viewTable);
 -            ImmutableBytesPtr cacheKey = new ImmutableBytesPtr(tableKey);
 -            PTable view = loadTable(env, tableKey, cacheKey, clientTimeStamp, clientTimeStamp, clientVersion);
 -            findAllChildViews(region, viewtenantId, view, result, clientTimeStamp, clientVersion);
--        }
-         locks.add(rowLock);
-         return rowLock;
--    }
- 
-     private static void printMutations(List<Mutation> mutations) {
-         for (Mutation mutation : mutations) {
-             if (mutation instanceof Put) {
-                 Put put = (Put) mutation;
-                 NavigableMap<byte[], List<Cell>> familyCellMap = put.getFamilyCellMap();
-                 for (List<Cell> cells : familyCellMap.values()) {
-                     StringBuilder builder = new StringBuilder();
-                     for (Cell cell : cells) {
-                         // print the rowkey
-                         builder.append("ROW_KEY: " + Bytes.toStringBinary(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength()));
-                         builder.append("\t");
-                         builder.append("QUALIFIER: "+  Bytes
-                             .toStringBinary(cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength()));
-                         builder.append("\t");
-                         builder.append("VALUE: " + Bytes
-                             .toStringBinary(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength()));
-                         builder.append("\n");
-                         System.out.println(builder.toString());
 -        
 -    // TODO remove this in 4.13 release 
 -    @Deprecated
 -    private TableViewFinder findChildViews_deprecated(Region region, byte[] tenantId, PTable table, byte[] linkTypeBytes) throws IOException {
 -        byte[] schemaName = table.getSchemaName().getBytes();
 -        byte[] tableName = table.getTableName().getBytes();
 -        boolean isMultiTenant = table.isMultiTenant();
 -        Scan scan = new Scan();
 -        // If the table is multi-tenant, we need to check across all tenant_ids,
 -        // so we can't constrain the row key. Otherwise, any views would have
 -        // the same tenantId.
 -        if (!isMultiTenant) {
 -            byte[] startRow = ByteUtil.concat(tenantId, QueryConstants.SEPARATOR_BYTE_ARRAY);
 -            byte[] stopRow = ByteUtil.nextKey(startRow);
 -            scan.setStartRow(startRow);
 -            scan.setStopRow(stopRow);
 -        }
 -        SingleColumnValueFilter linkFilter = new SingleColumnValueFilter(TABLE_FAMILY_BYTES, LINK_TYPE_BYTES, CompareOp.EQUAL, linkTypeBytes);
 -        SingleColumnValueFilter tableTypeFilter = new SingleColumnValueFilter(TABLE_FAMILY_BYTES, TABLE_TYPE_BYTES,
 -                CompareOp.EQUAL, PTableType.VIEW.getSerializedValue().getBytes());
 -        tableTypeFilter.setFilterIfMissing(false);
 -        linkFilter.setFilterIfMissing(true);
 -        byte[] suffix = ByteUtil.concat(QueryConstants.SEPARATOR_BYTE_ARRAY, SchemaUtil
 -                .getPhysicalHBaseTableName(schemaName, tableName, table.isNamespaceMapped())
 -                .getBytes());
 -        SuffixFilter rowFilter = new SuffixFilter(suffix);
 -        FilterList filter = new FilterList(linkFilter,tableTypeFilter,rowFilter);
 -        scan.setFilter(filter);
 -        scan.addColumn(TABLE_FAMILY_BYTES, LINK_TYPE_BYTES);
 -        scan.addColumn(TABLE_FAMILY_BYTES, TABLE_TYPE_BYTES);
 -        scan.addColumn(TABLE_FAMILY_BYTES, TABLE_SEQ_NUM_BYTES);
 -        
 -        // Original region-only scanner modified due to PHOENIX-1208
 -        // RegionScanner scanner = region.getScanner(scan);
 -        // The following *should* work, but doesn't due to HBASE-11837
 -        // TableName systemCatalogTableName = region.getTableDesc().getTableName();
 -        // HTableInterface hTable = env.getTable(systemCatalogTableName);
 -        // These deprecated calls work around the issue
 -        try (HTableInterface hTable = ServerUtil.getHTableForCoprocessorScan(env,
 -            region.getTableDesc().getTableName().getName())) {
 -            boolean allViewsInCurrentRegion = true;
 -            int numOfChildViews = 0;
 -            List<ViewInfo> viewInfoList = Lists.newArrayList();
 -            try (ResultScanner scanner = hTable.getScanner(scan)) {
 -                for (Result result = scanner.next(); (result != null); result = scanner.next()) {
 -                    numOfChildViews++;
 -                    ImmutableBytesWritable ptr = new ImmutableBytesWritable();
 -                    ResultTuple resultTuple = new ResultTuple(result);
 -                    resultTuple.getKey(ptr);
 -                    byte[] key = ptr.copyBytes();
 -                    if (checkTableKeyInRegion(key, region) != null) {
 -                        allViewsInCurrentRegion = false;
--                    }
 -                    byte[][] rowKeyMetaData = new byte[3][];
 -                    getVarChars(result.getRow(), 3, rowKeyMetaData);
 -                    byte[] viewTenantId = rowKeyMetaData[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
 -                    byte[] viewSchemaName = rowKeyMetaData[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX];
 -                    byte[] viewName = rowKeyMetaData[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
 -                    viewInfoList.add(new ViewInfo(viewTenantId, viewSchemaName, viewName));
 -                }
 -                TableViewFinder tableViewFinderResult = new TableViewFinder(viewInfoList);
 -                if (numOfChildViews > 0 && !allViewsInCurrentRegion) {
 -                    tableViewFinderResult.setAllViewsNotInSingleRegion();
--                }
 -                return tableViewFinderResult;
--            }
-         }
-     }
-     
 +    private void findAncestorViewsOfIndex(byte[] tenantId, byte[] schemaName, byte[] indexName, TableViewFinderResult result) throws IOException {
 +        HTableInterface hTable = env.getTable(SchemaUtil
 +                .getPhysicalTableName(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES, env.getConfiguration()));
 +        try {
 +            TableViewFinderResult currentResult = ViewFinder.findParentViewofIndex(hTable, tenantId, schemaName, indexName);
 +//            currentResult.addResult(ViewFinder.findBaseTable(hTable, tenantId, schemaName, indexName));
 +//            if ( currentResult.getResults().size()!=1 ) {
 +//                throw new RuntimeException("View index should have exactly one parent");
 +//            }
 +            if (currentResult.getResults().size()==1) {
 +            	result.addResult(currentResult);
 +            	TableInfo tableInfo = currentResult.getResults().get(0);
 +            	findAncestorViews(tableInfo.getTenantId(), tableInfo.getSchemaName(), tableInfo.getTableName(), result);
 +            }
 +            // else this is an index on a regular table and so we don't need to combine columns
 +        } finally {
 +            hTable.close();
          }
      }
      
@@@ -2056,6 -1953,23 +2139,23 @@@
                      parentTableName == null ? lockKey : SchemaUtil.getTableKey(tenantIdBytes,
                          schemaName, tableName);
  
+             
+             PTableType ptableType=PTableType.fromSerializedValue(tableType);
 -            long clientTimeStamp = MetaDataUtil.getClientTimeStamp(tableMetadata);
++            long clientTimeStamp = MetaDataUtil.getClientTimeStamp(catalogMutations);
+             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) {
@@@ -2077,20 -1992,11 +2177,20 @@@
                      done.run(MetaDataMutationResult.toProto(result));
                      return;
                  }
 -                Cache<ImmutableBytesPtr,PMetaDataEntity> metaDataCache = GlobalCache.getInstance(this.env).getMetaDataCache();
 -                // Commit the list of deletion.
 -                mutateRowsWithLocks(region, tableMetadata, Collections.<byte[]> emptySet(), HConstants.NO_NONCE,
 -                    HConstants.NO_NONCE);
 -                long currentTime = MetaDataUtil.getClientTimeStamp(tableMetadata);
 +				Cache<ImmutableBytesPtr, PMetaDataEntity> metaDataCache = GlobalCache.getInstance(this.env)
 +						.getMetaDataCache();
 +				// since the mutations in catalogMutations can span multiple
 +				// regions first we first process process mutations local to
 +				// this region, then we process the remaining mutations, finally
 +				// we process the child link mutations if any of the mutations
 +				// fail, we can will clean them up later using
 +				// OrphanCleaner.reapOrphans()
 +				separateLocalAndRemoteMutations(region, catalogMutations, localRegionMutations, remoteRegionMutations);
 +				// drop rows from catalog on this region
- 				region.mutateRowsWithLocks(localRegionMutations, Collections.<byte[]> emptyList(), HConstants.NO_NONCE,
++				mutateRowsWithLocks(region, localRegionMutations, Collections.<byte[]> emptySet(), HConstants.NO_NONCE,
 +						HConstants.NO_NONCE);
 +
 +                long currentTime = MetaDataUtil.getClientTimeStamp(catalogMutations);
                  for (ImmutableBytesPtr ckey : invalidateList) {
                      metaDataCache.put(ckey, newDeletedTableMarker(currentTime));
                  }
@@@ -2101,11 -2007,7 +2201,11 @@@
                  done.run(MetaDataMutationResult.toProto(result));
                  return;
              } finally {
-                 region.releaseRowLocks(locks);
 -                releaseRowLocks(region,locks);
++                releaseRowLocks(region, locks);
 +                // drop rows from catalog on remote regions
 +                processMutations(controller, PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES, SchemaUtil.getTableName(schemaName, tableName), remoteRegionMutations);
 +                // drop all child links 
 +                processMutations(controller, PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_NAME_BYTES, SchemaUtil.getTableName(schemaName, tableName), childLinkMutations);
              }
          } catch (Throwable t) {
            logger.error("dropTable failed", t);
@@@ -2113,29 -2015,32 +2213,46 @@@
                  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,
 -        List<ImmutableBytesPtr> invalidateList, List<RowLock> locks,
 -        List<byte[]> tableNamesToDelete, List<SharedTableState> sharedTablesToDelete, boolean isCascade, int clientVersion) throws IOException, SQLException {
 -
 +	private void processMutations(RpcController controller, byte[] systemTableName, String droppedTableName,
 +			List<Mutation> childLinkMutations) throws IOException {
 +		HTableInterface hTable = null;
 +		try {
 +			hTable = env.getTable(SchemaUtil.getPhysicalTableName(systemTableName, env.getConfiguration()));
 +			hTable.batch(childLinkMutations);
 +		} catch (Throwable t) {
 +			logger.error("dropTable failed", t);
 +			ProtobufUtil.setControllerException(controller, ServerUtil.createIOException(droppedTableName, t));
 +		} finally {
 +			if (hTable != null) {
 +				hTable.close();
 +			}
 +		}
 +	}
  
 -        long clientTimeStamp = MetaDataUtil.getClientTimeStamp(rowsToDelete);
 +	private MetaDataMutationResult doDropTable(byte[] key, byte[] tenantId, byte[] schemaName, byte[] tableName,
 +			byte[] parentTableName, PTableType tableType, List<Mutation> catalogMutations,
 +			List<Mutation> childLinkMutations, List<ImmutableBytesPtr> invalidateList, List<byte[]> tableNamesToDelete,
 +			List<SharedTableState> sharedTablesToDelete, boolean isCascade, int clientVersion)
 +			throws IOException, SQLException {
 +        long clientTimeStamp = MetaDataUtil.getClientTimeStamp(catalogMutations);
  
          Region region = env.getRegion();
          ImmutableBytesPtr cacheKey = new ImmutableBytesPtr(key);
@@@ -2862,15 -3108,18 +2976,20 @@@
                      byte[] schemaName = rowKeyMetaData[SCHEMA_NAME_INDEX];
                      byte[] tableName = rowKeyMetaData[TABLE_NAME_INDEX];
                      PTableType type = table.getType();
 +                    table = combineColumns(table, tenantId, schemaName, tableName, HConstants.LATEST_TIMESTAMP, request.getClientVersion()).getFirst();
                      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) {
 -                        TableViewFinder childViewsResult = new TableViewFinder();
 -                        findAllChildViews(region, tenantId, table, childViewsResult, clientTimeStamp, request.getClientVersion());
 +                        TableViewFinderResult childViewsResult = new TableViewFinderResult();
 +                        findAllChildViews(tenantId, table.getSchemaName().getBytes(), table.getTableName().getBytes(), childViewsResult);
                          if (childViewsResult.hasViews()) {
                              /* 
                               * Dis-allow if:
@@@ -3015,28 -3265,36 +3134,25 @@@
           * from getting rebuilt too often.
           */
          final boolean wasLocked = (rowLock != null);
 -        boolean blockWriteRebuildIndex = env.getConfiguration().getBoolean(QueryServices.INDEX_FAILURE_BLOCK_WRITE, 
 +        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);
 -            // We only cache the latest, so we'll end up building the table with every call if the
 -            // client connection has specified an SCN.
 -            // TODO: If we indicate to the client that we're returning an older version, but there's a
 -            // newer version available, the client
 -            // can safely not call this, since we only allow modifications to the latest.
 -            if (table != null && table.getTimeStamp() < clientTimeStamp) {
 -                // Table on client is up-to-date with table on server, so just return
 -                if (isTableDeleted(table)) {
 -                    return null;
 -                }
 -                return table;
 -            }
 -            // Try cache again in case we were waiting on a lock
 -            table = (PTable)metaDataCache.getIfPresent(cacheKey);
 -            // We only cache the latest, so we'll end up building the table with every call if the
 -            // client connection has specified an SCN.
 -            // TODO: If we indicate to the client that we're returning an older version, but there's
 -            // a newer version available, the client
 -            // can safely not call this, since we only allow modifications to the latest.
 -            if (table != null && table.getTimeStamp() < clientTimeStamp) {
 -                // Table on client is up-to-date with table on server, so just return
 -                if (isTableDeleted(table)) {
 -                    return null;
 +            PTable table = getCachedTable(clientTimeStamp, cacheKey, metaDataCache);
 +            if (table == null) {
 +                // Try cache again in case we were waiting on a lock
 +                table = getCachedTable(clientTimeStamp, cacheKey, metaDataCache);
 +                if (table == null) {
 +                    // Query for the latest table first, since it's not cached
 +                    table = buildTable(key, cacheKey, region, HConstants.LATEST_TIMESTAMP, clientVersion);
 +                    if ((table == null || table.getTimeStamp() >= clientTimeStamp) && (!blockWriteRebuildIndex
 +                        || table.getIndexDisableTimestamp() <= 0)) {
 +                        // Otherwise, query for an older version of the table - it won't be cached
 +                        table = buildTable(key, cacheKey, region, clientTimeStamp, clientVersion);
 +
 +                    }
                  }
                  return table;
              }
@@@ -3124,27 -3358,9 +3234,24 @@@
              if(functionsAvailable.size() == numFunctions) return functionsAvailable;
              return null;
          } finally {
-             for (Region.RowLock lock : rowLocks) {
-                 lock.release();
-             }
-             rowLocks.clear();
+             releaseRowLocks(region,rowLocks);
          }
      }
 +    
 +    private PColumn getColumn(int pkCount, byte[][] rowKeyMetaData, PTable table) throws ColumnFamilyNotFoundException, ColumnNotFoundException {
 +    	PColumn col = null;
 +        if (pkCount > FAMILY_NAME_INDEX
 +            && rowKeyMetaData[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX].length > 0) {
 +            PColumnFamily family =
 +                table.getColumnFamily(rowKeyMetaData[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX]);
 +            col =
 +                family.getPColumnForColumnNameBytes(rowKeyMetaData[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX]);
 +        } else if (pkCount > COLUMN_NAME_INDEX
 +            && rowKeyMetaData[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX].length > 0) {
 +            col = table.getPKColumn(new String(rowKeyMetaData[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX]));
 +        }
 +    	return col;
 +    }
  
      @Override
      public void dropColumn(RpcController controller, final DropColumnRequest request,
@@@ -3164,42 -3379,56 +3271,47 @@@
                      byte[] tenantId = rowKeyMetaData[TENANT_ID_INDEX];
                      byte[] schemaName = rowKeyMetaData[SCHEMA_NAME_INDEX];
                      byte[] tableName = rowKeyMetaData[TABLE_NAME_INDEX];
 +                    table = combineColumns(table, tenantId, schemaName, tableName, clientTimeStamp, request.getClientVersion()).getFirst();
 +                    boolean isView = table.getType() == PTableType.VIEW;
                      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();
 -                    if (type == PTableType.TABLE || type == PTableType.SYSTEM) {
 -                        TableViewFinder childViewsResult = new TableViewFinder();
 -                        findAllChildViews(region, tenantId, table, childViewsResult, clientTimeStamp, request.getClientVersion());
 -                        if (childViewsResult.hasViews()) {
 -                            MetaDataMutationResult mutationResult =
 -                                    dropColumnsFromChildViews(region, table,
 -                                        locks, tableMetaData, additionalTableMetaData,
 -                                        schemaName, tableName, invalidateList,
 -                                        clientTimeStamp, childViewsResult, tableNamesToDelete, sharedTablesToDelete, request.getClientVersion());
 -                            // return if we were not able to drop the column successfully
 -                            if (mutationResult != null) return mutationResult;
 -                        }
 -                    }
 -                    
 -                    for (Mutation m : tableMetaData) {
 -                        if (m instanceof Delete) {
 -                            byte[] key = m.getRow();
 -                            int pkCount = getVarChars(key, rowKeyMetaData);
 +                    ListIterator<Mutation> iterator = tableMetaData.listIterator();
 +                    while (iterator.hasNext()) {
 +                        Mutation mutation = iterator.next();
 +                        byte[] key = mutation.getRow();
 +                        int pkCount = getVarChars(key, rowKeyMetaData);
 +                        if (isView && mutation instanceof Put) {
 +                        	PColumn column = getColumn(pkCount, rowKeyMetaData, table);
 +							if (column == null)
 +								continue;
 +                        	// ignore any puts that modify the ordinal positions of columns
 +                        	iterator.remove();
 +                        } 
 +                        else if (mutation instanceof Delete) {
                              if (pkCount > COLUMN_NAME_INDEX
 -                                    && Bytes.compareTo(schemaName,
 -                                        rowKeyMetaData[SCHEMA_NAME_INDEX]) == 0
 -                                    && Bytes.compareTo(tableName, rowKeyMetaData[TABLE_NAME_INDEX]) == 0) {
 +                                && Bytes.compareTo(schemaName, rowKeyMetaData[SCHEMA_NAME_INDEX]) == 0
 +                                && Bytes.compareTo(tableName, rowKeyMetaData[TABLE_NAME_INDEX]) == 0) {
                                  PColumn columnToDelete = null;
                                  try {
 -                                    if (pkCount > FAMILY_NAME_INDEX
 -                                            && rowKeyMetaData[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX].length > 0) {
 -                                        PColumnFamily family =
 -                                                table.getColumnFamily(rowKeyMetaData[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX]);
 -                                        columnToDelete =
 -                                                family.getPColumnForColumnNameBytes(rowKeyMetaData[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX]);
 -                                    } else if (pkCount > COLUMN_NAME_INDEX
 -                                            && rowKeyMetaData[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX].length > 0) {
 -                                        deletePKColumn = true;
 -                                        columnToDelete = table.getPKColumn(new String(
 -                                          rowKeyMetaData[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX]));
 -                                    } else {
 -                                        continue;
 -                                    }
 -                                    if (table.getType() == PTableType.VIEW) {
 +                                	columnToDelete = getColumn(pkCount, rowKeyMetaData, table);
 +									if (columnToDelete == null)
 +										continue;
 +                                    deletePKColumn = columnToDelete.getFamilyName() == null;
 +									if (isView) {
 +                                        // if we are dropping a derived column add it to the excluded column list
 +                                        if (columnToDelete.isDerived()) {
 +                                            mutation = MetaDataUtil
 +                                                .cloneDeleteToPutAndAddColumn((Delete) mutation, TABLE_FAMILY_BYTES, LINK_TYPE_BYTES, LinkType.EXCLUDED_COLUMN.getSerializedValueAsByteArray());
 +                                            iterator.set(mutation);
 +                                        }
 +
                                          if (table.getBaseColumnCount() != DIVERGED_VIEW_BASE_COLUMN_COUNT
 -                                                && columnToDelete.getPosition() < table.getBaseColumnCount()) {
 +                                            && columnToDelete.isDerived()) {
                                              /*
                                               * If the column being dropped is inherited from the base table, then the
                                               * view is about to diverge itself from the base table. The consequence of

http://git-wip-us.apache.org/repos/asf/phoenix/blob/39c0d1d4/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
----------------------------------------------------------------------
diff --cc phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
index dd619eb,fe11ec7..7b82581
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
@@@ -92,9 -92,9 +92,10 @@@ public abstract class MetaDataProtocol 
      public static final long MIN_SYSTEM_TABLE_TIMESTAMP_4_11_0 = MIN_TABLE_TIMESTAMP + 27;
      // Since there's no upgrade code, keep the version the same as the previous version
      public static final long MIN_SYSTEM_TABLE_TIMESTAMP_4_12_0 = MIN_SYSTEM_TABLE_TIMESTAMP_4_11_0;
-     public static final long MIN_SYSTEM_TABLE_TIMESTAMP_4_13_0 = MIN_TABLE_TIMESTAMP + 28;
+     public static final long MIN_SYSTEM_TABLE_TIMESTAMP_4_13_0 = MIN_SYSTEM_TABLE_TIMESTAMP_4_11_0;
++    public static final long MIN_SYSTEM_TABLE_TIMESTAMP_4_14_0 = MIN_TABLE_TIMESTAMP + 28;
      // MIN_SYSTEM_TABLE_TIMESTAMP needs to be set to the max of all the MIN_SYSTEM_TABLE_TIMESTAMP_* constants
--    public static final long MIN_SYSTEM_TABLE_TIMESTAMP = MIN_SYSTEM_TABLE_TIMESTAMP_4_13_0;
++    public static final long MIN_SYSTEM_TABLE_TIMESTAMP = MIN_SYSTEM_TABLE_TIMESTAMP_4_14_0;
      
      // ALWAYS update this map whenever rolling out a new release (major, minor or patch release). 
      // Key is the SYSTEM.CATALOG timestamp for the version and value is the version string.


[12/50] [abbrv] phoenix git commit: PHOENIX-4386 Calculate the estimatedSize of MutationState using Map> mutations

Posted by td...@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/4944081e
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/4944081e
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/4944081e

Branch: refs/heads/system-catalog
Commit: 4944081ec093204a859d8c9ba57606e4e59bd0fb
Parents: ddd3ef2
Author: Thomas D'Silva <td...@apache.org>
Authored: Fri Nov 17 11:11:43 2017 -0800
Committer: Thomas D'Silva <td...@apache.org>
Committed: Fri Nov 17 15:43:58 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/4944081e/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/4944081e/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/4944081e/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/4944081e/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/4944081e/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 0cdb010..b5a55b8 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/4944081e/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);
+    }
 }


[33/50] [abbrv] phoenix git commit: PHOENIX-4449 Bundle a copy of Argparse-1.4.0 for installations that need it

Posted by td...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cb02da7/bin/sqlline-thin.py
----------------------------------------------------------------------
diff --git a/bin/sqlline-thin.py b/bin/sqlline-thin.py
index 47384d8..fecc96c 100755
--- a/bin/sqlline-thin.py
+++ b/bin/sqlline-thin.py
@@ -25,7 +25,14 @@ import sys
 import phoenix_utils
 import atexit
 import urlparse
-import argparse
+
+# import argparse
+try:
+    import argparse
+except ImportError:
+    current_dir = os.path.dirname(os.path.abspath(__file__))
+    sys.path.append(os.path.join(current_dir, 'argparse-1.4.0'))
+    import argparse
 
 global childProc
 childProc = None

http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cb02da7/bin/sqlline.py
----------------------------------------------------------------------
diff --git a/bin/sqlline.py b/bin/sqlline.py
index 7a724de..4a676ee 100755
--- a/bin/sqlline.py
+++ b/bin/sqlline.py
@@ -24,7 +24,14 @@ import subprocess
 import sys
 import phoenix_utils
 import atexit
-import argparse
+
+# import argparse
+try:
+    import argparse
+except ImportError:
+    current_dir = os.path.dirname(os.path.abspath(__file__))
+    sys.path.append(os.path.join(current_dir, 'argparse-1.4.0'))
+    import argparse
 
 global childProc
 childProc = None
@@ -39,8 +46,9 @@ atexit.register(kill_child)
 phoenix_utils.setPath()
 
 parser = argparse.ArgumentParser(description='Launches the Apache Phoenix Client.')
-# Positional argument 'zookeepers' is optional
-parser.add_argument('zookeepers', nargs='?', help='The ZooKeeper quorum string', default='localhost:2181:/hbase')
+# Positional argument 'zookeepers' is optional. The PhoenixDriver will automatically populate
+# this if it's not provided by the user (so, we want to leave a default value of empty)
+parser.add_argument('zookeepers', nargs='?', help='The ZooKeeper quorum string', default='')
 # Positional argument 'sqlfile' is optional
 parser.add_argument('sqlfile', nargs='?', help='A file of SQL commands to execute', default='')
 # Common arguments across sqlline.py and sqlline-thin.py


[03/50] [abbrv] phoenix git commit: Set version to 4.14.0-HBase-1.3-SNAPSHOT after release

Posted by td...@apache.org.
Set version to 4.14.0-HBase-1.3-SNAPSHOT after release


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

Branch: refs/heads/system-catalog
Commit: 2a8e1c750f081f7f020d4321f8d76ae02c074aa5
Parents: 217867c
Author: Mujtaba <mu...@apache.org>
Authored: Fri Nov 10 15:35:46 2017 -0800
Committer: Mujtaba <mu...@apache.org>
Committed: Fri Nov 10 15:35:46 2017 -0800

----------------------------------------------------------------------
 phoenix-assembly/pom.xml           | 2 +-
 phoenix-client/pom.xml             | 2 +-
 phoenix-core/pom.xml               | 2 +-
 phoenix-flume/pom.xml              | 2 +-
 phoenix-hive/pom.xml               | 2 +-
 phoenix-kafka/pom.xml              | 2 +-
 phoenix-load-balancer/pom.xml      | 2 +-
 phoenix-pherf/pom.xml              | 2 +-
 phoenix-pig/pom.xml                | 2 +-
 phoenix-queryserver-client/pom.xml | 2 +-
 phoenix-queryserver/pom.xml        | 2 +-
 phoenix-server/pom.xml             | 2 +-
 phoenix-spark/pom.xml              | 2 +-
 phoenix-tracing-webapp/pom.xml     | 2 +-
 pom.xml                            | 2 +-
 15 files changed, 15 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/2a8e1c75/phoenix-assembly/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-assembly/pom.xml b/phoenix-assembly/pom.xml
index 8ec4ebb..2ce4871 100644
--- a/phoenix-assembly/pom.xml
+++ b/phoenix-assembly/pom.xml
@@ -27,7 +27,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.13.0-HBase-1.3</version>
+    <version>4.14.0-HBase-1.3-SNAPSHOT</version>
   </parent>
   <artifactId>phoenix-assembly</artifactId>
   <name>Phoenix Assembly</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2a8e1c75/phoenix-client/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-client/pom.xml b/phoenix-client/pom.xml
index 77df4ca..c209435 100644
--- a/phoenix-client/pom.xml
+++ b/phoenix-client/pom.xml
@@ -27,7 +27,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.13.0-HBase-1.3</version>
+    <version>4.14.0-HBase-1.3-SNAPSHOT</version>
   </parent>
   <artifactId>phoenix-client</artifactId>
   <name>Phoenix Client</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2a8e1c75/phoenix-core/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-core/pom.xml b/phoenix-core/pom.xml
index 936aca3..fbda221 100644
--- a/phoenix-core/pom.xml
+++ b/phoenix-core/pom.xml
@@ -4,7 +4,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.13.0-HBase-1.3</version>
+    <version>4.14.0-HBase-1.3-SNAPSHOT</version>
   </parent>
   <artifactId>phoenix-core</artifactId>
   <name>Phoenix Core</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2a8e1c75/phoenix-flume/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-flume/pom.xml b/phoenix-flume/pom.xml
index bd1cd7e..cd957b2 100644
--- a/phoenix-flume/pom.xml
+++ b/phoenix-flume/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.13.0-HBase-1.3</version>
+    <version>4.14.0-HBase-1.3-SNAPSHOT</version>
   </parent>
   <artifactId>phoenix-flume</artifactId>
   <name>Phoenix - Flume</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2a8e1c75/phoenix-hive/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-hive/pom.xml b/phoenix-hive/pom.xml
index a2531e0..0e55812 100644
--- a/phoenix-hive/pom.xml
+++ b/phoenix-hive/pom.xml
@@ -27,7 +27,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.13.0-HBase-1.3</version>
+    <version>4.14.0-HBase-1.3-SNAPSHOT</version>
   </parent>
   <artifactId>phoenix-hive</artifactId>
   <name>Phoenix - Hive</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2a8e1c75/phoenix-kafka/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-kafka/pom.xml b/phoenix-kafka/pom.xml
index 610bb01..a1b7251 100644
--- a/phoenix-kafka/pom.xml
+++ b/phoenix-kafka/pom.xml
@@ -26,7 +26,7 @@
 	<parent>
 		<groupId>org.apache.phoenix</groupId>
 		<artifactId>phoenix</artifactId>
-		<version>4.13.0-HBase-1.3</version>
+		<version>4.14.0-HBase-1.3-SNAPSHOT</version>
 	</parent>
 	<artifactId>phoenix-kafka</artifactId>
 	<name>Phoenix - Kafka</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2a8e1c75/phoenix-load-balancer/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-load-balancer/pom.xml b/phoenix-load-balancer/pom.xml
index 7b819e1..8aa2b04 100644
--- a/phoenix-load-balancer/pom.xml
+++ b/phoenix-load-balancer/pom.xml
@@ -27,7 +27,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.13.0-HBase-1.3</version>
+    <version>4.14.0-HBase-1.3-SNAPSHOT</version>
   </parent>
   <artifactId>phoenix-load-balancer</artifactId>
   <name>Phoenix Load Balancer</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2a8e1c75/phoenix-pherf/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-pherf/pom.xml b/phoenix-pherf/pom.xml
index edd9a27..304469e 100644
--- a/phoenix-pherf/pom.xml
+++ b/phoenix-pherf/pom.xml
@@ -15,7 +15,7 @@
 	<parent>
 		<groupId>org.apache.phoenix</groupId>
 		<artifactId>phoenix</artifactId>
-		<version>4.13.0-HBase-1.3</version>
+		<version>4.14.0-HBase-1.3-SNAPSHOT</version>
 	</parent>
 
 	<artifactId>phoenix-pherf</artifactId>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2a8e1c75/phoenix-pig/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-pig/pom.xml b/phoenix-pig/pom.xml
index aa86fc9..fa512e2 100644
--- a/phoenix-pig/pom.xml
+++ b/phoenix-pig/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.13.0-HBase-1.3</version>
+    <version>4.14.0-HBase-1.3-SNAPSHOT</version>
   </parent>
   <artifactId>phoenix-pig</artifactId>
   <name>Phoenix - Pig</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2a8e1c75/phoenix-queryserver-client/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-queryserver-client/pom.xml b/phoenix-queryserver-client/pom.xml
index 1c23e25..6c21cc7 100644
--- a/phoenix-queryserver-client/pom.xml
+++ b/phoenix-queryserver-client/pom.xml
@@ -27,7 +27,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.13.0-HBase-1.3</version>
+    <version>4.14.0-HBase-1.3-SNAPSHOT</version>
   </parent>
   <artifactId>phoenix-queryserver-client</artifactId>
   <name>Phoenix Query Server Client</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2a8e1c75/phoenix-queryserver/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-queryserver/pom.xml b/phoenix-queryserver/pom.xml
index dd923df..2cb33fe 100644
--- a/phoenix-queryserver/pom.xml
+++ b/phoenix-queryserver/pom.xml
@@ -26,7 +26,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.13.0-HBase-1.3</version>
+    <version>4.14.0-HBase-1.3-SNAPSHOT</version>
   </parent>
   <artifactId>phoenix-queryserver</artifactId>
   <name>Phoenix Query Server</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2a8e1c75/phoenix-server/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-server/pom.xml b/phoenix-server/pom.xml
index ba9524e..e67927e 100644
--- a/phoenix-server/pom.xml
+++ b/phoenix-server/pom.xml
@@ -27,7 +27,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.13.0-HBase-1.3</version>
+    <version>4.14.0-HBase-1.3-SNAPSHOT</version>
   </parent>
   <artifactId>phoenix-server</artifactId>
   <name>Phoenix Server</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2a8e1c75/phoenix-spark/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-spark/pom.xml b/phoenix-spark/pom.xml
index e4d14c4..f9a58cf 100644
--- a/phoenix-spark/pom.xml
+++ b/phoenix-spark/pom.xml
@@ -28,7 +28,7 @@
   <parent>
     <groupId>org.apache.phoenix</groupId>
     <artifactId>phoenix</artifactId>
-    <version>4.13.0-HBase-1.3</version>
+    <version>4.14.0-HBase-1.3-SNAPSHOT</version>
   </parent>
   <artifactId>phoenix-spark</artifactId>
   <name>Phoenix - Spark</name>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2a8e1c75/phoenix-tracing-webapp/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-tracing-webapp/pom.xml b/phoenix-tracing-webapp/pom.xml
index 79f0a86..1b944c6 100755
--- a/phoenix-tracing-webapp/pom.xml
+++ b/phoenix-tracing-webapp/pom.xml
@@ -27,7 +27,7 @@
     <parent>
       <groupId>org.apache.phoenix</groupId>
       <artifactId>phoenix</artifactId>
-      <version>4.13.0-HBase-1.3</version>
+      <version>4.14.0-HBase-1.3-SNAPSHOT</version>
     </parent>
 
     <artifactId>phoenix-tracing-webapp</artifactId>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2a8e1c75/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index e393993..2327c80 100644
--- a/pom.xml
+++ b/pom.xml
@@ -3,7 +3,7 @@
   <modelVersion>4.0.0</modelVersion>
   <groupId>org.apache.phoenix</groupId>
   <artifactId>phoenix</artifactId>
-  <version>4.13.0-HBase-1.3</version>
+  <version>4.14.0-HBase-1.3-SNAPSHOT</version>
   <packaging>pom</packaging>
   <name>Apache Phoenix</name>
   <description>A SQL layer over HBase</description>


[40/50] [abbrv] phoenix git commit: PHOENIX-4466 Relocate Avatica and hadoop-common in thin-client jar (Toshihiro Suzuki)

Posted by td...@apache.org.
PHOENIX-4466 Relocate Avatica and hadoop-common in thin-client jar (Toshihiro Suzuki)

When using the thin-client in Spark, we encounter problems in that Spark
is placing its own version of avatica on the classpath as well. We can
relocate most of Avatica (all but the protobuf generated messages as
their classnames are required to be 'org.apache.calcite.avatica.proto'
presently) and hadoop-common to avoid future problems.


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

Branch: refs/heads/system-catalog
Commit: 34693843abe4490b54fbd30512bf7d98d0f59c0d
Parents: 412329a
Author: Josh Elser <el...@apache.org>
Authored: Fri Dec 22 13:28:12 2017 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Fri Dec 22 13:28:12 2017 -0500

----------------------------------------------------------------------
 phoenix-queryserver-client/pom.xml | 16 +++++++++++++---
 1 file changed, 13 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/34693843/phoenix-queryserver-client/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-queryserver-client/pom.xml b/phoenix-queryserver-client/pom.xml
index 6c21cc7..486d89b 100644
--- a/phoenix-queryserver-client/pom.xml
+++ b/phoenix-queryserver-client/pom.xml
@@ -123,9 +123,19 @@
                   <shadedPattern>${shaded.package}.com.google.protobuf</shadedPattern>
                 </relocation>
                 <!-- ORG relocation -->
-                <!-- Calcite/Avatica is not relocated because the wire API (as of <=1.8.0) expects
-                     consistent class names on client and server. Relocating these would break
-                     backwards compatibility. -->
+                <relocation>
+                  <pattern>org.apache.calcite.avatica</pattern>
+                  <shadedPattern>${shaded.package}.org.apache.calcite.avatica</shadedPattern>
+                  <!-- The protobuf messages can't be relocated due to a limitation
+                       in the Avatica protocol. -->
+                  <excludes>
+                    <exclude>org.apache.calcite.avatica.proto.*</exclude>
+                  </excludes>
+                </relocation>
+                <relocation>
+                  <pattern>org.apache.hadoop</pattern>
+                  <shadedPattern>${shaded.package}.org.apache.hadoop</shadedPattern>
+                </relocation>
                 <relocation>
                   <pattern>org.apache.commons</pattern>
                   <shadedPattern>${shaded.package}.org.apache.commons</shadedPattern>


[50/50] [abbrv] phoenix git commit: Merge branch 'PHOENIX-3534' into system-catalog

Posted by td...@apache.org.
Merge branch 'PHOENIX-3534' into system-catalog


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

Branch: refs/heads/system-catalog
Commit: 15d4dace4169009ac4fd11261d6c8cc72905fcec
Parents: ae359fe 41f8142
Author: Thomas D'Silva <td...@apache.org>
Authored: Fri Jan 19 10:00:46 2018 -0800
Committer: Thomas D'Silva <td...@apache.org>
Committed: Fri Jan 19 10:00:46 2018 -0800

----------------------------------------------------------------------
 .gitignore                                      |    2 +
 LICENSE                                         |   43 +-
 NOTICE                                          |    2 +
 bin/argparse-1.4.0/argparse.py                  | 2392 ++++++++++++
 bin/daemon.py                                   |   16 +-
 bin/queryserver.py                              |    2 +-
 bin/sqlline-thin.py                             |    9 +-
 bin/sqlline.py                                  |   14 +-
 dev/make_rc.sh                                  |   26 +-
 dev/release_files/LICENSE                       |    2 +
 dev/release_files/NOTICE                        |    8 +
 dev/test-patch.properties                       |    7 +-
 dev/test-patch.sh                               |   66 +-
 phoenix-assembly/pom.xml                        |    2 +-
 phoenix-client/pom.xml                          |    2 +-
 phoenix-core/pom.xml                            |   19 +-
 .../wal/ReadWriteKeyValuesWithCodecIT.java      |  184 -
 .../wal/WALRecoveryRegionPostOpenIT.java        |  330 ++
 ...ReplayWithIndexWritesAndCompressedWALIT.java |   18 +-
 .../org/apache/phoenix/end2end/AggregateIT.java | 1012 +++++
 .../phoenix/end2end/AggregateQueryIT.java       |   23 +-
 .../AlterMultiTenantTableWithViewsIT.java       |  118 +-
 .../apache/phoenix/end2end/AlterTableIT.java    | 1249 +------
 .../phoenix/end2end/AlterTableWithViewsIT.java  |  223 +-
 .../org/apache/phoenix/end2end/Array1IT.java    | 1004 +++++
 .../org/apache/phoenix/end2end/Array2IT.java    |  815 +++++
 .../org/apache/phoenix/end2end/Array3IT.java    |  770 ++++
 .../phoenix/end2end/ArrayAppendFunctionIT.java  |   42 +-
 .../phoenix/end2end/ArrayConcatFunctionIT.java  |   39 +-
 .../org/apache/phoenix/end2end/ArrayIT.java     | 2731 +-------------
 .../phoenix/end2end/AutoPartitionViewsIT.java   |   22 +-
 .../phoenix/end2end/AutomaticRebuildIT.java     |  219 --
 .../end2end/BaseClientManagedTimeIT.java        |   79 -
 .../org/apache/phoenix/end2end/BaseJoinIT.java  |  456 ---
 .../phoenix/end2end/BasePermissionsIT.java      |  754 ++++
 .../org/apache/phoenix/end2end/BaseQueryIT.java |  169 +-
 .../end2end/BaseUniqueNamesOwnClusterIT.java    |    5 -
 .../apache/phoenix/end2end/CaseStatementIT.java |   56 +-
 .../apache/phoenix/end2end/CastAndCoerceIT.java |   20 +-
 .../phoenix/end2end/ChangePermissionsIT.java    |  270 ++
 .../phoenix/end2end/ClientManagedTimeTest.java  |   42 -
 .../end2end/ClientTimeArithmeticQueryIT.java    |  808 ----
 .../phoenix/end2end/CoalesceFunctionIT.java     |   76 +-
 .../phoenix/end2end/CollationKeyFunctionIT.java |  181 +
 ...mnEncodedImmutableNonTxStatsCollectorIT.java |   39 +
 ...olumnEncodedImmutableTxStatsCollectorIT.java |   38 +
 ...lumnEncodedMutableNonTxStatsCollectorIT.java |   38 +
 .../ColumnEncodedMutableTxStatsCollectorIT.java |   38 +
 .../end2end/ColumnProjectionOptimizationIT.java |   70 +-
 .../phoenix/end2end/ConcurrentMutationsIT.java  |  730 ++++
 .../ConnectionQueryServicesTestImpl.java        |    4 +-
 .../phoenix/end2end/ConnectionUtilIT.java       |   10 -
 .../phoenix/end2end/ContextClassloaderIT.java   |    9 -
 .../phoenix/end2end/CostBasedDecisionIT.java    |  466 +++
 .../CountDistinctApproximateHyperLogLogIT.java  |  154 +
 .../apache/phoenix/end2end/CreateSchemaIT.java  |   86 +-
 .../apache/phoenix/end2end/CreateTableIT.java   |  924 +++--
 .../phoenix/end2end/CsvBulkLoadToolIT.java      |   38 +
 .../CursorWithRowValueConstructorIT.java        |  672 ++++
 .../phoenix/end2end/CustomEntityDataIT.java     |   75 +-
 .../phoenix/end2end/DateArithmeticIT.java       |  309 ++
 .../org/apache/phoenix/end2end/DateTimeIT.java  |  347 +-
 .../phoenix/end2end/DefaultColumnValueIT.java   |   10 +-
 .../org/apache/phoenix/end2end/DeleteIT.java    |  216 +-
 .../apache/phoenix/end2end/DerivedTableIT.java  |  340 +-
 .../apache/phoenix/end2end/DistinctCountIT.java |  118 +-
 .../apache/phoenix/end2end/DropSchemaIT.java    |   80 +-
 .../phoenix/end2end/EncodeFunctionIT.java       |   20 +
 .../phoenix/end2end/ExecuteStatementsIT.java    |    4 +-
 .../end2end/ExplainPlanWithStatsDisabledIT.java |  259 ++
 .../end2end/ExplainPlanWithStatsEnabledIT.java  | 1107 ++++++
 .../phoenix/end2end/ExtendedQueryExecIT.java    |   30 +-
 .../phoenix/end2end/FirstValuesFunctionIT.java  |  638 ++++
 .../phoenix/end2end/FlappingAlterTableIT.java   |    4 +-
 .../phoenix/end2end/FlappingLocalIndexIT.java   |   79 +-
 .../apache/phoenix/end2end/FunkyNamesIT.java    |   47 +-
 .../apache/phoenix/end2end/GroupByCaseIT.java   |  948 -----
 .../org/apache/phoenix/end2end/GroupByIT.java   |  397 +-
 .../org/apache/phoenix/end2end/HashJoinIT.java  | 3456 ------------------
 .../phoenix/end2end/HashJoinLocalIndexIT.java   |  128 -
 .../apache/phoenix/end2end/HashJoinMoreIT.java  |  909 -----
 .../org/apache/phoenix/end2end/InListIT.java    |   13 +-
 .../org/apache/phoenix/end2end/InQueryIT.java   |  247 ++
 .../apache/phoenix/end2end/IndexExtendedIT.java |  401 +-
 .../phoenix/end2end/IndexScrutinyToolIT.java    |  759 ++++
 .../end2end/IndexToolForPartialBuildIT.java     |   81 +-
 ...olForPartialBuildWithNamespaceEnabledIT.java |   22 +-
 .../org/apache/phoenix/end2end/IndexToolIT.java |  309 ++
 .../apache/phoenix/end2end/IntArithmeticIT.java |  214 ++
 .../phoenix/end2end/LastValuesFunctionIT.java   |  640 ++++
 .../phoenix/end2end/LikeExpressionIT.java       |  149 +-
 .../phoenix/end2end/LocalIndexSplitMergeIT.java |  266 ++
 .../phoenix/end2end/MetaDataEndPointIT.java     |   55 +
 .../MigrateSystemTablesToSystemNamespaceIT.java |  410 +++
 .../phoenix/end2end/MultiCfQueryExecIT.java     |   69 +
 .../apache/phoenix/end2end/MutableQueryIT.java  |  424 ---
 .../apache/phoenix/end2end/MutationStateIT.java |  161 +
 .../phoenix/end2end/NativeHBaseTypesIT.java     |  151 +-
 ...mnEncodedImmutableNonTxStatsCollectorIT.java |   38 +
 ...olumnEncodedImmutableTxStatsCollectorIT.java |   38 +
 .../org/apache/phoenix/end2end/NotQueryIT.java  |   25 +-
 .../NotQueryWithGlobalImmutableIndexesIT.java   |   42 +
 .../NotQueryWithLocalImmutableIndexesIT.java    |   42 +
 .../phoenix/end2end/NthValueFunctionIT.java     |  154 +
 .../java/org/apache/phoenix/end2end/NullIT.java |  157 +
 .../phoenix/end2end/NumericArithmeticIT.java    |  426 +++
 .../end2end/OctetLengthFunctionEnd2EndIT.java   |    9 +-
 .../phoenix/end2end/OnDuplicateKeyIT.java       |   39 +-
 .../phoenix/end2end/ParallelRunListener.java    |   42 -
 .../end2end/ParallelStatsDisabledIT.java        |    8 +-
 .../phoenix/end2end/ParallelStatsEnabledIT.java |   14 +-
 .../PartialResultServerConfigurationIT.java     |  148 +
 .../PartialScannerResultsDisabledIT.java        |  193 +
 .../apache/phoenix/end2end/PercentileIT.java    |  168 +-
 .../apache/phoenix/end2end/PhoenixDriverIT.java |  160 +
 .../phoenix/end2end/PhoenixRuntimeIT.java       |    1 -
 .../phoenix/end2end/PointInTimeQueryIT.java     |  246 +-
 .../phoenix/end2end/ProductMetricsIT.java       | 1031 ++----
 .../end2end/QueryDatabaseMetaDataIT.java        | 1697 ++++-----
 .../phoenix/end2end/QueryExecWithoutSCNIT.java  |    3 +-
 .../org/apache/phoenix/end2end/QueryIT.java     |  378 +-
 .../org/apache/phoenix/end2end/QueryMoreIT.java |   40 +-
 .../phoenix/end2end/QueryWithOffsetIT.java      |   10 +-
 .../phoenix/end2end/QueryWithTableSampleIT.java |  305 ++
 .../org/apache/phoenix/end2end/RangeScanIT.java |  281 ++
 .../phoenix/end2end/ReadIsolationLevelIT.java   |   46 +-
 .../end2end/RebuildIndexConnectionPropsIT.java  |  130 +
 .../phoenix/end2end/RoundFloorCeilFuncIT.java   |   21 +-
 .../apache/phoenix/end2end/RowTimestampIT.java  |  369 ++
 .../phoenix/end2end/RowValueConstructorIT.java  |  365 +-
 .../org/apache/phoenix/end2end/ScanQueryIT.java |  439 ---
 .../end2end/SequenceBulkAllocationIT.java       |  551 ++-
 .../org/apache/phoenix/end2end/SequenceIT.java  |  973 ++---
 .../apache/phoenix/end2end/SetPropertyIT.java   | 1022 ++++++
 .../end2end/SetPropertyOnEncodedTableIT.java    |   34 +
 .../end2end/SetPropertyOnNonEncodedTableIT.java |   34 +
 .../apache/phoenix/end2end/SkipScanQueryIT.java |   88 +
 .../apache/phoenix/end2end/SortMergeJoinIT.java | 2563 -------------
 .../phoenix/end2end/SortMergeJoinMoreIT.java    |  135 +
 .../org/apache/phoenix/end2end/SortOrderIT.java |   11 +-
 .../phoenix/end2end/SpillableGroupByIT.java     |    2 +-
 .../phoenix/end2end/StatsCollectorIT.java       |  648 ----
 .../apache/phoenix/end2end/StoreNullsIT.java    |   80 +-
 .../phoenix/end2end/StoreNullsPropIT.java       |    2 +-
 .../org/apache/phoenix/end2end/StringIT.java    |   60 +
 .../org/apache/phoenix/end2end/SubqueryIT.java  |  788 ----
 .../end2end/SubqueryUsingSortMergeJoinIT.java   |  566 ---
 ...SysTableNamespaceMappedStatsCollectorIT.java |   21 +-
 .../apache/phoenix/end2end/SystemCatalogIT.java |   96 +
 .../phoenix/end2end/SystemCatalogUpgradeIT.java |  121 +
 .../end2end/SystemTablePermissionsIT.java       |   88 +
 .../phoenix/end2end/TableDDLPermissionsIT.java  |  233 ++
 .../end2end/TableSnapshotReadsMapReduceIT.java  |  234 ++
 .../apache/phoenix/end2end/TenantIdTypeIT.java  |    6 +-
 .../end2end/TenantSpecificTablesDDLIT.java      |    5 +-
 .../end2end/TenantSpecificViewIndexIT.java      |    8 +-
 .../phoenix/end2end/ToNumberFunctionIT.java     |  273 +-
 .../java/org/apache/phoenix/end2end/TopNIT.java |   64 +-
 .../phoenix/end2end/TruncateFunctionIT.java     |   26 +-
 .../UngroupedAggregateRegionObserverIT.java     |  171 +
 .../org/apache/phoenix/end2end/UngroupedIT.java |  244 ++
 .../apache/phoenix/end2end/UnnestArrayIT.java   |   99 +-
 .../UpdateCacheAcrossDifferentClientsIT.java    |  337 ++
 .../org/apache/phoenix/end2end/UpgradeIT.java   |  395 +-
 .../phoenix/end2end/UpsertBigValuesIT.java      |   72 +-
 .../apache/phoenix/end2end/UpsertSelectIT.java  |  705 ++--
 .../apache/phoenix/end2end/UpsertValuesIT.java  |  492 +--
 .../phoenix/end2end/UserDefinedFunctionsIT.java |   11 +-
 .../phoenix/end2end/VariableLengthPKIT.java     | 1171 +++---
 .../java/org/apache/phoenix/end2end/ViewIT.java |  252 +-
 .../phoenix/end2end/index/BaseIndexIT.java      | 1215 ++++++
 .../phoenix/end2end/index/BaseLocalIndexIT.java |    9 +-
 .../index/ChildViewsUseParentViewIndexIT.java   |    7 +-
 .../phoenix/end2end/index/DropColumnIT.java     |   26 +-
 .../index/GlobalImmutableNonTxIndexIT.java      |   38 +
 .../end2end/index/GlobalImmutableTxIndexIT.java |   38 +
 .../index/GlobalMutableNonTxIndexIT.java        |   38 +
 .../end2end/index/GlobalMutableTxIndexIT.java   |   38 +
 .../phoenix/end2end/index/ImmutableIndexIT.java |  149 +-
 .../end2end/index/IndexExpressionIT.java        | 1552 --------
 .../phoenix/end2end/index/IndexHandlerIT.java   |  166 -
 .../apache/phoenix/end2end/index/IndexIT.java   | 1150 ------
 .../end2end/index/IndexMaintenanceIT.java       |  444 +++
 .../phoenix/end2end/index/IndexMetadataIT.java  |  132 +-
 .../phoenix/end2end/index/IndexTestUtil.java    |    2 +-
 .../phoenix/end2end/index/IndexUsageIT.java     |  775 ++++
 .../index/IndexWithTableSchemaChangeIT.java     |  692 ++++
 .../index/LocalImmutableNonTxIndexIT.java       |   38 +
 .../end2end/index/LocalImmutableTxIndexIT.java  |   38 +
 .../phoenix/end2end/index/LocalIndexIT.java     |  143 +-
 .../end2end/index/LocalMutableNonTxIndexIT.java |   38 +
 .../end2end/index/LocalMutableTxIndexIT.java    |   37 +
 .../end2end/index/MutableIndexFailureIT.java    |  373 +-
 .../phoenix/end2end/index/MutableIndexIT.java   |  162 +-
 .../index/MutableIndexReplicationIT.java        |   13 -
 .../index/MutableIndexSplitForwardScanIT.java   |   34 +
 .../end2end/index/MutableIndexSplitIT.java      |  182 +
 .../index/MutableIndexSplitReverseScanIT.java   |   34 +
 .../end2end/index/PartialIndexRebuilderIT.java  | 1035 ++++++
 .../end2end/index/ReadOnlyIndexFailureIT.java   |  291 --
 .../phoenix/end2end/index/ViewIndexIT.java      |   41 +-
 .../apache/phoenix/end2end/join/BaseJoinIT.java |  473 +++
 .../phoenix/end2end/join/HashJoinCacheIT.java   |  101 +
 .../end2end/join/HashJoinGlobalIndexIT.java     |  399 ++
 .../apache/phoenix/end2end/join/HashJoinIT.java | 2316 ++++++++++++
 .../end2end/join/HashJoinLocalIndexIT.java      |  528 +++
 .../phoenix/end2end/join/HashJoinMoreIT.java    |  915 +++++
 .../phoenix/end2end/join/HashJoinNoIndexIT.java |  391 ++
 .../join/SortMergeJoinGlobalIndexIT.java        |   84 +
 .../phoenix/end2end/join/SortMergeJoinIT.java   | 2431 ++++++++++++
 .../end2end/join/SortMergeJoinLocalIndexIT.java |   84 +
 .../end2end/join/SortMergeJoinNoIndexIT.java    |   66 +
 .../apache/phoenix/end2end/join/SubqueryIT.java |  788 ++++
 .../join/SubqueryUsingSortMergeJoinIT.java      |  566 +++
 .../phoenix/end2end/salted/SaltedTableIT.java   |  103 +-
 .../apache/phoenix/execute/PartialCommitIT.java |   36 +-
 .../UpsertSelectOverlappingBatchesIT.java       |  306 ++
 .../FailForUnsupportedHBaseVersionsIT.java      |    6 +-
 .../EndToEndCoveredColumnsIndexBuilderIT.java   |  346 --
 .../example/EndToEndCoveredIndexingIT.java      |  903 -----
 .../EndtoEndIndexingWithCompressionIT.java      |   45 -
 .../covered/example/FailWithoutRetriesIT.java   |  143 -
 .../DelayedTableResultIteratorFactory.java      |   22 +-
 .../phoenix/iterate/ScannerLeaseRenewalIT.java  |    9 -
 .../phoenix/monitoring/PhoenixMetricsIT.java    |  302 +-
 .../SystemCatalogWALEntryFilterIT.java          |   85 +-
 .../apache/phoenix/rpc/PhoenixClientRpcIT.java  |    1 -
 .../apache/phoenix/rpc/PhoenixServerRpcIT.java  |  111 +-
 .../org/apache/phoenix/rpc/UpdateCacheIT.java   |   32 +-
 .../phoenix/rpc/UpdateCacheWithScnIT.java       |   49 -
 .../phoenix/schema/stats/StatsCollectorIT.java  |  832 +++++
 .../apache/phoenix/trace/BaseTracingTestIT.java |  149 +-
 .../phoenix/trace/DelegateConnection.java       |  324 --
 .../phoenix/trace/DisableableMetricsWriter.java |   84 -
 .../trace/PhoenixTableMetricsWriterIT.java      |   45 +-
 .../phoenix/trace/PhoenixTraceReaderIT.java     |  180 -
 .../phoenix/trace/PhoenixTracingEndToEndIT.java |  243 +-
 .../phoenix/tx/FlappingTransactionIT.java       |   43 +-
 .../phoenix/tx/ParameterizedTransactionIT.java  |   15 +-
 .../org/apache/phoenix/tx/TransactionIT.java    |   44 +-
 .../org/apache/phoenix/tx/TxCheckpointIT.java   |   81 +-
 .../apache/phoenix/util/IndexScrutinyIT.java    |  106 +
 phoenix-core/src/main/antlr3/PhoenixSQL.g       |   90 +-
 .../hadoop/hbase/ipc/PhoenixRpcScheduler.java   |   10 +
 .../org/apache/hadoop/hbase/ipc/RpcUtil.java    |   32 +
 .../ipc/controller/IndexRpcController.java      |    6 +
 ...erRegionServerIndexRpcControllerFactory.java |   65 +
 ...egionServerMetadataRpcControllerFactory.java |   62 +
 .../ipc/controller/MetadataRpcController.java   |    5 +
 .../controller/ServerRpcControllerFactory.java  |   42 +-
 .../DataTableLocalIndexRegionScanner.java       |   74 +-
 .../IndexHalfStoreFileReaderGenerator.java      |   17 +-
 .../hadoop/hbase/regionserver/ScanInfoUtil.java |   35 +
 .../hbase/regionserver/ScannerContextUtil.java  |   41 +
 .../org/apache/phoenix/cache/GlobalCache.java   |    4 +-
 .../phoenix/cache/IndexMetaDataCache.java       |    7 +-
 .../org/apache/phoenix/cache/NoOpCache.java     |   93 +
 .../apache/phoenix/cache/ServerCacheClient.java |  368 +-
 .../apache/phoenix/cache/TenantCacheImpl.java   |    2 +
 .../phoenix/compile/AggregationManager.java     |   18 +-
 .../phoenix/compile/BaseMutationPlan.java       |   20 +
 .../phoenix/compile/CloseStatementCompiler.java |   56 +
 .../ColumnNameTrackingExpressionCompiler.java   |   46 +
 .../phoenix/compile/CreateTableCompiler.java    |   24 +-
 .../phoenix/compile/DeclareCursorCompiler.java  |   75 +
 .../phoenix/compile/DelegateMutationPlan.java   |   20 +
 .../apache/phoenix/compile/DeleteCompiler.java  | 1095 +++---
 .../phoenix/compile/ExpressionCompiler.java     |    2 +-
 .../apache/phoenix/compile/FromCompiler.java    |  159 +-
 .../apache/phoenix/compile/JoinCompiler.java    |   51 +-
 .../phoenix/compile/ListJarsQueryPlan.java      |   26 +-
 .../apache/phoenix/compile/MutationPlan.java    |    5 +-
 .../phoenix/compile/OpenStatementCompiler.java  |   56 +
 .../phoenix/compile/OrderPreservingTracker.java |    2 +-
 .../apache/phoenix/compile/PostDDLCompiler.java |    4 +-
 .../phoenix/compile/PostIndexDDLCompiler.java   |    9 +-
 .../compile/PostLocalIndexDDLCompiler.java      |    2 +-
 .../phoenix/compile/ProjectionCompiler.java     |   57 +-
 .../apache/phoenix/compile/QueryCompiler.java   |    6 +-
 .../org/apache/phoenix/compile/QueryPlan.java   |    6 +-
 .../org/apache/phoenix/compile/ScanRanges.java  |   32 +-
 .../phoenix/compile/StatementContext.java       |    4 +-
 .../apache/phoenix/compile/StatementPlan.java   |   19 +
 .../phoenix/compile/SubselectRewriter.java      |  137 +-
 .../apache/phoenix/compile/TraceQueryPlan.java  |   26 +-
 .../compile/TupleProjectionCompiler.java        |    8 +-
 .../apache/phoenix/compile/UnionCompiler.java   |   11 +-
 .../apache/phoenix/compile/UpsertCompiler.java  | 1179 +++---
 .../apache/phoenix/compile/WhereCompiler.java   |   54 +-
 .../apache/phoenix/compile/WhereOptimizer.java  |   13 +-
 .../BaseMetaDataEndpointObserver.java           |  111 +
 .../phoenix/coprocessor/BaseRegionScanner.java  |    4 +-
 .../coprocessor/BaseScannerRegionObserver.java  |  339 +-
 .../DelegateRegionCoprocessorEnvironment.java   |  114 +
 .../coprocessor/DelegateRegionScanner.java      |    4 +-
 .../HashJoinCacheNotFoundException.java         |   45 +
 .../coprocessor/HashJoinRegionScanner.java      |   53 +-
 .../coprocessor/MetaDataEndpointImpl.java       | 2380 ++++++------
 .../coprocessor/MetaDataEndpointObserver.java   |   68 +
 .../phoenix/coprocessor/MetaDataProtocol.java   |   27 +-
 .../coprocessor/MetaDataRegionObserver.java     |  403 +-
 .../phoenix/coprocessor/OrphanCleaner.java      |   68 +
 .../coprocessor/PhoenixAccessController.java    |  611 ++++
 .../PhoenixMetaDataCoprocessorHost.java         |  236 ++
 .../PhoenixTransactionalProcessor.java          |    4 +-
 .../phoenix/coprocessor/ScanRegionObserver.java |  329 +-
 .../coprocessor/ServerCachingEndpointImpl.java  |   48 +-
 .../apache/phoenix/coprocessor/TableInfo.java   |   82 +
 .../coprocessor/TableViewFinderResult.java      |   49 +
 .../UngroupedAggregateRegionObserver.java       |  439 ++-
 .../apache/phoenix/coprocessor/ViewFinder.java  |  221 ++
 .../coprocessor/WhereConstantParser.java        |  117 +
 .../coprocessor/generated/MetaDataProtos.java   |   61 +-
 .../coprocessor/generated/PTableProtos.java     |  340 +-
 .../phoenix/exception/SQLExceptionCode.java     |   39 +-
 .../exception/UpgradeInProgressException.java   |    8 +-
 .../apache/phoenix/execute/AggregatePlan.java   |   51 +-
 .../apache/phoenix/execute/BaseQueryPlan.java   |  113 +-
 .../phoenix/execute/ClientAggregatePlan.java    |   28 +
 .../apache/phoenix/execute/ClientScanPlan.java  |   25 +
 .../apache/phoenix/execute/CommitException.java |    8 +-
 .../apache/phoenix/execute/CorrelatePlan.java   |   25 +
 .../apache/phoenix/execute/CursorFetchPlan.java |   71 +
 .../apache/phoenix/execute/DelegateHTable.java  |   22 +-
 .../phoenix/execute/DelegateQueryPlan.java      |   25 +
 .../apache/phoenix/execute/HashJoinPlan.java    |  204 +-
 .../execute/LiteralResultIterationPlan.java     |   34 +-
 .../apache/phoenix/execute/MutationState.java   |  642 ++--
 .../org/apache/phoenix/execute/ScanPlan.java    |   96 +-
 .../phoenix/execute/SortMergeJoinPlan.java      |  115 +-
 .../apache/phoenix/execute/TupleProjector.java  |   12 +-
 .../org/apache/phoenix/execute/UnionPlan.java   |   64 +-
 .../phoenix/expression/ExpressionType.java      |   10 +-
 .../phoenix/expression/LikeExpression.java      |    2 +-
 .../expression/ProjectedColumnExpression.java   |    7 +-
 .../RowValueConstructorExpression.java          |    4 +-
 .../expression/SingleCellColumnExpression.java  |   35 +-
 .../expression/StringConcatExpression.java      |    3 +
 .../FirstLastValueBaseClientAggregator.java     |   65 +-
 .../FirstLastValueServerAggregator.java         |   15 +-
 .../aggregator/PercentileClientAggregator.java  |    6 +-
 .../function/ArrayConcatFunction.java           |   12 +-
 .../function/ArrayModifierFunction.java         |    2 -
 .../expression/function/CoalesceFunction.java   |    2 +-
 .../function/CollationKeyFunction.java          |  199 +
 .../function/ConvertTimezoneFunction.java       |   10 +
 .../function/CurrentDateFunction.java           |   14 +-
 .../function/CurrentTimeFunction.java           |   12 +-
 ...stinctCountHyperLogLogAggregateFunction.java |  192 +
 .../expression/function/EncodeFunction.java     |    5 +-
 .../function/FirstLastValueBaseFunction.java    |   20 +-
 .../expression/function/FirstValueFunction.java |    8 +-
 .../function/FirstValuesFunction.java           |   97 +
 .../function/FloorDateExpression.java           |   11 +-
 .../expression/function/GetBitFunction.java     |    1 +
 .../expression/function/GetByteFunction.java    |    1 +
 .../expression/function/LastValueFunction.java  |    8 +-
 .../expression/function/LastValuesFunction.java |   97 +
 .../expression/function/LowerFunction.java      |    6 +-
 .../expression/function/LpadFunction.java       |   11 +-
 .../expression/function/MD5Function.java        |    3 +-
 .../expression/function/NthValueFunction.java   |   12 +-
 .../function/OctetLengthFunction.java           |    1 +
 .../function/RegexpReplaceFunction.java         |    9 +
 .../function/RoundDecimalExpression.java        |    9 +-
 .../function/RoundTimestampExpression.java      |   11 +-
 .../expression/function/ScalarFunction.java     |    9 +
 .../expression/function/SetBitFunction.java     |    3 +
 .../expression/function/SetByteFunction.java    |    2 +
 .../expression/function/SignFunction.java       |    3 +
 .../expression/function/SubstrFunction.java     |   13 +-
 .../function/TimezoneOffsetFunction.java        |   11 +-
 .../visitor/CloneExpressionVisitor.java         |    2 +-
 .../visitor/TraverseNoExpressionVisitor.java    |    5 +-
 .../MultiCFCQKeyValueComparisonFilter.java      |    4 +-
 .../filter/MultiCQKeyValueComparisonFilter.java |    4 +-
 .../MultiEncodedCQKeyValueComparisonFilter.java |   69 +-
 .../filter/MultiKeyValueComparisonFilter.java   |   41 +-
 .../org/apache/phoenix/hbase/index/Indexer.java |  552 ++-
 .../apache/phoenix/hbase/index/LockManager.java |  252 ++
 .../apache/phoenix/hbase/index/ValueGetter.java |   10 +-
 .../hbase/index/builder/BaseIndexBuilder.java   |    6 +
 .../hbase/index/builder/BaseIndexCodec.java     |   33 +-
 .../hbase/index/builder/IndexBuildManager.java  |   91 +-
 .../hbase/index/builder/IndexBuilder.java       |    3 +
 .../hbase/index/covered/IndexMetaData.java      |   24 +-
 .../hbase/index/covered/LocalTableState.java    |   91 +-
 .../hbase/index/covered/NonTxIndexBuilder.java  |  200 +-
 .../phoenix/hbase/index/covered/TableState.java |    3 +-
 .../index/covered/data/LazyValueGetter.java     |  107 +-
 .../hbase/index/covered/data/LocalTable.java    |   22 +-
 .../index/covered/example/ColumnGroup.java      |  112 -
 .../index/covered/example/CoveredColumn.java    |  107 -
 .../example/CoveredColumnIndexCodec.java        |  364 --
 .../CoveredColumnIndexSpecifierBuilder.java     |  184 -
 .../covered/example/CoveredColumnIndexer.java   |  164 -
 .../filter/ApplyAndFilterDeletesFilter.java     |    7 +-
 .../covered/update/IndexUpdateManager.java      |   26 +-
 .../index/covered/update/SortedCollection.java  |  128 -
 .../index/metrics/MetricsIndexerSource.java     |  150 +
 .../metrics/MetricsIndexerSourceFactory.java    |   38 +
 .../index/metrics/MetricsIndexerSourceImpl.java |  136 +
 .../hbase/index/scanner/EmptyScanner.java       |   16 +-
 .../hbase/index/scanner/ScannerBuilder.java     |   25 +-
 .../hbase/index/table/CachingHTableFactory.java |  184 -
 .../hbase/index/util/IndexManagementUtil.java   |   96 +-
 .../phoenix/hbase/index/write/IndexWriter.java  |   15 +-
 .../hbase/index/write/IndexWriterUtils.java     |   99 +-
 .../write/LeaveIndexActiveFailurePolicy.java    |   62 +
 .../write/ParallelWriterIndexCommitter.java     |   15 +-
 .../hbase/index/write/RecoveryIndexWriter.java  |    1 -
 .../TrackingParallelWriterIndexCommitter.java   |  245 ++
 .../recovery/StoreFailuresInCachePolicy.java    |    1 +
 .../TrackingParallelWriterIndexCommitter.java   |  241 --
 .../apache/phoenix/index/IndexMaintainer.java   |  167 +-
 .../index/IndexMetaDataCacheFactory.java        |   15 +-
 .../phoenix/index/PhoenixIndexBuilder.java      |   16 +-
 .../apache/phoenix/index/PhoenixIndexCodec.java |    4 +-
 .../index/PhoenixIndexFailurePolicy.java        |  190 +-
 .../phoenix/index/PhoenixIndexMetaData.java     |   45 +-
 .../index/PhoenixTransactionalIndexer.java      |  175 +-
 .../phoenix/iterate/BaseResultIterators.java    |  428 ++-
 .../phoenix/iterate/ChunkedResultIterator.java  |   11 +-
 .../phoenix/iterate/CursorResultIterator.java   |   75 +
 .../iterate/DefaultParallelScanGrouper.java     |   64 +-
 .../DefaultTableResultIteratorFactory.java      |   15 +-
 .../DistinctAggregatingResultIterator.java      |    6 +-
 .../apache/phoenix/iterate/ExplainTable.java    |    6 +-
 .../phoenix/iterate/FilterResultIterator.java   |    5 +-
 .../iterate/MapReduceParallelScanGrouper.java   |   67 +-
 .../NonAggregateRegionScannerFactory.java       |  376 ++
 .../phoenix/iterate/OrderedResultIterator.java  |    2 +-
 .../phoenix/iterate/ParallelIterators.java      |   23 +-
 .../phoenix/iterate/ParallelScanGrouper.java    |    9 +-
 .../phoenix/iterate/RegionScannerFactory.java   |  303 ++
 .../phoenix/iterate/ScanningResultIterator.java |   96 +-
 .../apache/phoenix/iterate/SerialIterators.java |   26 +-
 .../apache/phoenix/iterate/SnapshotScanner.java |  183 +
 .../phoenix/iterate/TableResultIterator.java    |   71 +-
 .../iterate/TableResultIteratorFactory.java     |   10 +-
 .../phoenix/iterate/TableSamplerPredicate.java  |  114 +
 .../iterate/TableSnapshotResultIterator.java    |  156 +
 .../apache/phoenix/jdbc/DelegateConnection.java |  321 ++
 .../phoenix/jdbc/DelegatePreparedStatement.java |  553 +++
 .../apache/phoenix/jdbc/DelegateResultSet.java  | 1008 +++++
 .../apache/phoenix/jdbc/DelegateStatement.java  |  254 ++
 .../phoenix/jdbc/LoggingPhoenixConnection.java  |  109 +
 .../jdbc/LoggingPhoenixPreparedStatement.java   |   53 +
 .../phoenix/jdbc/LoggingPhoenixResultSet.java   |   45 +
 .../phoenix/jdbc/LoggingPhoenixStatement.java   |   48 +
 .../apache/phoenix/jdbc/PhoenixConnection.java  |  612 ++--
 .../phoenix/jdbc/PhoenixDatabaseMetaData.java   |  537 ++-
 .../phoenix/jdbc/PhoenixEmbeddedDriver.java     |   46 +-
 .../apache/phoenix/jdbc/PhoenixMetricsLog.java  |   35 +
 .../phoenix/jdbc/PhoenixPreparedStatement.java  |    6 +-
 .../apache/phoenix/jdbc/PhoenixResultSet.java   |    5 +-
 .../apache/phoenix/jdbc/PhoenixStatement.java   |  335 +-
 .../apache/phoenix/join/HashCacheClient.java    |   20 +-
 .../mapreduce/FormatToBytesWritableMapper.java  |    1 +
 .../mapreduce/FormatToKeyValueReducer.java      |    7 +-
 .../mapreduce/MultiHfileOutputFormat.java       |    5 +-
 .../phoenix/mapreduce/PhoenixInputFormat.java   |   12 +-
 .../phoenix/mapreduce/PhoenixOutputFormat.java  |   13 +-
 .../phoenix/mapreduce/PhoenixRecordReader.java  |   42 +-
 .../phoenix/mapreduce/PhoenixRecordWriter.java  |    8 +-
 .../mapreduce/index/IndexScrutinyMapper.java    |  377 ++
 .../index/IndexScrutinyTableOutput.java         |  345 ++
 .../mapreduce/index/IndexScrutinyTool.java      |  522 +++
 .../phoenix/mapreduce/index/IndexTool.java      |   57 +-
 .../mapreduce/index/PhoenixIndexDBWritable.java |   11 +-
 .../index/PhoenixIndexImportDirectMapper.java   |    2 +-
 .../index/PhoenixIndexImportMapper.java         |    2 +-
 .../index/PhoenixIndexPartialBuildMapper.java   |    7 +-
 .../index/PhoenixScrutinyJobCounters.java       |   41 +
 .../index/SourceTargetColumnNames.java          |  195 +
 .../phoenix/mapreduce/util/ConnectionUtil.java  |   25 +-
 .../mapreduce/util/IndexColumnNames.java        |  251 ++
 .../util/PhoenixConfigurationUtil.java          |  144 +-
 .../mapreduce/util/PhoenixMapReduceUtil.java    |   64 +
 .../phoenix/memory/GlobalMemoryManager.java     |   52 +-
 .../apache/phoenix/monitoring/AtomicMetric.java |   11 +-
 .../phoenix/monitoring/CombinableMetric.java    |   12 +-
 .../monitoring/CombinableMetricImpl.java        |   11 +-
 .../phoenix/monitoring/GlobalClientMetrics.java |    9 +-
 .../phoenix/monitoring/GlobalMetricImpl.java    |   11 +-
 .../org/apache/phoenix/monitoring/Metric.java   |   10 +-
 .../apache/phoenix/monitoring/MetricType.java   |   83 +-
 .../phoenix/monitoring/MutationMetricQueue.java |   25 +-
 .../phoenix/monitoring/NonAtomicMetric.java     |   13 +-
 .../phoenix/monitoring/OverAllQueryMetrics.java |   16 +-
 .../phoenix/monitoring/ReadMetricQueue.java     |   12 +-
 .../phoenix/monitoring/ScanMetricsHolder.java   |  121 +
 .../java/org/apache/phoenix/optimize/Cost.java  |  123 +
 .../apache/phoenix/optimize/QueryOptimizer.java |   44 +-
 .../phoenix/parse/AddColumnStatement.java       |    2 +-
 .../phoenix/parse/AlterIndexStatement.java      |   14 +
 .../phoenix/parse/ChangePermsStatement.java     |  102 +
 .../apache/phoenix/parse/CloseStatement.java    |   40 +
 .../apache/phoenix/parse/ConcreteTableNode.java |   22 +
 .../phoenix/parse/CreateSchemaStatement.java    |    2 +-
 .../org/apache/phoenix/parse/CursorName.java    |   44 +
 .../phoenix/parse/DeclareCursorStatement.java   |   60 +
 .../apache/phoenix/parse/DeleteStatement.java   |    5 +
 ...tinctCountHyperLogLogAggregateParseNode.java |   39 +
 .../apache/phoenix/parse/FetchStatement.java    |   52 +
 .../phoenix/parse/FilterableStatement.java      |    1 +
 .../parse/FirstValuesAggregateParseNode.java    |   38 +
 .../parse/LastValuesAggregateParseNode.java     |   38 +
 .../apache/phoenix/parse/NamedTableNode.java    |   21 +-
 .../org/apache/phoenix/parse/OpenStatement.java |   40 +
 .../apache/phoenix/parse/ParseNodeFactory.java  |   59 +-
 .../org/apache/phoenix/parse/SQLParser.java     |   76 +
 .../apache/phoenix/parse/SelectStatement.java   |   15 +
 .../phoenix/query/BaseQueryServicesImpl.java    |    3 +-
 .../phoenix/query/ConnectionQueryServices.java  |    4 +-
 .../query/ConnectionQueryServicesImpl.java      |  606 ++-
 .../query/ConnectionlessQueryServicesImpl.java  |   29 +-
 .../query/DelegateConnectionQueryServices.java  |   14 +-
 .../query/PropertyNotAllowedException.java      |   35 +
 .../apache/phoenix/query/PropertyPolicy.java    |   63 +
 .../phoenix/query/PropertyPolicyProvider.java   |   39 +
 .../apache/phoenix/query/QueryConstants.java    |   23 +
 .../org/apache/phoenix/query/QueryServices.java |   57 +-
 .../phoenix/query/QueryServicesOptions.java     |  173 +-
 .../SystemCatalogWALEntryFilter.java            |   42 +-
 .../schema/ColumnFamilyNotFoundException.java   |    4 +-
 .../phoenix/schema/ColumnNotFoundException.java |   14 +-
 .../org/apache/phoenix/schema/ColumnRef.java    |    4 +-
 .../apache/phoenix/schema/DelegateColumn.java   |   15 +
 .../apache/phoenix/schema/DelegateTable.java    |    5 +
 .../schema/FunctionNotFoundException.java       |    2 +-
 .../apache/phoenix/schema/MetaDataClient.java   | 1026 ++++--
 .../schema/MetaDataEntityNotFoundException.java |   20 +-
 .../phoenix/schema/MetaDataSplitPolicy.java     |   23 +-
 .../java/org/apache/phoenix/schema/PColumn.java |   12 +
 .../phoenix/schema/PColumnFamilyImpl.java       |    3 -
 .../org/apache/phoenix/schema/PColumnImpl.java  |  114 +-
 .../org/apache/phoenix/schema/PIndexState.java  |    3 +-
 .../apache/phoenix/schema/PMetaDataImpl.java    |    3 +-
 .../java/org/apache/phoenix/schema/PTable.java  |   39 +-
 .../org/apache/phoenix/schema/PTableImpl.java   |  164 +-
 .../org/apache/phoenix/schema/RowKeySchema.java |    4 +-
 .../org/apache/phoenix/schema/SaltingUtil.java  |    4 +-
 .../phoenix/schema/SchemaNotFoundException.java |   10 +-
 .../schema/SequenceNotFoundException.java       |   14 +-
 .../phoenix/schema/TableNotFoundException.java  |   19 +-
 .../apache/phoenix/schema/TableProperty.java    |   18 +
 .../org/apache/phoenix/schema/TableRef.java     |    9 +-
 .../schema/TablesNotInSyncException.java        |   22 +
 .../UpsertColumnsValuesMismatchException.java   |   41 +
 .../org/apache/phoenix/schema/ValueSchema.java  |    2 +-
 .../stats/DefaultStatisticsCollector.java       |  127 +-
 .../phoenix/schema/stats/GuidePostsInfo.java    |   59 +-
 .../schema/stats/GuidePostsInfoBuilder.java     |   48 +-
 .../stats/StatisticsCollectorFactory.java       |   13 +-
 .../phoenix/schema/stats/StatisticsScanner.java |    6 +-
 .../phoenix/schema/stats/StatisticsUtil.java    |   62 +-
 .../phoenix/schema/stats/StatisticsWriter.java  |  102 +-
 .../tuple/EncodedColumnQualiferCellsList.java   |   85 +-
 .../schema/tuple/MultiKeyValueTuple.java        |    7 +-
 .../phoenix/schema/tuple/ValueGetterTuple.java  |   12 +-
 .../phoenix/schema/types/PArrayDataType.java    |  107 +-
 .../schema/types/PArrayDataTypeDecoder.java     |   79 +-
 .../schema/types/PArrayDataTypeEncoder.java     |   81 +-
 .../apache/phoenix/schema/types/PBoolean.java   |    9 +
 .../apache/phoenix/schema/types/PDataType.java  |   23 +-
 .../org/apache/phoenix/schema/types/PDate.java  |    9 +
 .../apache/phoenix/schema/types/PDecimal.java   |    9 +
 .../apache/phoenix/schema/types/PDouble.java    |    9 +
 .../org/apache/phoenix/schema/types/PLong.java  |   18 +-
 .../apache/phoenix/schema/types/PTimestamp.java |    9 +
 .../phoenix/schema/types/PUnsignedDouble.java   |  259 +-
 .../phoenix/schema/types/PUnsignedLong.java     |    9 +
 .../apache/phoenix/schema/types/PVarchar.java   |    9 +
 .../apache/phoenix/trace/TraceMetricSource.java |  183 -
 .../org/apache/phoenix/trace/TraceReader.java   |   16 +-
 .../apache/phoenix/trace/TraceSpanReceiver.java |  104 +
 .../org/apache/phoenix/trace/TraceWriter.java   |  333 ++
 .../org/apache/phoenix/trace/util/Tracing.java  |   16 +-
 .../transaction/OmidTransactionContext.java     |  174 +
 .../transaction/OmidTransactionTable.java       |  364 ++
 .../transaction/PhoenixTransactionContext.java  |  191 +
 .../transaction/PhoenixTransactionalTable.java  |  149 +
 .../transaction/TephraTransactionContext.java   |  514 +++
 .../transaction/TephraTransactionTable.java     |  350 ++
 .../phoenix/transaction/TransactionFactory.java |  143 +
 .../java/org/apache/phoenix/util/ByteUtil.java  |   15 +
 .../java/org/apache/phoenix/util/CostUtil.java  |   90 +
 .../org/apache/phoenix/util/CursorUtil.java     |  189 +
 .../phoenix/util/DefaultEnvironmentEdge.java    |    3 +-
 .../apache/phoenix/util/EncodedColumnsUtil.java |   50 +-
 .../apache/phoenix/util/EnvironmentEdge.java    |    5 +-
 .../phoenix/util/EnvironmentEdgeManager.java    |    1 +
 .../org/apache/phoenix/util/ExpressionUtil.java |   14 +
 .../util/FirstLastNthValueDataContainer.java    |    7 +-
 .../java/org/apache/phoenix/util/IndexUtil.java |  121 +-
 .../java/org/apache/phoenix/util/JDBCUtil.java  |    6 +-
 .../org/apache/phoenix/util/KeyValueUtil.java   |   46 +
 .../java/org/apache/phoenix/util/LogUtil.java   |   10 +
 .../org/apache/phoenix/util/MetaDataUtil.java   |  175 +-
 .../org/apache/phoenix/util/NumberUtil.java     |   32 +
 .../org/apache/phoenix/util/PhoenixRuntime.java |  140 +-
 .../org/apache/phoenix/util/PropertiesUtil.java |   53 +-
 .../java/org/apache/phoenix/util/QueryUtil.java |  122 +-
 .../org/apache/phoenix/util/RepairUtil.java     |   10 +-
 .../java/org/apache/phoenix/util/ScanUtil.java  |    9 +-
 .../org/apache/phoenix/util/SchemaUtil.java     |  139 +-
 .../org/apache/phoenix/util/ServerUtil.java     |   59 +
 .../apache/phoenix/util/TransactionUtil.java    |   38 +-
 .../org/apache/phoenix/util/UpgradeUtil.java    |  130 +-
 .../apache/phoenix/util/VarBinaryFormatter.java |   52 +
 .../hbase/ipc/PhoenixIndexRpcSchedulerTest.java |    2 +-
 .../wal/ReadWriteKeyValuesWithCodecTest.java    |  186 +
 .../apache/phoenix/cache/TenantCacheTest.java   |   43 +-
 .../phoenix/compile/CursorCompilerTest.java     |   87 +
 .../phoenix/compile/QueryCompilerTest.java      |  384 +-
 .../phoenix/compile/QueryOptimizerTest.java     |   13 +-
 .../phoenix/compile/WhereOptimizerTest.java     |    3 +-
 .../coprocessor/MetaDataEndpointImplTest.java   |  287 ++
 .../phoenix/execute/CorrelatePlanTest.java      |    8 +-
 .../execute/LiteralResultIteratorPlanTest.java  |    9 +-
 .../phoenix/execute/UnnestArrayPlanTest.java    |    7 +-
 .../expression/ArrayConcatFunctionTest.java     |   27 +-
 .../expression/ColumnExpressionTest.java        |    9 +-
 .../expression/OctetLengthFunctionTest.java     |    7 +-
 .../function/CollationKeyFunctionTest.java      |  243 ++
 .../phoenix/filter/SkipScanBigFilterTest.java   |    3 +-
 .../hbase/index/covered/ColumnGroup.java        |  112 +
 .../hbase/index/covered/CoveredColumn.java      |  106 +
 .../index/covered/CoveredColumnIndexCodec.java  |  372 ++
 .../CoveredColumnIndexSpecifierBuilder.java     |  182 +
 .../hbase/index/covered/CoveredColumnsTest.java |   46 +
 .../index/covered/LocalTableStateTest.java      |  295 ++
 .../index/covered/NonTxIndexBuilderTest.java    |  342 ++
 .../hbase/index/covered/TestColumnTracker.java  |   61 +
 .../covered/TestCoveredColumnIndexCodec.java    |  246 ++
 .../hbase/index/covered/TestCoveredColumns.java |   46 -
 .../TestCoveredIndexSpecifierBuilder.java       |   72 +
 .../index/covered/TestLocalTableState.java      |  198 -
 .../index/covered/data/TestLocalTable.java      |   63 +
 .../covered/example/TestColumnTracker.java      |   61 -
 .../example/TestCoveredColumnIndexCodec.java    |  250 --
 .../TestCoveredIndexSpecifierBuilder.java       |   75 -
 .../covered/update/TestIndexUpdateManager.java  |   20 +-
 .../index/util/TestIndexManagementUtil.java     |   29 +
 .../index/write/TestCachingHTableFactory.java   |   69 -
 .../hbase/index/write/TestIndexWriter.java      |  102 +-
 .../index/write/TestParalleIndexWriter.java     |   12 +-
 .../write/TestParalleWriterIndexCommitter.java  |   12 +-
 .../index/write/TestWALRecoveryCaching.java     |   10 +-
 .../recovery/TestPerRegionIndexWriteCache.java  |    2 +-
 .../phoenix/index/IndexMaintainerTest.java      |    5 +-
 .../iterate/SpoolingResultIteratorTest.java     |    2 +-
 .../apache/phoenix/jdbc/PhoenixDriverTest.java  |    8 +-
 .../phoenix/jdbc/PhoenixEmbeddedDriverTest.java |   19 +
 .../phoenix/mapreduce/index/BaseIndexTest.java  |   97 +
 .../index/IndexScrutinyTableOutputTest.java     |   87 +
 .../mapreduce/util/IndexColumnNamesTest.java    |   74 +
 .../phoenix/memory/MemoryManagerTest.java       |  208 +-
 .../apache/phoenix/metrics/MetricTypeTest.java  |   42 +
 .../apache/phoenix/parse/CursorParserTest.java  |  367 ++
 .../apache/phoenix/parse/QueryParserTest.java   |   80 +-
 .../java/org/apache/phoenix/query/BaseTest.java |  318 +-
 .../query/ConnectionQueryServicesImplTest.java  |   76 +
 .../query/ParallelIteratorsSplitTest.java       |   23 +-
 .../query/PropertyPolicyProviderTest.java       |   52 +
 .../phoenix/query/QueryServicesTestImpl.java    |   12 +-
 .../phoenix/query/TestPropertyPolicy.java       |   47 +
 .../schema/ImmutableStorageSchemeTest.java      |  241 +-
 .../apache/phoenix/schema/RowKeySchemaTest.java |    2 +-
 .../phoenix/schema/RowKeyValueAccessorTest.java |    2 +-
 .../schema/stats/StatisticsScannerTest.java     |    4 +-
 .../schema/types/PDataTypeForArraysTest.java    |   73 +
 .../phoenix/schema/types/PDataTypeTest.java     |   31 +
 .../phoenix/trace/TraceMetricsSourceTest.java   |   95 -
 .../phoenix/trace/TraceSpanReceiverTest.java    |   82 +
 .../org/apache/phoenix/util/IndexScrutiny.java  |  165 +
 .../apache/phoenix/util/MetaDataUtilTest.java   |   22 +-
 .../apache/phoenix/util/PropertiesUtilTest.java |   19 +-
 .../org/apache/phoenix/util/QueryUtilTest.java  |   18 +-
 .../java/org/apache/phoenix/util/Repeat.java    |   30 +
 .../apache/phoenix/util/RunUntilFailure.java    |   90 +
 .../java/org/apache/phoenix/util/TestUtil.java  |  199 +-
 .../org.apache.phoenix.query.PropertyPolicy     |   16 +
 phoenix-flume/pom.xml                           |    6 +-
 phoenix-hive/pom.xml                            |   40 +-
 .../phoenix/hive/BaseHivePhoenixStoreIT.java    |    5 +-
 .../apache/phoenix/hive/HiveMapReduceIT.java    |    1 -
 .../apache/phoenix/hive/HivePhoenixStoreIT.java |    3 +
 .../org/apache/phoenix/hive/HiveTestUtil.java   |    9 +-
 .../java/org/apache/phoenix/hive/HiveTezIT.java |    1 -
 .../org/apache/phoenix/hive/PhoenixRowKey.java  |   17 +-
 .../phoenix/hive/PhoenixStorageHandler.java     |   14 +-
 .../org/apache/phoenix/hive/PrimaryKeyData.java |   88 +
 .../hive/mapreduce/PhoenixRecordReader.java     |   11 +-
 .../phoenix/hive/query/PhoenixQueryBuilder.java |    4 +-
 .../hive/util/PhoenixConnectionUtil.java        |    2 +-
 .../hive/util/PhoenixStorageHandlerUtil.java    |    9 +-
 .../apache/phoenix/hive/PrimaryKeyDataTest.java |   79 +
 phoenix-kafka/pom.xml                           |   32 +-
 phoenix-load-balancer/pom.xml                   |   84 +
 .../phoenix/end2end/LoadBalancerEnd2EndIT.java  |  144 +
 .../service/LoadBalanceZookeeperConfImpl.java   |  103 +
 .../loadbalancer/service/LoadBalancer.java      |  178 +
 .../queryserver/register/ZookeeperRegistry.java |   72 +
 ...oadbalancer.service.LoadBalanceZookeeperConf |    1 +
 .../config/scenario/user_defined_scenario.xml   |    4 +-
 phoenix-pherf/pom.xml                           |    6 +-
 .../org/apache/phoenix/pherf/DataIngestIT.java  |   19 +
 .../apache/phoenix/pherf/ResultBaseTestIT.java  |    4 +-
 .../phoenix/pherf/configuration/DataModel.java  |    1 +
 .../apache/phoenix/pherf/configuration/Ddl.java |   68 +
 .../phoenix/pherf/configuration/Scenario.java   |   49 +-
 .../pherf/configuration/XMLConfigParser.java    |   15 +-
 .../pherf/result/impl/XMLResultHandler.java     |   17 +-
 .../apache/phoenix/pherf/util/PhoenixUtil.java  |   75 +-
 .../phoenix/pherf/workload/WriteWorkload.java   |   14 +-
 .../scenario/prod_test_unsalted_scenario.xml    |   11 +-
 .../phoenix/pherf/ConfigurationParserTest.java  |   10 +-
 .../phoenix/pherf/XMLConfigParserTest.java      |   53 +
 .../pherf/result/impl/XMLResultHandlerTest.java |   53 +
 .../resources/malicious_results_with_dtd.xml    |  676 ++++
 .../scenario/malicious_scenario_with_dtd.xml    |   48 +
 .../test/resources/scenario/test_scenario.xml   |   26 +-
 phoenix-pig/pom.xml                             |    2 +-
 .../java/org/apache/phoenix/pig/BasePigIT.java  |    4 +
 .../apache/phoenix/pig/PhoenixHBaseStorage.java |   12 +-
 phoenix-protocol/src/main/MetaDataService.proto |    1 +
 phoenix-protocol/src/main/PTable.proto          |    3 +
 phoenix-queryserver-client/pom.xml              |   18 +-
 phoenix-queryserver/pom.xml                     |   12 +-
 .../HttpParamImpersonationQueryServerIT.java    |  434 +++
 .../phoenix/end2end/QueryServerBasicsIT.java    |  183 +-
 .../phoenix/end2end/SecureQueryServerIT.java    |  320 ++
 .../src/it/resources/log4j.properties           |    9 +-
 .../service/LoadBalanceZookeeperConf.java       |   42 +
 .../phoenix/queryserver/register/Registry.java  |   48 +
 .../phoenix/queryserver/server/QueryServer.java |  164 +-
 .../server/RemoteUserExtractorFactory.java      |   36 +
 .../server/PhoenixRemoteUserExtractorTest.java  |  108 +
 .../server/RemoteUserExtractorFactoryTest.java  |   35 +
 phoenix-server/pom.xml                          |    7 +-
 phoenix-spark/pom.xml                           |   24 +-
 phoenix-spark/src/it/resources/globalSetup.sql  |    3 +-
 .../phoenix/spark/AbstractPhoenixSparkIT.scala  |   17 +-
 .../apache/phoenix/spark/PhoenixSparkIT.scala   |   27 +-
 .../phoenix/spark/DataFrameFunctions.scala      |   19 +-
 .../apache/phoenix/spark/DefaultSource.scala    |    2 +-
 .../org/apache/phoenix/spark/PhoenixRDD.scala   |    4 +-
 phoenix-tracing-webapp/pom.xml                  |    2 +-
 pom.xml                                         |   94 +-
 751 files changed, 79056 insertions(+), 40002 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/15d4dace/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
----------------------------------------------------------------------
diff --cc phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
index 4ad3a8c,6277b78..d48d0b7
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
@@@ -1904,20 -2401,16 +2401,16 @@@ public class MetaDataEndpointImpl exten
                              PName parentTenantId = parentTenantIdCell!=null ? PNameFactory.newName(parentTenantIdCell.getValueArray(), parentTenantIdCell.getValueOffset(), parentTenantIdCell.getValueLength()) : null;
                              byte[] linkKey = MetaDataUtil.getChildLinkKey(parentTenantId, table.getParentSchemaName(), table.getParentTableName(), table.getTenantId(), table.getName());
                              Delete linkDelete = new Delete(linkKey, clientTimeStamp);
-                             rowsToDelete.add(linkDelete);
+                             childLinkMutations.add(linkDelete);
                          }
                  }
-                 // FIXME: Remove when unintentionally deprecated method is fixed (HBASE-7870).
-                 // FIXME: the version of the Delete constructor without the lock args was introduced
-                 // in 0.94.4, thus if we try to use it here we can no longer use the 0.94.2 version
-                 // of the client.
-                 Delete delete = new Delete(kv.getRowArray(), kv.getRowOffset(), kv.getRowLength(), clientTimeStamp);
-                 rowsToDelete.add(delete);
-                 results.clear();
-                 scanner.next(results);
-             } while (!results.isEmpty());
+                 catalogMutations.add(scannerAdaptor.getRowDelete(clientTimeStamp));
+             } while (scannerAdaptor.next());
+         }
+         finally {
+         	scanner.close();
          }
 -
 +        
          // Recursively delete indexes
          for (byte[] indexName : indexNames) {
              byte[] indexKey = SchemaUtil.getTableKey(tenantId, schemaName, indexName);
@@@ -3537,56 -3790,30 +3790,37 @@@
          return checkKeyInRegion(key, region, MutationCode.SCHEMA_NOT_IN_REGION);
  
      }
- 
-     /**
-      * Certain operations, such as DROP TABLE are not allowed if there a table has child views. This class wraps the
-      * Results of a scanning the Phoenix Metadata for child views for a specific table and stores an additional flag for
-      * whether whether SYSTEM.CATALOG has split across multiple regions.
-      */
-     private static class TableViewFinderResult {
- 
-         private List<Result> results = Lists.newArrayList();
-         private boolean allViewsNotInSingleRegion = false;
- 
-         private TableViewFinderResult() {
-         }
-         
-         private TableViewFinderResult(List<Result> results) {
-             this.results = results;
-         }
+     
+     private static class ViewInfo {
+         private byte[] tenantId;
+         private byte[] schemaName;
+         private byte[] viewName;
          
-         public boolean hasViews() {
-             return !results.isEmpty();
+         public ViewInfo(byte[] tenantId, byte[] schemaName, byte[] viewName) {
+             super();
+             this.tenantId = tenantId;
+             this.schemaName = schemaName;
+             this.viewName = viewName;
          }
  
-         private void setAllViewsNotInSingleRegion() {
-             allViewsNotInSingleRegion = true;
+         public byte[] getTenantId() {
+             return tenantId;
          }
  
-         private List<Result> getResults() {
-             return results;
+         public byte[] getSchemaName() {
+             return schemaName;
          }
  
-         /**
-          * Returns true is the table has views and they are all in the same HBase region.
-          */
-         private boolean allViewsInSingleRegion() {
-             return results.size() > 0 && !allViewsNotInSingleRegion;
-         }
- 
-         /**
-          * Returns true is the table has views and they are all NOT in the same HBase region.
-          */
-         private boolean allViewsInMultipleRegions() {
-             return results.size() > 0 && allViewsNotInSingleRegion;
+         public byte[] getViewName() {
+             return viewName;
          }
 +        
 +        private void addResult(TableViewFinderResult result) {
 +            this.results.addAll(result.getResults());
 +            if (result.allViewsInMultipleRegions()) {
 +                this.setAllViewsNotInSingleRegion();
 +            }
 +        }
      }
  
      @Override


[30/50] [abbrv] phoenix git commit: PHOENIX-4446 Sequence table region opening failing because of property setting attempt on read-only configuration(Rajeshbabu)

Posted by td...@apache.org.
PHOENIX-4446 Sequence table region opening failing because of property setting attempt on read-only configuration(Rajeshbabu)


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

Branch: refs/heads/system-catalog
Commit: 334eb15b4a7a80ce8d4e1c1dc09b7724663fc4da
Parents: c075a17
Author: Rajeshbabu Chintaguntla <ra...@apache.org>
Authored: Sat Dec 9 09:48:59 2017 +0530
Committer: Rajeshbabu Chintaguntla <ra...@apache.org>
Committed: Sat Dec 9 09:48:59 2017 +0530

----------------------------------------------------------------------
 .../java/org/apache/phoenix/hbase/index/write/IndexWriter.java  | 3 ++-
 .../main/java/org/apache/phoenix/index/PhoenixIndexBuilder.java | 5 -----
 2 files changed, 2 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/334eb15b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/IndexWriter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/IndexWriter.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/IndexWriter.java
index 6b57025..4e5e182 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/IndexWriter.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/write/IndexWriter.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.hbase.util.Pair;
 import org.apache.phoenix.hbase.index.exception.IndexWriteException;
 import org.apache.phoenix.hbase.index.table.HTableInterfaceReference;
 import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
+import org.apache.phoenix.index.PhoenixIndexFailurePolicy;
 
 import com.google.common.collect.ArrayListMultimap;
 import com.google.common.collect.Multimap;
@@ -88,7 +89,7 @@ public class IndexWriter implements Stoppable {
     Configuration conf = env.getConfiguration();
     try {
       IndexFailurePolicy committer =
-          conf.getClass(INDEX_FAILURE_POLICY_CONF_KEY, KillServerOnFailurePolicy.class,
+          conf.getClass(INDEX_FAILURE_POLICY_CONF_KEY, PhoenixIndexFailurePolicy.class,
             IndexFailurePolicy.class).newInstance();
       return committer;
     } catch (InstantiationException e) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/334eb15b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexBuilder.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexBuilder.java b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexBuilder.java
index 679c5df..8b1e2f1 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexBuilder.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexBuilder.java
@@ -102,11 +102,6 @@ public class PhoenixIndexBuilder extends NonTxIndexBuilder {
     @Override
     public void setup(RegionCoprocessorEnvironment env) throws IOException {
         super.setup(env);
-        Configuration conf = env.getConfiguration();
-        // Install handler that will attempt to disable the index first before killing the region
-        // server
-        conf.setIfUnset(IndexWriter.INDEX_FAILURE_POLICY_CONF_KEY,
-            PhoenixIndexFailurePolicy.class.getName());
     }
 
     @Override


[08/50] [abbrv] phoenix git commit: PHOENIX-4373 Local index variable length key can have trailing nulls while upserting

Posted by td...@apache.org.
PHOENIX-4373 Local index variable length key can have trailing nulls while upserting


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

Branch: refs/heads/system-catalog
Commit: b2d5b4d75d4698981b291fecfac3efa3fb6e2649
Parents: 1d8a6bc
Author: Vincent Poon <vi...@apache.org>
Authored: Tue Nov 14 10:47:46 2017 -0800
Committer: Vincent Poon <vi...@apache.org>
Committed: Tue Nov 14 10:47:46 2017 -0800

----------------------------------------------------------------------
 .../org/apache/phoenix/end2end/IndexToolIT.java | 40 ++++++++++++++++++++
 .../org/apache/phoenix/compile/ScanRanges.java  | 11 ++++--
 .../apache/phoenix/compile/UpsertCompiler.java  |  3 +-
 3 files changed, 50 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/b2d5b4d7/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolIT.java
index 913a147..a9128ab 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolIT.java
@@ -209,6 +209,46 @@ public class IndexToolIT extends BaseTest {
         }
     }
 
+    @Test
+    public void testSaltedVariableLengthPK() throws Exception {
+        String schemaName = generateUniqueName();
+        String dataTableName = generateUniqueName();
+        String dataTableFullName = SchemaUtil.getTableName(schemaName, dataTableName);
+        String indexTableName = generateUniqueName();
+        try (Connection conn =
+                DriverManager.getConnection(getUrl(), PropertiesUtil.deepCopy(TEST_PROPERTIES))) {
+            String dataDDL =
+                    "CREATE TABLE " + dataTableFullName + "(\n"
+                            + "ID VARCHAR NOT NULL PRIMARY KEY,\n"
+                            + "\"info\".CAR_NUM VARCHAR(18) NULL,\n"
+                            + "\"info\".CAP_DATE VARCHAR NULL,\n" + "\"info\".ORG_ID BIGINT NULL,\n"
+                            + "\"info\".ORG_NAME VARCHAR(255) NULL\n" + ") SALT_BUCKETS=3";
+            conn.createStatement().execute(dataDDL);
+
+            String upsert =
+                    "UPSERT INTO " + dataTableFullName
+                            + "(ID,CAR_NUM,CAP_DATE,ORG_ID,ORG_NAME) VALUES('1','car1','2016-01-01 00:00:00',11,'orgname1')";
+            conn.createStatement().execute(upsert);
+            conn.commit();
+
+            String indexDDL =
+                    String.format(
+                        "CREATE %s INDEX %s on %s (\"info\".CAR_NUM,\"info\".CAP_DATE) ASYNC",
+                        (localIndex ? "LOCAL" : ""), indexTableName, dataTableFullName);
+            conn.createStatement().execute(indexDDL);
+
+            runIndexTool(directApi, useSnapshot, schemaName, dataTableName, indexTableName);
+
+            ResultSet rs =
+                    conn.createStatement().executeQuery(
+                        "SELECT ORG_ID,CAP_DATE,CAR_NUM,ORG_NAME FROM " + dataTableFullName
+                                + " WHERE CAR_NUM='car1' AND CAP_DATE>='2016-01-01' AND CAP_DATE<='2016-05-02' LIMIT 10");
+            assertTrue(rs.next());
+            int orgId = rs.getInt(1);
+            assertEquals(11, orgId);
+        }
+    }
+
     public static void assertExplainPlan(boolean localIndex, String actualExplainPlan,
             String dataTableFullName, String indexTableFullName) {
         String expectedExplainPlan;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b2d5b4d7/phoenix-core/src/main/java/org/apache/phoenix/compile/ScanRanges.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/ScanRanges.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/ScanRanges.java
index 1b94cff..817c1bc 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/ScanRanges.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/ScanRanges.java
@@ -207,13 +207,18 @@ public class ScanRanges {
     }
     
     public static byte[] prefixKey(byte[] key, int keyOffset, byte[] prefixKey, int prefixKeyOffset) {
-        if (key.length > 0) {
-            byte[] newKey = new byte[key.length + prefixKeyOffset];
+        return prefixKey(key, keyOffset, key.length, prefixKey, prefixKeyOffset);
+    }
+
+    public static byte[] prefixKey(byte[] key, int keyOffset, int keyLength, byte[] prefixKey,
+            int prefixKeyOffset) {
+        if (keyLength > 0) {
+            byte[] newKey = new byte[keyLength + prefixKeyOffset];
             int totalKeyOffset = keyOffset + prefixKeyOffset;
             if (prefixKey.length >= totalKeyOffset) { // otherwise it's null padded
                 System.arraycopy(prefixKey, 0, newKey, 0, totalKeyOffset);
             }
-            System.arraycopy(key, keyOffset, newKey, totalKeyOffset, key.length - keyOffset);
+            System.arraycopy(key, keyOffset, newKey, totalKeyOffset, keyLength - keyOffset);
             return newKey;
         } 
         return key;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/b2d5b4d7/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 3603ce7..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
@@ -162,7 +162,8 @@ public class UpsertCompiler {
                             .getRegionInfo().getEndKey().length] : region.getRegionInfo()
                             .getStartKey();
             if (regionPrefix.length != 0) {
-                ptr.set(ScanRanges.prefixKey(ptr.get(), 0, regionPrefix, regionPrefix.length));
+                ptr.set(ScanRanges.prefixKey(ptr.get(), 0, ptr.getLength(), regionPrefix,
+                    regionPrefix.length));
             }
         } 
         mutation.put(ptr, new RowMutationState(columnValues, statement.getConnection().getStatementExecutionCounter(), rowTsColInfo, onDupKeyBytes));


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

Posted by td...@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/4e0c0a33
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/4e0c0a33
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/4e0c0a33

Branch: refs/heads/system-catalog
Commit: 4e0c0a33ed8b401f7785dde8979041dd5ab9a1f4
Parents: c216b66
Author: Thomas D'Silva <td...@apache.org>
Authored: Mon Nov 20 19:13:53 2017 -0800
Committer: Thomas D'Silva <td...@apache.org>
Committed: Tue Nov 21 12:26:13 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/4e0c0a33/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/4e0c0a33/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/4e0c0a33/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/4e0c0a33/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 b5a55b8..7462baa 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/4e0c0a33/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/4e0c0a33/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);


[04/50] [abbrv] phoenix git commit: Store parent -> child view links in SYSTEM.CHILD_LINK table

Posted by td...@apache.org.
Store parent -> child view links in SYSTEM.CHILD_LINK table


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

Branch: refs/heads/system-catalog
Commit: 758efd9b14e61bcb99ebbb0724a948b5f4e8ed11
Parents: 57d8bc4
Author: Thomas D'Silva <td...@apache.org>
Authored: Sat Nov 11 22:32:35 2017 -0800
Committer: Thomas D'Silva <td...@apache.org>
Committed: Sat Nov 11 22:32:35 2017 -0800

----------------------------------------------------------------------
 .../phoenix/end2end/AlterTableWithViewsIT.java  |   2 +-
 .../org/apache/phoenix/end2end/BaseViewIT.java  |   2 +-
 .../MigrateSystemTablesToSystemNamespaceIT.java |   4 +-
 .../end2end/QueryDatabaseMetaDataIT.java        |   5 +
 .../end2end/TenantSpecificTablesDDLIT.java      |   2 +
 .../java/org/apache/phoenix/end2end/ViewIT.java |   1 -
 .../SystemCatalogWALEntryFilterIT.java          |  18 +-
 .../apache/phoenix/coprocessor/CatalogInfo.java |  90 ----------
 .../coprocessor/MetaDataEndpointImpl.java       | 174 +++++++++++--------
 .../phoenix/coprocessor/MetaDataProtocol.java   |   3 +-
 .../phoenix/coprocessor/OrphanCleaner.java      |   3 +-
 .../apache/phoenix/coprocessor/ViewFinder.java  |   6 +-
 .../phoenix/jdbc/PhoenixDatabaseMetaData.java   |   5 +
 .../query/ConnectionQueryServicesImpl.java      |   8 +-
 .../query/ConnectionlessQueryServicesImpl.java  |   5 +
 .../apache/phoenix/query/QueryConstants.java    |  17 ++
 .../SystemCatalogWALEntryFilter.java            |   7 +-
 .../java/org/apache/phoenix/schema/PTable.java  |   5 +-
 .../org/apache/phoenix/util/MetaDataUtil.java   |  29 +++-
 .../org/apache/phoenix/util/SchemaUtil.java     |   6 +
 .../org/apache/phoenix/util/UpgradeUtil.java    |  29 ++++
 .../coprocessor/MetaDataEndpointImplTest.java   |   6 +-
 22 files changed, 232 insertions(+), 195 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/758efd9b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
index 51e664b..94dc02d 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
@@ -792,7 +792,7 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
 
 			// PK2 should be in both views
 			sql = "SELECT PK2 FROM " + view1;
-			viewConn2.createStatement().execute(sql);
+			viewConn.createStatement().execute(sql);
 			sql = "SELECT PK2 FROM " + view2;
 			viewConn2.createStatement().execute(sql);
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/758efd9b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseViewIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseViewIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseViewIT.java
index de5d993..478b234 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseViewIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseViewIT.java
@@ -74,7 +74,7 @@ public abstract class BaseViewIT extends ParallelStatsEnabledIT {
     
     @Parameters(name="transactional = {0}")
     public static Collection<Boolean> data() {
-        return Arrays.asList(new Boolean[] { /*false, */true });
+        return Arrays.asList(new Boolean[] { false, true });
     }
     
     protected void testUpdatableViewWithIndex(Integer saltBuckets, boolean localIndex) throws Exception {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/758efd9b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java
index 91e34be..ebd4c35 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java
@@ -58,10 +58,10 @@ public class MigrateSystemTablesToSystemNamespaceIT extends BaseTest {
 
     private static final Set<String> PHOENIX_SYSTEM_TABLES = new HashSet<>(Arrays.asList(
             "SYSTEM.CATALOG", "SYSTEM.SEQUENCE", "SYSTEM.STATS", "SYSTEM.FUNCTION",
-            "SYSTEM.MUTEX"));
+            "SYSTEM.MUTEX", "SYSTEM.CHILD_LINK"));
     private static final Set<String> PHOENIX_NAMESPACE_MAPPED_SYSTEM_TABLES = new HashSet<>(
             Arrays.asList("SYSTEM:CATALOG", "SYSTEM:SEQUENCE", "SYSTEM:STATS", "SYSTEM:FUNCTION",
-                    "SYSTEM:MUTEX"));
+                    "SYSTEM:MUTEX", "SYSTEM:CHILD_LINK"));
     private static final String SCHEMA_NAME = "MIGRATETEST";
     private static final String TABLE_NAME =
             SCHEMA_NAME + "." + MigrateSystemTablesToSystemNamespaceIT.class.getSimpleName().toUpperCase();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/758efd9b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryDatabaseMetaDataIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryDatabaseMetaDataIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryDatabaseMetaDataIT.java
index eff053c..eb5bb4b 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryDatabaseMetaDataIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryDatabaseMetaDataIT.java
@@ -19,6 +19,7 @@ package org.apache.phoenix.end2end;
 
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_TABLE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_TABLE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_FUNCTION_TABLE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TYPE_SEQUENCE;
 import static org.apache.phoenix.util.TestUtil.ATABLE_NAME;
@@ -134,6 +135,10 @@ public class QueryDatabaseMetaDataIT extends ParallelStatsDisabledIT {
             assertEquals(PTableType.SYSTEM.toString(), rs.getString("TABLE_TYPE"));
             assertTrue(rs.next());
             assertEquals(SYSTEM_CATALOG_SCHEMA, rs.getString("TABLE_SCHEM"));
+            assertEquals(SYSTEM_CHILD_LINK_TABLE, rs.getString("TABLE_NAME"));
+            assertEquals(PTableType.SYSTEM.toString(), rs.getString("TABLE_TYPE"));
+            assertTrue(rs.next());
+            assertEquals(SYSTEM_CATALOG_SCHEMA, rs.getString("TABLE_SCHEM"));
             assertEquals(SYSTEM_FUNCTION_TABLE, rs.getString("TABLE_NAME"));
             assertEquals(PTableType.SYSTEM.toString(), rs.getString("TABLE_TYPE"));
             assertTrue(rs.next());

http://git-wip-us.apache.org/repos/asf/phoenix/blob/758efd9b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java
index 1e6f59c..6830d54 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java
@@ -491,6 +491,8 @@ public class TenantSpecificTablesDDLIT extends BaseTenantSpecificTablesIT {
             assertTrue(rs.next());
             assertTableMetaData(rs, PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA, PhoenixDatabaseMetaData.SYSTEM_CATALOG_TABLE, PTableType.SYSTEM);
             assertTrue(rs.next());
+            assertTableMetaData(rs, PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA, PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_TABLE, PTableType.SYSTEM);
+            assertTrue(rs.next());
             assertTableMetaData(rs, SYSTEM_CATALOG_SCHEMA, SYSTEM_FUNCTION_TABLE, SYSTEM);
             assertTrue(rs.next());
             assertTableMetaData(rs, PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA, PhoenixDatabaseMetaData.TYPE_SEQUENCE, PTableType.SYSTEM);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/758efd9b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java
index f32ec9b..e74ddba 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java
@@ -51,7 +51,6 @@ import org.apache.phoenix.schema.TableNotFoundException;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.SchemaUtil;
-import org.junit.Ignore;
 import org.junit.Test;
 
 public class ViewIT extends BaseViewIT {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/758efd9b/phoenix-core/src/it/java/org/apache/phoenix/replication/SystemCatalogWALEntryFilterIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/replication/SystemCatalogWALEntryFilterIT.java b/phoenix-core/src/it/java/org/apache/phoenix/replication/SystemCatalogWALEntryFilterIT.java
index 776e300..0be5665 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/replication/SystemCatalogWALEntryFilterIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/replication/SystemCatalogWALEntryFilterIT.java
@@ -69,9 +69,12 @@ public class SystemCatalogWALEntryFilterIT extends ParallelStatsDisabledIT {
   private static final String DROP_TENANT_VIEW_SQL = "DROP VIEW IF EXISTS " + TENANT_VIEW_NAME;
   private static final String DROP_NONTENANT_VIEW_SQL = "DROP VIEW IF EXISTS " + NONTENANT_VIEW_NAME;
   private static PTable catalogTable;
+  private static PTable childLinkTable;
   private static WALKey walKey = null;
   private static TableName systemCatalogTableName =
       TableName.valueOf(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME);
+  private static TableName systemChildLinkTableName =
+	      TableName.valueOf(PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_NAME);
 
 
   @BeforeClass
@@ -85,6 +88,7 @@ public class SystemCatalogWALEntryFilterIT extends ParallelStatsDisabledIT {
       ensureTableCreated(getUrl(), TestUtil.ENTITY_HISTORY_TABLE_NAME);
       connection.createStatement().execute(CREATE_TENANT_VIEW_SQL);
       catalogTable = PhoenixRuntime.getTable(connection, PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME);
+      childLinkTable = PhoenixRuntime.getTable(connection, PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_NAME);
       walKey = new WALKey(REGION, TableName.valueOf(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME), 0, 0, uuid);
     };
     Assert.assertNotNull(catalogTable);
@@ -127,15 +131,15 @@ public class SystemCatalogWALEntryFilterIT extends ParallelStatsDisabledIT {
     Get nonTenantViewGet = getTenantViewGet(catalogTable,
         DEFAULT_TENANT_BYTES, NONTENANT_VIEW_NAME);
 
-    Get tenantLinkGet = getParentChildLinkGet(catalogTable, TENANT_BYTES, TENANT_VIEW_NAME);
-    Get nonTenantLinkGet = getParentChildLinkGet(catalogTable,
+    Get tenantLinkGet = getParentChildLinkGet(childLinkTable, TENANT_BYTES, TENANT_VIEW_NAME);
+    Get nonTenantLinkGet = getParentChildLinkGet(childLinkTable,
         DEFAULT_TENANT_BYTES, NONTENANT_VIEW_NAME);
 
     WAL.Entry nonTenantViewEntry = getEntry(systemCatalogTableName, nonTenantViewGet);
     WAL.Entry tenantViewEntry = getEntry(systemCatalogTableName, tenantViewGet);
 
-    WAL.Entry nonTenantLinkEntry = getEntry(systemCatalogTableName, nonTenantLinkGet);
-    WAL.Entry tenantLinkEntry = getEntry(systemCatalogTableName, tenantLinkGet);
+    WAL.Entry nonTenantLinkEntry = getEntry(systemChildLinkTableName, nonTenantLinkGet);
+    WAL.Entry tenantLinkEntry = getEntry(systemChildLinkTableName, tenantLinkGet);
 
     //verify that the tenant view WAL.Entry passes the filter and the non-tenant view does not
     SystemCatalogWALEntryFilter filter = new SystemCatalogWALEntryFilter();
@@ -191,8 +195,8 @@ public class SystemCatalogWALEntryFilterIT extends ParallelStatsDisabledIT {
     return new Get(key.copyBytes());
   }
 
-  public Get getParentChildLinkGet(PTable catalogTable, byte[] tenantBytes, String viewName) {
-    /* For parent-child link, the system.catalog key becomes
+  public Get getParentChildLinkGet(PTable linkTable, byte[] tenantBytes, String viewName) {
+    /* For parent-child link, the system.child_link key becomes
       1. Parent tenant id
       2. Parent Schema
       3. Parent Table name
@@ -206,7 +210,7 @@ public class SystemCatalogWALEntryFilterIT extends ParallelStatsDisabledIT {
     tenantKeyParts[3] = tenantBytes;
     tenantKeyParts[4] = Bytes.toBytes(SchemaUtil.getTableName(SCHEMA_NAME.toUpperCase(), viewName.toUpperCase()));
     ImmutableBytesWritable key = new ImmutableBytesWritable();
-    catalogTable.newKey(key, tenantKeyParts);
+    linkTable.newKey(key, tenantKeyParts);
     //the backing byte array of key might have extra space at the end.
     // need to just slice "the good parts" which we do by calling copyBytes
     return new Get(key.copyBytes());

http://git-wip-us.apache.org/repos/asf/phoenix/blob/758efd9b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/CatalogInfo.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/CatalogInfo.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/CatalogInfo.java
deleted file mode 100644
index 52d7fad..0000000
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/CatalogInfo.java
+++ /dev/null
@@ -1,90 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * http://www.apache.org/licenses/LICENSE-2.0
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.phoenix.coprocessor;
-
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import org.apache.hadoop.hbase.client.Mutation;
-import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.phoenix.util.MetaDataUtil;
-
-import java.util.Iterator;
-import java.util.LinkedHashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-
-public class CatalogInfo {
-
-    private List<Mutation> allMutations;
-
-    public CatalogInfo(List<Mutation> mutations) {
-        this.allMutations = mutations;
-    }
-
-    public LinkedList<MutationType> getGroupedMutations() {
-        LinkedList<MutationType> result = Lists.newLinkedList();
-        Iterator<Map.Entry<String, MutationType>> iterator = groupByTable().entrySet().iterator();
-        while (iterator.hasNext()) {
-            result.addLast(iterator.next().getValue());
-        }
-        return result;
-    }
-
-    private LinkedHashMap<String, MutationType> groupByTable() {
-        LinkedHashMap<String, MutationType> map = Maps.newLinkedHashMap();
-        for (Mutation dataMutation : allMutations) {
-            String groupBy = Bytes.toString(MetaDataUtil.getTenantIdAndSchemaAndTableName(dataMutation));
-            if (!map.containsKey(groupBy)) {
-                map.put(groupBy, new MutationType());
-            }
-            map.get(groupBy).addMutation(dataMutation);
-        }
-        return map;
-    }
-
-    static class MutationType {
-        private final List<Mutation> dataMutations;
-        private final List<Mutation> linkMutations;
-
-        public MutationType() {
-            this.dataMutations = Lists.newArrayList();
-            this.linkMutations = Lists.newArrayList();
-        }
-
-        public void addMutation(Mutation mutation) {
-            if (MetaDataUtil.isLinkingRow(mutation)) {
-                this.linkMutations.add(mutation);
-            } else {
-                this.dataMutations.add(mutation);
-            }
-        }
-
-        public List<Mutation> getDataMutations() {
-            return dataMutations;
-        }
-
-        public List<Mutation> getLinkMutations() {
-            return linkMutations;
-        }
-
-        public List<Mutation> getAllMutations() {
-            List<Mutation> result = Lists.newArrayList(dataMutations);
-            result.addAll(linkMutations);
-            return result;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/758efd9b/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 1ccd1fb..9f7a629 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
@@ -104,10 +104,13 @@ import org.apache.hadoop.hbase.Coprocessor;
 import org.apache.hadoop.hbase.CoprocessorEnvironment;
 import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
 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.HTable;
 import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Put;
@@ -1199,7 +1202,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
               } else if (linkType == LinkType.PARENT_TABLE) {
                   parentTableName = PNameFactory.newName(SchemaUtil.getTableNameFromFullName(famName.getBytes()));
                   parentSchemaName = PNameFactory.newName(SchemaUtil.getSchemaNameFromFullName(famName.getBytes()));
-              } else if (linkType == LinkType.DROPPED_COLUMN) {
+              } else if (linkType == LinkType.EXCLUDED_COLUMN) {
                   // add the excludedColumn
                   addExcludedColumnToTable(columns, colName, famName, colKv.getTimestamp());
               }
@@ -1583,13 +1586,14 @@ 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];
-            String fullName = SchemaUtil.getTableName(schemaName, tableName);
-            // no need to run OrpanCleaner (which cleans up orphaned views) on SYSTEM tables 
-            if (!schemaName.equals(PhoenixDatabaseMetaData.SYSTEM_SCHEMA_NAME)) {
+            // no need to run OrpanCleaner (which cleans up orphaned views) while creating SYSTEM tables  env.getTable
+            if (Bytes.compareTo(schemaName,PhoenixDatabaseMetaData.SYSTEM_SCHEMA_NAME_BYTES)!=0) {
 	            HTableInterface systemCatalog = null;
 	            try {
-	                systemCatalog = env.getTable(SchemaUtil
-	                        .getPhysicalTableName(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES, env.getConfiguration()));
+	            	// can't use SchemaUtil.getPhysicalTableName on server side as we don't know whether 
+	            	// the system tables have been migrated to the system namespaces
+	            	TableName systemCatalogTableName = env.getRegion().getTableDesc().getTableName();
+	                systemCatalog = env.getTable(systemCatalogTableName);
 	                OrphanCleaner.reapOrphans(systemCatalog, tenantIdBytes, schemaName, tableName);
 	            } finally {
 	                if (systemCatalog != null) {
@@ -1870,7 +1874,24 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                         indexId = (short) seqValue;
                     }
                 }
-
+                
+                // the child links are stored in a separate table SYSTEM.CHILD_LINK from 4.14 onwards
+                List<Mutation> childLinkMutations = MetaDataUtil.removeChildLinks(tableMetadata);
+                HTableInterface hTable = null;
+                try {
+                	hTable = env.getTable(SchemaUtil
+	                        .getPhysicalTableName(PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_NAME_BYTES, env.getConfiguration()));
+                    hTable.batch(childLinkMutations);
+                } catch (Throwable t) {
+                    logger.error("creating child links failed", t);
+                    ProtobufUtil.setControllerException(controller,
+                        ServerUtil.createIOException(SchemaUtil.getTableName(schemaName, tableName), t));
+                } finally {
+                    if (hTable != null) {
+                        hTable.close();
+                    }
+                }
+                
                 // TODO: Switch this to HRegion#batchMutate when we want to support indexes on the
                 // system table. Basically, we get all the locks that we don't already hold for all the
                 // tableMetadata rows. This ensures we don't have deadlock situations (ensuring
@@ -1980,7 +2001,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
 
     private void findAllChildViews(byte[] tenantId, byte[] schemaName, byte[] tableName, TableViewFinderResult result) throws IOException {
     	HTableInterface hTable = env.getTable(SchemaUtil
-                .getPhysicalTableName(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES, env.getConfiguration()));
+                .getPhysicalTableName(PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_NAME_BYTES, env.getConfiguration()));
         try {
             ViewFinder.findAllRelatives(hTable, tenantId, schemaName, tableName, LinkType.CHILD_TABLE, result);
         } finally {
@@ -1988,8 +2009,17 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         }
     }
     
-    private static final byte[] PHYSICAL_TABLE_BYTES =
-            new byte[] { PTable.LinkType.PHYSICAL_TABLE.getSerializedValue() };
+	private void separateLocalAndRemoteMutations(Region region, List<Mutation> mutations,
+			List<Mutation> localRegionMutations, List<Mutation> remoteRegionMutations) {
+		HRegionInfo regionInfo = region.getRegionInfo();
+		for (Mutation mutation : mutations) {
+			if (regionInfo.containsRow(mutation.getRow())) {
+				localRegionMutations.add(mutation);
+			} else {
+				remoteRegionMutations.add(mutation);
+			}
+		}
+	}
 
     @Override
     public void dropTable(RpcController controller, DropTableRequest request,
@@ -2002,8 +2032,11 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         byte[] tableName = null;
 
         try {
-            List<Mutation> tableMetadata = ProtobufUtil.getMutations(request);
-            MetaDataUtil.getTenantIdAndSchemaAndTableName(tableMetadata, rowKeyMetaData);
+            List<Mutation> catalogMutations = ProtobufUtil.getMutations(request);
+            List<Mutation> childLinkMutations = Lists.newArrayList();
+        	List<Mutation> localRegionMutations = Lists.newArrayList();
+			List<Mutation> remoteRegionMutations = Lists.newArrayList();
+            MetaDataUtil.getTenantIdAndSchemaAndTableName(catalogMutations, rowKeyMetaData);
             byte[] tenantIdBytes = rowKeyMetaData[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
             schemaName = rowKeyMetaData[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX];
             tableName = rowKeyMetaData[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
@@ -2016,7 +2049,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             }
             List<byte[]> tableNamesToDelete = Lists.newArrayList();
             List<SharedTableState> sharedTablesToDelete = Lists.newArrayList();
-            byte[] parentTableName = MetaDataUtil.getParentTableName(tableMetadata);
+            byte[] parentTableName = MetaDataUtil.getParentTableName(catalogMutations);
             byte[] lockTableName = parentTableName == null ? tableName : parentTableName;
             byte[] lockKey = SchemaUtil.getTableKey(tenantIdBytes, schemaName, lockTableName);
             byte[] key =
@@ -2030,7 +2063,6 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 return;
             }
             List<RowLock> locks = Lists.newArrayList();
-            LinkedList<CatalogInfo.MutationType> groupedMutations = Lists.newLinkedList();
             try {
                 acquireLock(region, lockKey, locks);
                 if (key != lockKey) {
@@ -2039,21 +2071,26 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 List<ImmutableBytesPtr> invalidateList = new ArrayList<ImmutableBytesPtr>();
                 result =
                         doDropTable(key, tenantIdBytes, schemaName, tableName, parentTableName,
-                            PTableType.fromSerializedValue(tableType), tableMetadata,
-                            invalidateList, locks, tableNamesToDelete, sharedTablesToDelete, isCascade, request.getClientVersion());
+                            PTableType.fromSerializedValue(tableType), catalogMutations, childLinkMutations,
+                            invalidateList, tableNamesToDelete, sharedTablesToDelete, isCascade, request.getClientVersion());
                 if (result.getMutationCode() != MutationCode.TABLE_ALREADY_EXISTS) {
                     done.run(MetaDataMutationResult.toProto(result));
                     return;
                 }
-                Cache<ImmutableBytesPtr,PMetaDataEntity> metaDataCache = GlobalCache.getInstance(this.env).getMetaDataCache();
-                groupedMutations = new CatalogInfo(tableMetadata).getGroupedMutations();
-                // for the parent table, we don't want to delete the link, this happens last.
-                CatalogInfo.MutationType originalMutations = groupedMutations.getFirst();
-                region.mutateRowsWithLocks(originalMutations.getDataMutations(),
-                    Collections.<byte[]>emptyList(), HConstants.NO_NONCE, HConstants.NO_NONCE);
-                logger.debug("Issuing Deletes: " + originalMutations.getDataMutations());
-
-                long currentTime = MetaDataUtil.getClientTimeStamp(tableMetadata);
+				Cache<ImmutableBytesPtr, PMetaDataEntity> metaDataCache = GlobalCache.getInstance(this.env)
+						.getMetaDataCache();
+				// since the mutations in catalogMutations can span multiple
+				// regions first we first process process mutations local to
+				// this region, then we process the remaining mutations, finally
+				// we process the child link mutations if any of the mutations
+				// fail, we can will clean them up later using
+				// OrphanCleaner.reapOrphans()
+				separateLocalAndRemoteMutations(region, catalogMutations, localRegionMutations, remoteRegionMutations);
+				// drop rows from catalog on this region
+				region.mutateRowsWithLocks(localRegionMutations, Collections.<byte[]> emptyList(), HConstants.NO_NONCE,
+						HConstants.NO_NONCE);
+
+                long currentTime = MetaDataUtil.getClientTimeStamp(catalogMutations);
                 for (ImmutableBytesPtr ckey : invalidateList) {
                     metaDataCache.put(ckey, newDeletedTableMarker(currentTime));
                 }
@@ -2065,35 +2102,10 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 return;
             } finally {
                 region.releaseRowLocks(locks);
-                HTableInterface hTable = null;
-                try {
-                	hTable = env.getTable(SchemaUtil
-	                        .getPhysicalTableName(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES, env.getConfiguration()));
-                    // Now batch each table and delete, any table / view should not span more than a single
-                    // region so either this succeeds or fails.
-                    for (int i = groupedMutations.size() - 1; i >= 1; i--) {
-                        List<Mutation> mutationsToApply = groupedMutations.get(i).getAllMutations();
-                        logger.info("Issuing Deletes: " + mutationsToApply);
-                        Object[] appliedMutations = new Object[mutationsToApply.size()];
-                        hTable.batch(mutationsToApply, appliedMutations);
-                    }
-                    // now we can finally delete that linking row for the original table and we are done
-                    // if there are deletes:
-                    if (!groupedMutations.isEmpty()) {
-                        List<Mutation> linkMutations = groupedMutations.getFirst().getLinkMutations();
-                        logger.info("Issuing Deletes: " + linkMutations);
-                        Object[] appliedMutations = new Object[linkMutations.size()];
-                        hTable.batch(linkMutations, appliedMutations);
-                    }
-                } catch (Throwable t) {
-                    logger.error("dropTable failed", t);
-                    ProtobufUtil.setControllerException(controller,
-                        ServerUtil.createIOException(SchemaUtil.getTableName(schemaName, tableName), t));
-                } finally {
-                    if (hTable != null) {
-                        hTable.close();
-                    }
-                }
+                // drop rows from catalog on remote regions
+                processMutations(controller, PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES, SchemaUtil.getTableName(schemaName, tableName), remoteRegionMutations);
+                // drop all child links 
+                processMutations(controller, PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_NAME_BYTES, SchemaUtil.getTableName(schemaName, tableName), childLinkMutations);
             }
         } catch (Throwable t) {
           logger.error("dropTable failed", t);
@@ -2102,13 +2114,28 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         }
     }
 
-    private MetaDataMutationResult doDropTable(byte[] key, byte[] tenantId, byte[] schemaName,
-        byte[] tableName, byte[] parentTableName, PTableType tableType, List<Mutation> rowsToDelete,
-        List<ImmutableBytesPtr> invalidateList, List<RowLock> locks,
-        List<byte[]> tableNamesToDelete, List<SharedTableState> sharedTablesToDelete, boolean isCascade, int clientVersion) throws IOException, SQLException {
-
+	private void processMutations(RpcController controller, byte[] systemTableName, String droppedTableName,
+			List<Mutation> childLinkMutations) throws IOException {
+		HTableInterface hTable = null;
+		try {
+			hTable = env.getTable(SchemaUtil.getPhysicalTableName(systemTableName, env.getConfiguration()));
+			hTable.batch(childLinkMutations);
+		} catch (Throwable t) {
+			logger.error("dropTable failed", t);
+			ProtobufUtil.setControllerException(controller, ServerUtil.createIOException(droppedTableName, t));
+		} finally {
+			if (hTable != null) {
+				hTable.close();
+			}
+		}
+	}
 
-        long clientTimeStamp = MetaDataUtil.getClientTimeStamp(rowsToDelete);
+	private MetaDataMutationResult doDropTable(byte[] key, byte[] tenantId, byte[] schemaName, byte[] tableName,
+			byte[] parentTableName, PTableType tableType, List<Mutation> catalogMutations,
+			List<Mutation> childLinkMutations, List<ImmutableBytesPtr> invalidateList, List<byte[]> tableNamesToDelete,
+			List<SharedTableState> sharedTablesToDelete, boolean isCascade, int clientVersion)
+			throws IOException, SQLException {
+        long clientTimeStamp = MetaDataUtil.getClientTimeStamp(catalogMutations);
 
         Region region = env.getRegion();
         ImmutableBytesPtr cacheKey = new ImmutableBytesPtr(key);
@@ -2159,17 +2186,16 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 findAllChildViews(tenantId, table.getSchemaName().getBytes(), table.getTableName().getBytes(), tableViewFinderResult);
                 if (tableViewFinderResult.hasViews()) {
                     if (isCascade) {
-                        // Recursively delete views - safe as all the views as all in the same region
-
+                        // Recursively delete views adding the mutations to delete child views to rowsToDelete
                         for (TableInfo tableInfo : tableViewFinderResult.getResults()) {
                             byte[] viewTenantId = tableInfo.getTenantId();
                             byte[] viewSchemaName = tableInfo.getSchemaName();
                             byte[] viewName = tableInfo.getTableName();
                             byte[] viewKey = tableInfo.getRowKeyPrefix();
                             Delete delete = new Delete(tableInfo.getRowKeyPrefix(), clientTimeStamp);
-                            rowsToDelete.add(delete);
+                            catalogMutations.add(delete);
                             MetaDataMutationResult result = doDropTable(viewKey, viewTenantId, viewSchemaName,
-                                    viewName, null, PTableType.VIEW, rowsToDelete, invalidateList, locks,
+                                    viewName, null, PTableType.VIEW, catalogMutations, childLinkMutations, invalidateList,
                                     tableNamesToDelete, sharedTablesToDelete, false, clientVersion);
                             if (result.getMutationCode() != MutationCode.TABLE_ALREADY_EXISTS) {
                                 return result;
@@ -2208,7 +2234,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                             PName parentTenantId = parentTenantIdCell!=null ? PNameFactory.newName(parentTenantIdCell.getValueArray(), parentTenantIdCell.getValueOffset(), parentTenantIdCell.getValueLength()) : null;
                             byte[] linkKey = MetaDataUtil.getChildLinkKey(parentTenantId, table.getParentSchemaName(), table.getParentTableName(), table.getTenantId(), table.getName());
                             Delete linkDelete = new Delete(linkKey, clientTimeStamp);
-                            rowsToDelete.add(linkDelete);
+                            childLinkMutations.add(linkDelete);
                         }
                 }
                 // FIXME: Remove when unintentionally deprecated method is fixed (HBASE-7870).
@@ -2216,7 +2242,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 // in 0.94.4, thus if we try to use it here we can no longer use the 0.94.2 version
                 // of the client.
                 Delete delete = new Delete(kv.getRowArray(), kv.getRowOffset(), kv.getRowLength(), clientTimeStamp);
-                rowsToDelete.add(delete);
+                catalogMutations.add(delete);
                 results.clear();
                 scanner.next(results);
             } while (!results.isEmpty());
@@ -2230,10 +2256,10 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             // in 0.94.4, thus if we try to use it here we can no longer use the 0.94.2 version
             // of the client.
             Delete delete = new Delete(indexKey, clientTimeStamp);
-            rowsToDelete.add(delete);
+            catalogMutations.add(delete);
             MetaDataMutationResult result =
                     doDropTable(indexKey, tenantId, schemaName, indexName, tableName, PTableType.INDEX,
-                        rowsToDelete, invalidateList, locks, tableNamesToDelete, sharedTablesToDelete, false, clientVersion);
+                        catalogMutations, childLinkMutations, invalidateList, tableNamesToDelete, sharedTablesToDelete, false, clientVersion);
             if (result.getMutationCode() != MutationCode.TABLE_ALREADY_EXISTS) {
                 return result;
             }
@@ -3165,16 +3191,15 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
 										continue;
                                     deletePKColumn = columnToDelete.getFamilyName() == null;
 									if (isView) {
-                                        // convert a delete into a put row.  I know this is weird but see Phoenix-xxxx jira for
-                                        // reasoning.  Only drop if its in a view and its not created by the view
+                                        // if we are dropping a derived column add it to the excluded column list
                                         if (columnToDelete.isDerived()) {
                                             mutation = MetaDataUtil
-                                                .cloneDeleteToPutAndAddColumn((Delete) mutation, TABLE_FAMILY_BYTES, LINK_TYPE_BYTES, LinkType.DROPPED_COLUMN.getSerializedValueAsByteArray());
+                                                .cloneDeleteToPutAndAddColumn((Delete) mutation, TABLE_FAMILY_BYTES, LINK_TYPE_BYTES, LinkType.EXCLUDED_COLUMN.getSerializedValueAsByteArray());
                                             iterator.set(mutation);
                                         }
 
                                         if (table.getBaseColumnCount() != DIVERGED_VIEW_BASE_COLUMN_COUNT
-                                            && columnToDelete.getPosition() < table.getBaseColumnCount()) {
+                                            && columnToDelete.isDerived()) {
                                             /*
                                              * If the column being dropped is inherited from the base table, then the
                                              * view is about to diverge itself from the base table. The consequence of
@@ -3275,9 +3300,12 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 // Drop the link between the data table and the
                 // index table
                 additionalTableMetaData.add(new Delete(linkKey, clientTimeStamp));
+                List<Mutation> childLinksMutations = Lists.newArrayList();
                 doDropTable(indexKey, tenantId, index.getSchemaName().getBytes(), index
                         .getTableName().getBytes(), tableName, index.getType(),
-                    additionalTableMetaData, invalidateList, locks, tableNamesToDelete, sharedTablesToDelete, false, clientVersion);
+                    additionalTableMetaData, childLinksMutations, invalidateList, tableNamesToDelete, sharedTablesToDelete, false, clientVersion);
+                // there should be no child links to delete since we are just dropping an index
+                assert(childLinksMutations.isEmpty());
                 invalidateList.add(new ImmutableBytesPtr(indexKey));
             }
             // If the dropped column is a covered index column, invalidate the index

http://git-wip-us.apache.org/repos/asf/phoenix/blob/758efd9b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
index fd438a8..dd619eb 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
@@ -92,8 +92,9 @@ public abstract class MetaDataProtocol extends MetaDataService {
     public static final long MIN_SYSTEM_TABLE_TIMESTAMP_4_11_0 = MIN_TABLE_TIMESTAMP + 27;
     // Since there's no upgrade code, keep the version the same as the previous version
     public static final long MIN_SYSTEM_TABLE_TIMESTAMP_4_12_0 = MIN_SYSTEM_TABLE_TIMESTAMP_4_11_0;
+    public static final long MIN_SYSTEM_TABLE_TIMESTAMP_4_13_0 = MIN_TABLE_TIMESTAMP + 28;
     // MIN_SYSTEM_TABLE_TIMESTAMP needs to be set to the max of all the MIN_SYSTEM_TABLE_TIMESTAMP_* constants
-    public static final long MIN_SYSTEM_TABLE_TIMESTAMP = MIN_SYSTEM_TABLE_TIMESTAMP_4_12_0;
+    public static final long MIN_SYSTEM_TABLE_TIMESTAMP = MIN_SYSTEM_TABLE_TIMESTAMP_4_13_0;
     
     // ALWAYS update this map whenever rolling out a new release (major, minor or patch release). 
     // Key is the SYSTEM.CATALOG timestamp for the version and value is the version string.

http://git-wip-us.apache.org/repos/asf/phoenix/blob/758efd9b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/OrphanCleaner.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/OrphanCleaner.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/OrphanCleaner.java
index bd6627a..f8ba130 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/OrphanCleaner.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/OrphanCleaner.java
@@ -42,7 +42,8 @@ class OrphanCleaner {
         }
         for (int i = listOBytes.size() - 1; i >= 0; i--) {
             List<Delete> deletes = traverseUpAndDelete(hTable, listOBytes.get(i));
-            // add the linking row as well if needed
+            // TODO ask rahul if this delete is required
+            // add the linking row as well if needed 
             deletes.add(new Delete(listOBytes.get(i)));
             hTable.delete(deletes);
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/758efd9b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ViewFinder.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ViewFinder.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ViewFinder.java
index 267faed..36a3f9c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ViewFinder.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ViewFinder.java
@@ -60,9 +60,9 @@ class ViewFinder {
                 HConstants.LATEST_TIMESTAMP);
         }
 
-    static void findAllRelatives(Table systemCatalog, byte[] tenantId, byte[] schema, byte[] table,
+    static void findAllRelatives(Table systemTable, byte[] tenantId, byte[] schema, byte[] table,
         PTable.LinkType linkType, TableViewFinderResult result) throws IOException {
-        findAllRelatives(systemCatalog, tenantId, schema, table, linkType, HConstants.LATEST_TIMESTAMP, result);
+        findAllRelatives(systemTable, tenantId, schema, table, linkType, HConstants.LATEST_TIMESTAMP, result);
     }
 
     static void findAllRelatives(Table systemCatalog, byte[] tenantId, byte[] schema, byte[] table,
@@ -77,7 +77,7 @@ class ViewFinder {
 
     static TableViewFinderResult findRelatedViews(Table systemCatalog, byte[] tenantId, byte[] schema, byte[] table,
         PTable.LinkType linkType, long timestamp) throws IOException {
-        if (linkType==PTable.LinkType.INDEX_TABLE || linkType==PTable.LinkType.DROPPED_COLUMN) {
+        if (linkType==PTable.LinkType.INDEX_TABLE || linkType==PTable.LinkType.EXCLUDED_COLUMN) {
             throw new IllegalArgumentException("findAllRelatives does not support link type "+linkType);
         }
         Scan scan = new Scan();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/758efd9b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
index 3f5b8d4..fbb5a42 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
@@ -368,6 +368,11 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData {
     public static final byte[] COLUMN_QUALIFIER_COUNTER_BYTES = Bytes.toBytes(COLUMN_QUALIFIER_COUNTER);
     public static final String USE_STATS_FOR_PARALLELIZATION = "USE_STATS_FOR_PARALLELIZATION";
     public static final byte[] USE_STATS_FOR_PARALLELIZATION_BYTES = Bytes.toBytes(USE_STATS_FOR_PARALLELIZATION);
+    
+    public static final String SYSTEM_CHILD_LINK_TABLE = "CHILD_LINK";
+    public static final String SYSTEM_CHILD_LINK_NAME = SchemaUtil.getTableName(SYSTEM_CATALOG_SCHEMA, SYSTEM_CHILD_LINK_TABLE);
+    public static final byte[] SYSTEM_CHILD_LINK_NAME_BYTES = Bytes.toBytes(SYSTEM_CHILD_LINK_NAME);
+    public static final TableName SYSTEM_LINK_HBASE_TABLE_NAME = TableName.valueOf(SYSTEM_CHILD_LINK_NAME);
 
     PhoenixDatabaseMetaData(PhoenixConnection connection) throws SQLException {
         this.emptyResultSet = new PhoenixResultSet(ResultIterator.EMPTY_ITERATOR, RowProjector.EMPTY_PROJECTOR, new StatementContext(new PhoenixStatement(connection), false));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/758efd9b/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 a198437..3a4a5d6 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
@@ -2564,6 +2564,9 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         try {
             metaConnection.createStatement().execute(QueryConstants.CREATE_FUNCTION_METADATA);
         } catch (TableAlreadyExistsException ignore) {}
+        try {
+            metaConnection.createStatement().executeUpdate(QueryConstants.CREATE_CHILD_LINK_METADATA);
+        } catch (TableAlreadyExistsException e) {}
 
         // Catch the IOException to log the error message and then bubble it up for the client to retry.
         try {
@@ -2822,7 +2825,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                 + PBoolean.INSTANCE.getSqlTypeName());
                     addParentToChildLinks(metaConnection);
                 }
-                if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_12_0) {
+                if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_13_0) {
                     metaConnection = addColumnsIfNotExists(
                         metaConnection,
                         PhoenixDatabaseMetaData.SYSTEM_CATALOG,
@@ -2916,6 +2919,9 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             try {
                 metaConnection.createStatement().executeUpdate(QueryConstants.CREATE_FUNCTION_METADATA);
             } catch (NewerTableAlreadyExistsException e) {} catch (TableAlreadyExistsException e) {}
+            try {
+                metaConnection.createStatement().executeUpdate(QueryConstants.CREATE_CHILD_LINK_METADATA);
+            } catch (NewerTableAlreadyExistsException e) {} catch (TableAlreadyExistsException e) {}
             ConnectionQueryServicesImpl.this.upgradeRequired.set(false);
             success = true;
         } catch (UpgradeInProgressException | UpgradeNotRequiredException e) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/758efd9b/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 b8a2427..440ff47 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
@@ -321,6 +321,11 @@ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices imple
                    metaConnection.createStatement().executeUpdate(QueryConstants.CREATE_FUNCTION_METADATA);
                 } catch (NewerTableAlreadyExistsException ignore) {
                 }
+                
+                try {
+                    metaConnection.createStatement().executeUpdate(QueryConstants.CREATE_CHILD_LINK_METADATA);
+                 } catch (NewerTableAlreadyExistsException ignore) {
+                 }
             } catch (SQLException e) {
                 sqlE = e;
             } finally {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/758efd9b/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..aed22cd 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
@@ -93,6 +93,7 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.STORE_NULLS;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_TABLE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_FUNCTION_TABLE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_TABLE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_STATS_TABLE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_NAME;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_SCHEM;
@@ -401,4 +402,20 @@ public interface QueryConstants {
     public static final byte[] UPGRADE_MUTEX = "UPGRADE_MUTEX".getBytes();
     public static final String HASH_JOIN_CACHE_RETRIES = "hashjoin.client.retries.number";
     public static final int DEFAULT_HASH_JOIN_CACHE_RETRIES = 5;
+    
+    // Links from parent to child views are stored in a separate table for scalability
+    public static final String CREATE_CHILD_LINK_METADATA =
+            "CREATE TABLE " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CHILD_LINK_TABLE + "\"(\n" +
+            // PK columns
+            TENANT_ID + " VARCHAR NULL," +
+            TABLE_SCHEM + " VARCHAR NULL," +
+            TABLE_NAME + " VARCHAR NOT NULL," +
+            COLUMN_NAME + " VARCHAR NULL," + 
+            COLUMN_FAMILY + " VARCHAR NULL," +
+            LINK_TYPE + " UNSIGNED_TINYINT,\n" +
+            "CONSTRAINT " + SYSTEM_TABLE_PK_NAME + " PRIMARY KEY (" + TENANT_ID + ","
+            + TABLE_SCHEM + "," + TABLE_NAME + "," + COLUMN_NAME + "," + COLUMN_FAMILY + "))\n" +
+            HConstants.VERSIONS + "=" + MetaDataProtocol.DEFAULT_MAX_META_DATA_VERSIONS + ",\n" +
+            HColumnDescriptor.KEEP_DELETED_CELLS + "="  + MetaDataProtocol.DEFAULT_META_DATA_KEEP_DELETED_CELLS + ",\n" +
+            PhoenixDatabaseMetaData.TRANSACTIONAL + "=" + Boolean.FALSE;
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/758efd9b/phoenix-core/src/main/java/org/apache/phoenix/replication/SystemCatalogWALEntryFilter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/replication/SystemCatalogWALEntryFilter.java b/phoenix-core/src/main/java/org/apache/phoenix/replication/SystemCatalogWALEntryFilter.java
index 1dd79be..bc19fb6 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/replication/SystemCatalogWALEntryFilter.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/replication/SystemCatalogWALEntryFilter.java
@@ -35,7 +35,7 @@ import java.util.List;
  * during cluster upgrades. However, tenant-owned data such as tenant-owned views need to
  * be copied. This WALEntryFilter will only allow tenant-owned rows in SYSTEM.CATALOG to
  * be replicated. Data from all other tables is automatically passed. It will also copy
- * child links in SYSTEM.CATALOG that are globally-owned but point to tenant-owned views.
+ * child links in SYSTEM.CHILD_LINK that are globally-owned but point to tenant-owned views.
  *
  */
 public class SystemCatalogWALEntryFilter implements WALEntryFilter {
@@ -43,9 +43,10 @@ public class SystemCatalogWALEntryFilter implements WALEntryFilter {
   @Override
   public WAL.Entry filter(WAL.Entry entry) {
 
-    //if the WAL.Entry's table isn't System.Catalog, it auto-passes this filter
+    //if the WAL.Entry's table isn't System.Catalog or System.Child_Link, it auto-passes this filter
     //TODO: when Phoenix drops support for pre-1.3 versions of HBase, redo as a WALCellFilter
-    if (!SchemaUtil.isMetaTable(entry.getKey().getTablename().getName())){
+    byte[] tableName = entry.getKey().getTablename().getName();
+	if (!SchemaUtil.isMetaTable(tableName) && !SchemaUtil.isChildLinkTable(tableName)){
       return entry;
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/758efd9b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTable.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTable.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTable.java
index db43c75..4cd6dd1 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTable.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTable.java
@@ -150,12 +150,13 @@ public interface PTable extends PMetaDataEntity {
         PARENT_TABLE((byte)3),
         /**
          * Link from a parent table to its child view
+         * (these are stored in SYSTEM.CHILD_LINK for scalability)
          */
         CHILD_TABLE((byte)4),
         /**
-         * Link for an dropped column
+         * Link for an excluded (dropped) column
          */
-        DROPPED_COLUMN((byte)5),
+        EXCLUDED_COLUMN((byte)5),
         /**
          * Link from an index on a view to its parent table
          */

http://git-wip-us.apache.org/repos/asf/phoenix/blob/758efd9b/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 ea05f7f..f860d68 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
@@ -24,6 +24,7 @@ import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.Iterator;
 import java.util.List;
 import java.util.NavigableMap;
 
@@ -734,13 +735,6 @@ public class MetaDataUtil {
         return null;
     }
 
-    public static boolean isLinkingRow(Mutation tableMutation) {
-        byte[][] array = new byte[5][];
-        getVarChars(tableMutation.getRow(), array);
-        return array[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX] != null && Bytes
-            .equals(array[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX], HConstants.EMPTY_BYTE_ARRAY);
-    }
-
     public static boolean isLocalIndex(String physicalName) {
         if (physicalName.contains(LOCAL_INDEX_TABLE_PREFIX)) { return true; }
         return false;
@@ -800,4 +794,25 @@ public class MetaDataUtil {
         byte[] physicalTableName = Bytes.toBytes(SchemaUtil.getTableNameFromFullName(view.getPhysicalName().getString()));
         return SchemaUtil.getTableKey(ByteUtil.EMPTY_BYTE_ARRAY, physicalTableSchemaName, physicalTableName);
     }
+    
+	public static List<Mutation> removeChildLinks(List<Mutation> catalogMutations) {
+		List<Mutation> childLinks = Lists.newArrayList();
+		Iterator<Mutation> iter = catalogMutations.iterator();
+		while (iter.hasNext()) {
+			Mutation m = iter.next();
+			for (KeyValue kv : m.getFamilyMap().get(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES)) {
+				// remove mutations of link type LinkType.CHILD_TABLE
+				if ((Bytes.compareTo(kv.getQualifierArray(), kv.getQualifierOffset(), kv.getQualifierLength(),
+						PhoenixDatabaseMetaData.LINK_TYPE_BYTES, 0,
+						PhoenixDatabaseMetaData.LINK_TYPE_BYTES.length) == 0)
+						&& ((Bytes.compareTo(kv.getValueArray(), kv.getValueOffset(), kv.getValueLength(),
+								LinkType.CHILD_TABLE.getSerializedValueAsByteArray(), 0,
+								LinkType.CHILD_TABLE.getSerializedValueAsByteArray().length) == 0))) {
+					childLinks.add(m);
+					iter.remove();
+				}
+			}
+		}
+		return childLinks;
+	}
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/758efd9b/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 3e1a2f2..0060993 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
@@ -21,6 +21,7 @@ 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.SYSTEM_CATALOG_NAME_BYTES;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_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.query.QueryConstants.SEPARATOR_BYTE;
@@ -504,6 +505,11 @@ public class SchemaUtil {
                 || Bytes.compareTo(tableName, SchemaUtil
                         .getPhysicalTableName(PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_NAME_BYTES, true).getName()) == 0;
     }
+    
+    public static boolean isChildLinkTable(byte[] tableName) {
+        return Bytes.compareTo(tableName, SYSTEM_CHILD_LINK_NAME_BYTES) == 0 || Bytes.compareTo(tableName,
+                SchemaUtil.getPhysicalTableName(SYSTEM_CHILD_LINK_NAME_BYTES, true).getName()) == 0;
+    }
 
     public static boolean isSequenceTable(PTable table) {
         return PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_NAME.equals(table.getName().getString());

http://git-wip-us.apache.org/repos/asf/phoenix/blob/758efd9b/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java
index f8d98b9..cd574dc 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java
@@ -1146,6 +1146,35 @@ public class UpgradeUtil {
         }
     }
     
+    /**
+     * Move child links form SYSTEM.CATALOG to SYSTEM.CHILD_LINK
+     * @param oldMetaConnection caller should take care of closing the passed connection appropriately
+     * @throws SQLException
+     */
+    public static void moveChildLinks(PhoenixConnection oldMetaConnection) throws SQLException {
+        PhoenixConnection metaConnection = null;
+        try {
+            // Need to use own connection with max time stamp to be able to read all data from SYSTEM.CATALOG 
+            metaConnection = new PhoenixConnection(oldMetaConnection, HConstants.LATEST_TIMESTAMP);
+            logger.info("Upgrading metadata to add parent to child links for views");
+            metaConnection.commit();
+            String createChildLink = "UPSERT INTO SYSTEM.CHILD_LINK(TENANT_ID, TABLE_SCHEM, TABLE_NAME, COLUMN_NAME, COLUMN_FAMILY, LINK_TYPE)" +
+                                        "SELECT TENANT_ID, TABLE_SCHEM, TABLE_NAME, COLUMN_NAME, COLUMN_FAMILY, LINK_TYPE" + 
+                                        "FROM SYSTEM.CATALOG " + 
+                                        "WHERE LINK_TYPE = 4";
+            metaConnection.createStatement().execute(createChildLink);
+            metaConnection.commit();
+            String deleteChildLink = "DELETE FROM SYSTEM.CATALOG WHERE LINK_TYPE = 4 ";
+            metaConnection.createStatement().execute(deleteChildLink);
+            metaConnection.commit();
+            metaConnection.getQueryServices().clearCache();
+        } finally {
+            if (metaConnection != null) {
+                metaConnection.close();
+            }
+        }
+    }
+    
     public static void addViewIndexToParentLinks(PhoenixConnection oldMetaConnection) throws SQLException {
     	// Need to use own connection with max time stamp to be able to read all data from SYSTEM.CATALOG 
         try (PhoenixConnection queryConn = new PhoenixConnection(oldMetaConnection, HConstants.LATEST_TIMESTAMP);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/758efd9b/phoenix-core/src/test/java/org/apache/phoenix/coprocessor/MetaDataEndpointImplTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/coprocessor/MetaDataEndpointImplTest.java b/phoenix-core/src/test/java/org/apache/phoenix/coprocessor/MetaDataEndpointImplTest.java
index 2cecea2..fbde4b7 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/coprocessor/MetaDataEndpointImplTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/coprocessor/MetaDataEndpointImplTest.java
@@ -16,6 +16,7 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.phoenix.end2end.ParallelStatsDisabledIT;
 import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.TableNotFoundException;
@@ -43,7 +44,8 @@ import com.google.common.collect.Maps;
  * limitations under the License.
  */
 public class MetaDataEndpointImplTest extends ParallelStatsDisabledIT {
-    private final TableName catalogTable = TableName.valueOf("SYSTEM.CATALOG");
+    private final TableName catalogTable = TableName.valueOf(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES);
+    private final TableName linkTable = TableName.valueOf(PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_NAME_BYTES);
 
     /*
       The tree structure is as follows: Where ParentTable is the Base Table
@@ -77,7 +79,7 @@ public class MetaDataEndpointImplTest extends ParallelStatsDisabledIT {
         System.err.println(rightChildTable);
 
         TableViewFinderResult childViews = new TableViewFinderResult();
-        ViewFinder.findAllRelatives(getTable(catalogTable), HConstants.EMPTY_BYTE_ARRAY, table.getSchemaName().getBytes(),
+        ViewFinder.findAllRelatives(getTable(linkTable), HConstants.EMPTY_BYTE_ARRAY, table.getSchemaName().getBytes(),
             table.getTableName().getBytes(), PTable.LinkType.CHILD_TABLE, childViews);
         assertEquals(3, childViews.getResults().size());
 


[32/50] [abbrv] phoenix git commit: PHOENIX-4456 queryserver script doesn't perform as expected.

Posted by td...@apache.org.
PHOENIX-4456 queryserver script doesn't perform as expected.


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

Branch: refs/heads/system-catalog
Commit: 90c7241611667e3cd3689ce6a72762c6315231ef
Parents: 1a19d1e
Author: Sergey Soldatov <ss...@apache.org>
Authored: Tue Dec 12 23:20:48 2017 -0800
Committer: Sergey Soldatov <ss...@apache.org>
Committed: Wed Dec 13 10:16:54 2017 -0800

----------------------------------------------------------------------
 bin/daemon.py | 16 +++++++++++++---
 1 file changed, 13 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/90c72416/bin/daemon.py
----------------------------------------------------------------------
diff --git a/bin/daemon.py b/bin/daemon.py
index aeebae3..bb64148 100644
--- a/bin/daemon.py
+++ b/bin/daemon.py
@@ -57,6 +57,7 @@ import signal
 import socket
 import atexit
 import fcntl
+import time
 try:
     # Python 2 has both ‘str’ (bytes) and ‘unicode’ (text).
     basestring = basestring
@@ -386,7 +387,7 @@ class DaemonContext:
         change_process_owner(self.uid, self.gid)
 
         if self.detach_process:
-            detach_process_context()
+            detach_process_context(self.pidfile)
 
         signal_handler_map = self._make_signal_handler_map()
         set_signal_handlers(signal_handler_map)
@@ -657,7 +658,7 @@ def prevent_core_dump():
     resource.setrlimit(core_resource, core_limit)
 
 
-def detach_process_context():
+def detach_process_context(pidfile):
     """ Detach the process context from parent and session.
 
         :return: ``None``.
@@ -683,6 +684,8 @@ def detach_process_context():
         try:
             pid = os.fork()
             if pid > 0:
+                while not os.path.exists(pidfile.path):
+                    time.sleep(0.1)
                 os._exit(0)
         except OSError as exc:
             error = DaemonProcessDetachError(
@@ -959,7 +962,14 @@ found at [1].
     def __init__(self, path, enter_err_msg=None):
         self.path = path
         self.enter_err_msg = enter_err_msg
-        self.pidfile = None
+        self.pidfile = open(self.path, 'a+')
+        try:
+            fcntl.flock(self.pidfile.fileno(), fcntl.LOCK_EX | fcntl.LOCK_NB)
+            fcntl.flock(self.pidfile.fileno(), fcntl.LOCK_UN)
+            self.pidfile.close()
+            os.remove(self.path)
+        except IOError:
+            sys.exit(self.enter_err_msg)
 
     def __enter__(self):
         self.pidfile = open(self.path, 'a+')


[34/50] [abbrv] phoenix git commit: PHOENIX-4449 Bundle a copy of Argparse-1.4.0 for installations that need it

Posted by td...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cb02da7/bin/argparse-1.4.0/argparse.py
----------------------------------------------------------------------
diff --git a/bin/argparse-1.4.0/argparse.py b/bin/argparse-1.4.0/argparse.py
new file mode 100644
index 0000000..70a77cc
--- /dev/null
+++ b/bin/argparse-1.4.0/argparse.py
@@ -0,0 +1,2392 @@
+# Author: Steven J. Bethard <st...@gmail.com>.
+# Maintainer: Thomas Waldmann <tw...@waldmann-edv.de>
+
+"""Command-line parsing library
+
+This module is an optparse-inspired command-line parsing library that:
+
+    - handles both optional and positional arguments
+    - produces highly informative usage messages
+    - supports parsers that dispatch to sub-parsers
+
+The following is a simple usage example that sums integers from the
+command-line and writes the result to a file::
+
+    parser = argparse.ArgumentParser(
+        description='sum the integers at the command line')
+    parser.add_argument(
+        'integers', metavar='int', nargs='+', type=int,
+        help='an integer to be summed')
+    parser.add_argument(
+        '--log', default=sys.stdout, type=argparse.FileType('w'),
+        help='the file where the sum should be written')
+    args = parser.parse_args()
+    args.log.write('%s' % sum(args.integers))
+    args.log.close()
+
+The module contains the following public classes:
+
+    - ArgumentParser -- The main entry point for command-line parsing. As the
+        example above shows, the add_argument() method is used to populate
+        the parser with actions for optional and positional arguments. Then
+        the parse_args() method is invoked to convert the args at the
+        command-line into an object with attributes.
+
+    - ArgumentError -- The exception raised by ArgumentParser objects when
+        there are errors with the parser's actions. Errors raised while
+        parsing the command-line are caught by ArgumentParser and emitted
+        as command-line messages.
+
+    - FileType -- A factory for defining types of files to be created. As the
+        example above shows, instances of FileType are typically passed as
+        the type= argument of add_argument() calls.
+
+    - Action -- The base class for parser actions. Typically actions are
+        selected by passing strings like 'store_true' or 'append_const' to
+        the action= argument of add_argument(). However, for greater
+        customization of ArgumentParser actions, subclasses of Action may
+        be defined and passed as the action= argument.
+
+    - HelpFormatter, RawDescriptionHelpFormatter, RawTextHelpFormatter,
+        ArgumentDefaultsHelpFormatter -- Formatter classes which
+        may be passed as the formatter_class= argument to the
+        ArgumentParser constructor. HelpFormatter is the default,
+        RawDescriptionHelpFormatter and RawTextHelpFormatter tell the parser
+        not to change the formatting for help text, and
+        ArgumentDefaultsHelpFormatter adds information about argument defaults
+        to the help.
+
+All other classes in this module are considered implementation details.
+(Also note that HelpFormatter and RawDescriptionHelpFormatter are only
+considered public as object names -- the API of the formatter objects is
+still considered an implementation detail.)
+"""
+
+__version__ = '1.4.0'  # we use our own version number independant of the
+                       # one in stdlib and we release this on pypi.
+
+__external_lib__ = True  # to make sure the tests really test THIS lib,
+                         # not the builtin one in Python stdlib
+
+__all__ = [
+    'ArgumentParser',
+    'ArgumentError',
+    'ArgumentTypeError',
+    'FileType',
+    'HelpFormatter',
+    'ArgumentDefaultsHelpFormatter',
+    'RawDescriptionHelpFormatter',
+    'RawTextHelpFormatter',
+    'Namespace',
+    'Action',
+    'ONE_OR_MORE',
+    'OPTIONAL',
+    'PARSER',
+    'REMAINDER',
+    'SUPPRESS',
+    'ZERO_OR_MORE',
+]
+
+
+import copy as _copy
+import os as _os
+import re as _re
+import sys as _sys
+import textwrap as _textwrap
+
+from gettext import gettext as _
+
+try:
+    set
+except NameError:
+    # for python < 2.4 compatibility (sets module is there since 2.3):
+    from sets import Set as set
+
+try:
+    basestring
+except NameError:
+    basestring = str
+
+try:
+    sorted
+except NameError:
+    # for python < 2.4 compatibility:
+    def sorted(iterable, reverse=False):
+        result = list(iterable)
+        result.sort()
+        if reverse:
+            result.reverse()
+        return result
+
+
+def _callable(obj):
+    return hasattr(obj, '__call__') or hasattr(obj, '__bases__')
+
+
+SUPPRESS = '==SUPPRESS=='
+
+OPTIONAL = '?'
+ZERO_OR_MORE = '*'
+ONE_OR_MORE = '+'
+PARSER = 'A...'
+REMAINDER = '...'
+_UNRECOGNIZED_ARGS_ATTR = '_unrecognized_args'
+
+# =============================
+# Utility functions and classes
+# =============================
+
+class _AttributeHolder(object):
+    """Abstract base class that provides __repr__.
+
+    The __repr__ method returns a string in the format::
+        ClassName(attr=name, attr=name, ...)
+    The attributes are determined either by a class-level attribute,
+    '_kwarg_names', or by inspecting the instance __dict__.
+    """
+
+    def __repr__(self):
+        type_name = type(self).__name__
+        arg_strings = []
+        for arg in self._get_args():
+            arg_strings.append(repr(arg))
+        for name, value in self._get_kwargs():
+            arg_strings.append('%s=%r' % (name, value))
+        return '%s(%s)' % (type_name, ', '.join(arg_strings))
+
+    def _get_kwargs(self):
+        return sorted(self.__dict__.items())
+
+    def _get_args(self):
+        return []
+
+
+def _ensure_value(namespace, name, value):
+    if getattr(namespace, name, None) is None:
+        setattr(namespace, name, value)
+    return getattr(namespace, name)
+
+
+# ===============
+# Formatting Help
+# ===============
+
+class HelpFormatter(object):
+    """Formatter for generating usage messages and argument help strings.
+
+    Only the name of this class is considered a public API. All the methods
+    provided by the class are considered an implementation detail.
+    """
+
+    def __init__(self,
+                 prog,
+                 indent_increment=2,
+                 max_help_position=24,
+                 width=None):
+
+        # default setting for width
+        if width is None:
+            try:
+                width = int(_os.environ['COLUMNS'])
+            except (KeyError, ValueError):
+                width = 80
+            width -= 2
+
+        self._prog = prog
+        self._indent_increment = indent_increment
+        self._max_help_position = max_help_position
+        self._width = width
+
+        self._current_indent = 0
+        self._level = 0
+        self._action_max_length = 0
+
+        self._root_section = self._Section(self, None)
+        self._current_section = self._root_section
+
+        self._whitespace_matcher = _re.compile(r'\s+')
+        self._long_break_matcher = _re.compile(r'\n\n\n+')
+
+    # ===============================
+    # Section and indentation methods
+    # ===============================
+    def _indent(self):
+        self._current_indent += self._indent_increment
+        self._level += 1
+
+    def _dedent(self):
+        self._current_indent -= self._indent_increment
+        assert self._current_indent >= 0, 'Indent decreased below 0.'
+        self._level -= 1
+
+    class _Section(object):
+
+        def __init__(self, formatter, parent, heading=None):
+            self.formatter = formatter
+            self.parent = parent
+            self.heading = heading
+            self.items = []
+
+        def format_help(self):
+            # format the indented section
+            if self.parent is not None:
+                self.formatter._indent()
+            join = self.formatter._join_parts
+            for func, args in self.items:
+                func(*args)
+            item_help = join([func(*args) for func, args in self.items])
+            if self.parent is not None:
+                self.formatter._dedent()
+
+            # return nothing if the section was empty
+            if not item_help:
+                return ''
+
+            # add the heading if the section was non-empty
+            if self.heading is not SUPPRESS and self.heading is not None:
+                current_indent = self.formatter._current_indent
+                heading = '%*s%s:\n' % (current_indent, '', self.heading)
+            else:
+                heading = ''
+
+            # join the section-initial newline, the heading and the help
+            return join(['\n', heading, item_help, '\n'])
+
+    def _add_item(self, func, args):
+        self._current_section.items.append((func, args))
+
+    # ========================
+    # Message building methods
+    # ========================
+    def start_section(self, heading):
+        self._indent()
+        section = self._Section(self, self._current_section, heading)
+        self._add_item(section.format_help, [])
+        self._current_section = section
+
+    def end_section(self):
+        self._current_section = self._current_section.parent
+        self._dedent()
+
+    def add_text(self, text):
+        if text is not SUPPRESS and text is not None:
+            self._add_item(self._format_text, [text])
+
+    def add_usage(self, usage, actions, groups, prefix=None):
+        if usage is not SUPPRESS:
+            args = usage, actions, groups, prefix
+            self._add_item(self._format_usage, args)
+
+    def add_argument(self, action):
+        if action.help is not SUPPRESS:
+
+            # find all invocations
+            get_invocation = self._format_action_invocation
+            invocations = [get_invocation(action)]
+            for subaction in self._iter_indented_subactions(action):
+                invocations.append(get_invocation(subaction))
+
+            # update the maximum item length
+            invocation_length = max([len(s) for s in invocations])
+            action_length = invocation_length + self._current_indent
+            self._action_max_length = max(self._action_max_length,
+                                          action_length)
+
+            # add the item to the list
+            self._add_item(self._format_action, [action])
+
+    def add_arguments(self, actions):
+        for action in actions:
+            self.add_argument(action)
+
+    # =======================
+    # Help-formatting methods
+    # =======================
+    def format_help(self):
+        help = self._root_section.format_help()
+        if help:
+            help = self._long_break_matcher.sub('\n\n', help)
+            help = help.strip('\n') + '\n'
+        return help
+
+    def _join_parts(self, part_strings):
+        return ''.join([part
+                        for part in part_strings
+                        if part and part is not SUPPRESS])
+
+    def _format_usage(self, usage, actions, groups, prefix):
+        if prefix is None:
+            prefix = _('usage: ')
+
+        # if usage is specified, use that
+        if usage is not None:
+            usage = usage % dict(prog=self._prog)
+
+        # if no optionals or positionals are available, usage is just prog
+        elif usage is None and not actions:
+            usage = '%(prog)s' % dict(prog=self._prog)
+
+        # if optionals and positionals are available, calculate usage
+        elif usage is None:
+            prog = '%(prog)s' % dict(prog=self._prog)
+
+            # split optionals from positionals
+            optionals = []
+            positionals = []
+            for action in actions:
+                if action.option_strings:
+                    optionals.append(action)
+                else:
+                    positionals.append(action)
+
+            # build full usage string
+            format = self._format_actions_usage
+            action_usage = format(optionals + positionals, groups)
+            usage = ' '.join([s for s in [prog, action_usage] if s])
+
+            # wrap the usage parts if it's too long
+            text_width = self._width - self._current_indent
+            if len(prefix) + len(usage) > text_width:
+
+                # break usage into wrappable parts
+                part_regexp = r'\(.*?\)+|\[.*?\]+|\S+'
+                opt_usage = format(optionals, groups)
+                pos_usage = format(positionals, groups)
+                opt_parts = _re.findall(part_regexp, opt_usage)
+                pos_parts = _re.findall(part_regexp, pos_usage)
+                assert ' '.join(opt_parts) == opt_usage
+                assert ' '.join(pos_parts) == pos_usage
+
+                # helper for wrapping lines
+                def get_lines(parts, indent, prefix=None):
+                    lines = []
+                    line = []
+                    if prefix is not None:
+                        line_len = len(prefix) - 1
+                    else:
+                        line_len = len(indent) - 1
+                    for part in parts:
+                        if line_len + 1 + len(part) > text_width:
+                            lines.append(indent + ' '.join(line))
+                            line = []
+                            line_len = len(indent) - 1
+                        line.append(part)
+                        line_len += len(part) + 1
+                    if line:
+                        lines.append(indent + ' '.join(line))
+                    if prefix is not None:
+                        lines[0] = lines[0][len(indent):]
+                    return lines
+
+                # if prog is short, follow it with optionals or positionals
+                if len(prefix) + len(prog) <= 0.75 * text_width:
+                    indent = ' ' * (len(prefix) + len(prog) + 1)
+                    if opt_parts:
+                        lines = get_lines([prog] + opt_parts, indent, prefix)
+                        lines.extend(get_lines(pos_parts, indent))
+                    elif pos_parts:
+                        lines = get_lines([prog] + pos_parts, indent, prefix)
+                    else:
+                        lines = [prog]
+
+                # if prog is long, put it on its own line
+                else:
+                    indent = ' ' * len(prefix)
+                    parts = opt_parts + pos_parts
+                    lines = get_lines(parts, indent)
+                    if len(lines) > 1:
+                        lines = []
+                        lines.extend(get_lines(opt_parts, indent))
+                        lines.extend(get_lines(pos_parts, indent))
+                    lines = [prog] + lines
+
+                # join lines into usage
+                usage = '\n'.join(lines)
+
+        # prefix with 'usage:'
+        return '%s%s\n\n' % (prefix, usage)
+
+    def _format_actions_usage(self, actions, groups):
+        # find group indices and identify actions in groups
+        group_actions = set()
+        inserts = {}
+        for group in groups:
+            try:
+                start = actions.index(group._group_actions[0])
+            except ValueError:
+                continue
+            else:
+                end = start + len(group._group_actions)
+                if actions[start:end] == group._group_actions:
+                    for action in group._group_actions:
+                        group_actions.add(action)
+                    if not group.required:
+                        if start in inserts:
+                            inserts[start] += ' ['
+                        else:
+                            inserts[start] = '['
+                        inserts[end] = ']'
+                    else:
+                        if start in inserts:
+                            inserts[start] += ' ('
+                        else:
+                            inserts[start] = '('
+                        inserts[end] = ')'
+                    for i in range(start + 1, end):
+                        inserts[i] = '|'
+
+        # collect all actions format strings
+        parts = []
+        for i, action in enumerate(actions):
+
+            # suppressed arguments are marked with None
+            # remove | separators for suppressed arguments
+            if action.help is SUPPRESS:
+                parts.append(None)
+                if inserts.get(i) == '|':
+                    inserts.pop(i)
+                elif inserts.get(i + 1) == '|':
+                    inserts.pop(i + 1)
+
+            # produce all arg strings
+            elif not action.option_strings:
+                part = self._format_args(action, action.dest)
+
+                # if it's in a group, strip the outer []
+                if action in group_actions:
+                    if part[0] == '[' and part[-1] == ']':
+                        part = part[1:-1]
+
+                # add the action string to the list
+                parts.append(part)
+
+            # produce the first way to invoke the option in brackets
+            else:
+                option_string = action.option_strings[0]
+
+                # if the Optional doesn't take a value, format is:
+                #    -s or --long
+                if action.nargs == 0:
+                    part = '%s' % option_string
+
+                # if the Optional takes a value, format is:
+                #    -s ARGS or --long ARGS
+                else:
+                    default = action.dest.upper()
+                    args_string = self._format_args(action, default)
+                    part = '%s %s' % (option_string, args_string)
+
+                # make it look optional if it's not required or in a group
+                if not action.required and action not in group_actions:
+                    part = '[%s]' % part
+
+                # add the action string to the list
+                parts.append(part)
+
+        # insert things at the necessary indices
+        for i in sorted(inserts, reverse=True):
+            parts[i:i] = [inserts[i]]
+
+        # join all the action items with spaces
+        text = ' '.join([item for item in parts if item is not None])
+
+        # clean up separators for mutually exclusive groups
+        open = r'[\[(]'
+        close = r'[\])]'
+        text = _re.sub(r'(%s) ' % open, r'\1', text)
+        text = _re.sub(r' (%s)' % close, r'\1', text)
+        text = _re.sub(r'%s *%s' % (open, close), r'', text)
+        text = _re.sub(r'\(([^|]*)\)', r'\1', text)
+        text = text.strip()
+
+        # return the text
+        return text
+
+    def _format_text(self, text):
+        if '%(prog)' in text:
+            text = text % dict(prog=self._prog)
+        text_width = self._width - self._current_indent
+        indent = ' ' * self._current_indent
+        return self._fill_text(text, text_width, indent) + '\n\n'
+
+    def _format_action(self, action):
+        # determine the required width and the entry label
+        help_position = min(self._action_max_length + 2,
+                            self._max_help_position)
+        help_width = self._width - help_position
+        action_width = help_position - self._current_indent - 2
+        action_header = self._format_action_invocation(action)
+
+        # ho nelp; start on same line and add a final newline
+        if not action.help:
+            tup = self._current_indent, '', action_header
+            action_header = '%*s%s\n' % tup
+
+        # short action name; start on the same line and pad two spaces
+        elif len(action_header) <= action_width:
+            tup = self._current_indent, '', action_width, action_header
+            action_header = '%*s%-*s  ' % tup
+            indent_first = 0
+
+        # long action name; start on the next line
+        else:
+            tup = self._current_indent, '', action_header
+            action_header = '%*s%s\n' % tup
+            indent_first = help_position
+
+        # collect the pieces of the action help
+        parts = [action_header]
+
+        # if there was help for the action, add lines of help text
+        if action.help:
+            help_text = self._expand_help(action)
+            help_lines = self._split_lines(help_text, help_width)
+            parts.append('%*s%s\n' % (indent_first, '', help_lines[0]))
+            for line in help_lines[1:]:
+                parts.append('%*s%s\n' % (help_position, '', line))
+
+        # or add a newline if the description doesn't end with one
+        elif not action_header.endswith('\n'):
+            parts.append('\n')
+
+        # if there are any sub-actions, add their help as well
+        for subaction in self._iter_indented_subactions(action):
+            parts.append(self._format_action(subaction))
+
+        # return a single string
+        return self._join_parts(parts)
+
+    def _format_action_invocation(self, action):
+        if not action.option_strings:
+            metavar, = self._metavar_formatter(action, action.dest)(1)
+            return metavar
+
+        else:
+            parts = []
+
+            # if the Optional doesn't take a value, format is:
+            #    -s, --long
+            if action.nargs == 0:
+                parts.extend(action.option_strings)
+
+            # if the Optional takes a value, format is:
+            #    -s ARGS, --long ARGS
+            else:
+                default = action.dest.upper()
+                args_string = self._format_args(action, default)
+                for option_string in action.option_strings:
+                    parts.append('%s %s' % (option_string, args_string))
+
+            return ', '.join(parts)
+
+    def _metavar_formatter(self, action, default_metavar):
+        if action.metavar is not None:
+            result = action.metavar
+        elif action.choices is not None:
+            choice_strs = [str(choice) for choice in action.choices]
+            result = '{%s}' % ','.join(choice_strs)
+        else:
+            result = default_metavar
+
+        def format(tuple_size):
+            if isinstance(result, tuple):
+                return result
+            else:
+                return (result, ) * tuple_size
+        return format
+
+    def _format_args(self, action, default_metavar):
+        get_metavar = self._metavar_formatter(action, default_metavar)
+        if action.nargs is None:
+            result = '%s' % get_metavar(1)
+        elif action.nargs == OPTIONAL:
+            result = '[%s]' % get_metavar(1)
+        elif action.nargs == ZERO_OR_MORE:
+            result = '[%s [%s ...]]' % get_metavar(2)
+        elif action.nargs == ONE_OR_MORE:
+            result = '%s [%s ...]' % get_metavar(2)
+        elif action.nargs == REMAINDER:
+            result = '...'
+        elif action.nargs == PARSER:
+            result = '%s ...' % get_metavar(1)
+        else:
+            formats = ['%s' for _ in range(action.nargs)]
+            result = ' '.join(formats) % get_metavar(action.nargs)
+        return result
+
+    def _expand_help(self, action):
+        params = dict(vars(action), prog=self._prog)
+        for name in list(params):
+            if params[name] is SUPPRESS:
+                del params[name]
+        for name in list(params):
+            if hasattr(params[name], '__name__'):
+                params[name] = params[name].__name__
+        if params.get('choices') is not None:
+            choices_str = ', '.join([str(c) for c in params['choices']])
+            params['choices'] = choices_str
+        return self._get_help_string(action) % params
+
+    def _iter_indented_subactions(self, action):
+        try:
+            get_subactions = action._get_subactions
+        except AttributeError:
+            pass
+        else:
+            self._indent()
+            for subaction in get_subactions():
+                yield subaction
+            self._dedent()
+
+    def _split_lines(self, text, width):
+        text = self._whitespace_matcher.sub(' ', text).strip()
+        return _textwrap.wrap(text, width)
+
+    def _fill_text(self, text, width, indent):
+        text = self._whitespace_matcher.sub(' ', text).strip()
+        return _textwrap.fill(text, width, initial_indent=indent,
+                                           subsequent_indent=indent)
+
+    def _get_help_string(self, action):
+        return action.help
+
+
+class RawDescriptionHelpFormatter(HelpFormatter):
+    """Help message formatter which retains any formatting in descriptions.
+
+    Only the name of this class is considered a public API. All the methods
+    provided by the class are considered an implementation detail.
+    """
+
+    def _fill_text(self, text, width, indent):
+        return ''.join([indent + line for line in text.splitlines(True)])
+
+
+class RawTextHelpFormatter(RawDescriptionHelpFormatter):
+    """Help message formatter which retains formatting of all help text.
+
+    Only the name of this class is considered a public API. All the methods
+    provided by the class are considered an implementation detail.
+    """
+
+    def _split_lines(self, text, width):
+        return text.splitlines()
+
+
+class ArgumentDefaultsHelpFormatter(HelpFormatter):
+    """Help message formatter which adds default values to argument help.
+
+    Only the name of this class is considered a public API. All the methods
+    provided by the class are considered an implementation detail.
+    """
+
+    def _get_help_string(self, action):
+        help = action.help
+        if '%(default)' not in action.help:
+            if action.default is not SUPPRESS:
+                defaulting_nargs = [OPTIONAL, ZERO_OR_MORE]
+                if action.option_strings or action.nargs in defaulting_nargs:
+                    help += ' (default: %(default)s)'
+        return help
+
+
+# =====================
+# Options and Arguments
+# =====================
+
+def _get_action_name(argument):
+    if argument is None:
+        return None
+    elif argument.option_strings:
+        return  '/'.join(argument.option_strings)
+    elif argument.metavar not in (None, SUPPRESS):
+        return argument.metavar
+    elif argument.dest not in (None, SUPPRESS):
+        return argument.dest
+    else:
+        return None
+
+
+class ArgumentError(Exception):
+    """An error from creating or using an argument (optional or positional).
+
+    The string value of this exception is the message, augmented with
+    information about the argument that caused it.
+    """
+
+    def __init__(self, argument, message):
+        self.argument_name = _get_action_name(argument)
+        self.message = message
+
+    def __str__(self):
+        if self.argument_name is None:
+            format = '%(message)s'
+        else:
+            format = 'argument %(argument_name)s: %(message)s'
+        return format % dict(message=self.message,
+                             argument_name=self.argument_name)
+
+
+class ArgumentTypeError(Exception):
+    """An error from trying to convert a command line string to a type."""
+    pass
+
+
+# ==============
+# Action classes
+# ==============
+
+class Action(_AttributeHolder):
+    """Information about how to convert command line strings to Python objects.
+
+    Action objects are used by an ArgumentParser to represent the information
+    needed to parse a single argument from one or more strings from the
+    command line. The keyword arguments to the Action constructor are also
+    all attributes of Action instances.
+
+    Keyword Arguments:
+
+        - option_strings -- A list of command-line option strings which
+            should be associated with this action.
+
+        - dest -- The name of the attribute to hold the created object(s)
+
+        - nargs -- The number of command-line arguments that should be
+            consumed. By default, one argument will be consumed and a single
+            value will be produced.  Other values include:
+                - N (an integer) consumes N arguments (and produces a list)
+                - '?' consumes zero or one arguments
+                - '*' consumes zero or more arguments (and produces a list)
+                - '+' consumes one or more arguments (and produces a list)
+            Note that the difference between the default and nargs=1 is that
+            with the default, a single value will be produced, while with
+            nargs=1, a list containing a single value will be produced.
+
+        - const -- The value to be produced if the option is specified and the
+            option uses an action that takes no values.
+
+        - default -- The value to be produced if the option is not specified.
+
+        - type -- The type which the command-line arguments should be converted
+            to, should be one of 'string', 'int', 'float', 'complex' or a
+            callable object that accepts a single string argument. If None,
+            'string' is assumed.
+
+        - choices -- A container of values that should be allowed. If not None,
+            after a command-line argument has been converted to the appropriate
+            type, an exception will be raised if it is not a member of this
+            collection.
+
+        - required -- True if the action must always be specified at the
+            command line. This is only meaningful for optional command-line
+            arguments.
+
+        - help -- The help string describing the argument.
+
+        - metavar -- The name to be used for the option's argument with the
+            help string. If None, the 'dest' value will be used as the name.
+    """
+
+    def __init__(self,
+                 option_strings,
+                 dest,
+                 nargs=None,
+                 const=None,
+                 default=None,
+                 type=None,
+                 choices=None,
+                 required=False,
+                 help=None,
+                 metavar=None):
+        self.option_strings = option_strings
+        self.dest = dest
+        self.nargs = nargs
+        self.const = const
+        self.default = default
+        self.type = type
+        self.choices = choices
+        self.required = required
+        self.help = help
+        self.metavar = metavar
+
+    def _get_kwargs(self):
+        names = [
+            'option_strings',
+            'dest',
+            'nargs',
+            'const',
+            'default',
+            'type',
+            'choices',
+            'help',
+            'metavar',
+        ]
+        return [(name, getattr(self, name)) for name in names]
+
+    def __call__(self, parser, namespace, values, option_string=None):
+        raise NotImplementedError(_('.__call__() not defined'))
+
+
+class _StoreAction(Action):
+
+    def __init__(self,
+                 option_strings,
+                 dest,
+                 nargs=None,
+                 const=None,
+                 default=None,
+                 type=None,
+                 choices=None,
+                 required=False,
+                 help=None,
+                 metavar=None):
+        if nargs == 0:
+            raise ValueError('nargs for store actions must be > 0; if you '
+                             'have nothing to store, actions such as store '
+                             'true or store const may be more appropriate')
+        if const is not None and nargs != OPTIONAL:
+            raise ValueError('nargs must be %r to supply const' % OPTIONAL)
+        super(_StoreAction, self).__init__(
+            option_strings=option_strings,
+            dest=dest,
+            nargs=nargs,
+            const=const,
+            default=default,
+            type=type,
+            choices=choices,
+            required=required,
+            help=help,
+            metavar=metavar)
+
+    def __call__(self, parser, namespace, values, option_string=None):
+        setattr(namespace, self.dest, values)
+
+
+class _StoreConstAction(Action):
+
+    def __init__(self,
+                 option_strings,
+                 dest,
+                 const,
+                 default=None,
+                 required=False,
+                 help=None,
+                 metavar=None):
+        super(_StoreConstAction, self).__init__(
+            option_strings=option_strings,
+            dest=dest,
+            nargs=0,
+            const=const,
+            default=default,
+            required=required,
+            help=help)
+
+    def __call__(self, parser, namespace, values, option_string=None):
+        setattr(namespace, self.dest, self.const)
+
+
+class _StoreTrueAction(_StoreConstAction):
+
+    def __init__(self,
+                 option_strings,
+                 dest,
+                 default=False,
+                 required=False,
+                 help=None):
+        super(_StoreTrueAction, self).__init__(
+            option_strings=option_strings,
+            dest=dest,
+            const=True,
+            default=default,
+            required=required,
+            help=help)
+
+
+class _StoreFalseAction(_StoreConstAction):
+
+    def __init__(self,
+                 option_strings,
+                 dest,
+                 default=True,
+                 required=False,
+                 help=None):
+        super(_StoreFalseAction, self).__init__(
+            option_strings=option_strings,
+            dest=dest,
+            const=False,
+            default=default,
+            required=required,
+            help=help)
+
+
+class _AppendAction(Action):
+
+    def __init__(self,
+                 option_strings,
+                 dest,
+                 nargs=None,
+                 const=None,
+                 default=None,
+                 type=None,
+                 choices=None,
+                 required=False,
+                 help=None,
+                 metavar=None):
+        if nargs == 0:
+            raise ValueError('nargs for append actions must be > 0; if arg '
+                             'strings are not supplying the value to append, '
+                             'the append const action may be more appropriate')
+        if const is not None and nargs != OPTIONAL:
+            raise ValueError('nargs must be %r to supply const' % OPTIONAL)
+        super(_AppendAction, self).__init__(
+            option_strings=option_strings,
+            dest=dest,
+            nargs=nargs,
+            const=const,
+            default=default,
+            type=type,
+            choices=choices,
+            required=required,
+            help=help,
+            metavar=metavar)
+
+    def __call__(self, parser, namespace, values, option_string=None):
+        items = _copy.copy(_ensure_value(namespace, self.dest, []))
+        items.append(values)
+        setattr(namespace, self.dest, items)
+
+
+class _AppendConstAction(Action):
+
+    def __init__(self,
+                 option_strings,
+                 dest,
+                 const,
+                 default=None,
+                 required=False,
+                 help=None,
+                 metavar=None):
+        super(_AppendConstAction, self).__init__(
+            option_strings=option_strings,
+            dest=dest,
+            nargs=0,
+            const=const,
+            default=default,
+            required=required,
+            help=help,
+            metavar=metavar)
+
+    def __call__(self, parser, namespace, values, option_string=None):
+        items = _copy.copy(_ensure_value(namespace, self.dest, []))
+        items.append(self.const)
+        setattr(namespace, self.dest, items)
+
+
+class _CountAction(Action):
+
+    def __init__(self,
+                 option_strings,
+                 dest,
+                 default=None,
+                 required=False,
+                 help=None):
+        super(_CountAction, self).__init__(
+            option_strings=option_strings,
+            dest=dest,
+            nargs=0,
+            default=default,
+            required=required,
+            help=help)
+
+    def __call__(self, parser, namespace, values, option_string=None):
+        new_count = _ensure_value(namespace, self.dest, 0) + 1
+        setattr(namespace, self.dest, new_count)
+
+
+class _HelpAction(Action):
+
+    def __init__(self,
+                 option_strings,
+                 dest=SUPPRESS,
+                 default=SUPPRESS,
+                 help=None):
+        super(_HelpAction, self).__init__(
+            option_strings=option_strings,
+            dest=dest,
+            default=default,
+            nargs=0,
+            help=help)
+
+    def __call__(self, parser, namespace, values, option_string=None):
+        parser.print_help()
+        parser.exit()
+
+
+class _VersionAction(Action):
+
+    def __init__(self,
+                 option_strings,
+                 version=None,
+                 dest=SUPPRESS,
+                 default=SUPPRESS,
+                 help="show program's version number and exit"):
+        super(_VersionAction, self).__init__(
+            option_strings=option_strings,
+            dest=dest,
+            default=default,
+            nargs=0,
+            help=help)
+        self.version = version
+
+    def __call__(self, parser, namespace, values, option_string=None):
+        version = self.version
+        if version is None:
+            version = parser.version
+        formatter = parser._get_formatter()
+        formatter.add_text(version)
+        parser.exit(message=formatter.format_help())
+
+
+class _SubParsersAction(Action):
+
+    class _ChoicesPseudoAction(Action):
+
+        def __init__(self, name, aliases, help):
+            metavar = dest = name
+            if aliases:
+                metavar += ' (%s)' % ', '.join(aliases)
+            sup = super(_SubParsersAction._ChoicesPseudoAction, self)
+            sup.__init__(option_strings=[], dest=dest, help=help,
+                        metavar=metavar)
+
+    def __init__(self,
+                 option_strings,
+                 prog,
+                 parser_class,
+                 dest=SUPPRESS,
+                 help=None,
+                 metavar=None):
+
+        self._prog_prefix = prog
+        self._parser_class = parser_class
+        self._name_parser_map = {}
+        self._choices_actions = []
+
+        super(_SubParsersAction, self).__init__(
+            option_strings=option_strings,
+            dest=dest,
+            nargs=PARSER,
+            choices=self._name_parser_map,
+            help=help,
+            metavar=metavar)
+
+    def add_parser(self, name, **kwargs):
+        # set prog from the existing prefix
+        if kwargs.get('prog') is None:
+            kwargs['prog'] = '%s %s' % (self._prog_prefix, name)
+
+        aliases = kwargs.pop('aliases', ())
+
+        # create a pseudo-action to hold the choice help
+        if 'help' in kwargs:
+            help = kwargs.pop('help')
+            choice_action = self._ChoicesPseudoAction(name, aliases, help)
+            self._choices_actions.append(choice_action)
+
+        # create the parser and add it to the map
+        parser = self._parser_class(**kwargs)
+        self._name_parser_map[name] = parser
+
+        # make parser available under aliases also
+        for alias in aliases:
+            self._name_parser_map[alias] = parser
+
+        return parser
+
+    def _get_subactions(self):
+        return self._choices_actions
+
+    def __call__(self, parser, namespace, values, option_string=None):
+        parser_name = values[0]
+        arg_strings = values[1:]
+
+        # set the parser name if requested
+        if self.dest is not SUPPRESS:
+            setattr(namespace, self.dest, parser_name)
+
+        # select the parser
+        try:
+            parser = self._name_parser_map[parser_name]
+        except KeyError:
+            tup = parser_name, ', '.join(self._name_parser_map)
+            msg = _('unknown parser %r (choices: %s)' % tup)
+            raise ArgumentError(self, msg)
+
+        # parse all the remaining options into the namespace
+        # store any unrecognized options on the object, so that the top
+        # level parser can decide what to do with them
+        namespace, arg_strings = parser.parse_known_args(arg_strings, namespace)
+        if arg_strings:
+            vars(namespace).setdefault(_UNRECOGNIZED_ARGS_ATTR, [])
+            getattr(namespace, _UNRECOGNIZED_ARGS_ATTR).extend(arg_strings)
+
+
+# ==============
+# Type classes
+# ==============
+
+class FileType(object):
+    """Factory for creating file object types
+
+    Instances of FileType are typically passed as type= arguments to the
+    ArgumentParser add_argument() method.
+
+    Keyword Arguments:
+        - mode -- A string indicating how the file is to be opened. Accepts the
+            same values as the builtin open() function.
+        - bufsize -- The file's desired buffer size. Accepts the same values as
+            the builtin open() function.
+    """
+
+    def __init__(self, mode='r', bufsize=None):
+        self._mode = mode
+        self._bufsize = bufsize
+
+    def __call__(self, string):
+        # the special argument "-" means sys.std{in,out}
+        if string == '-':
+            if 'r' in self._mode:
+                return _sys.stdin
+            elif 'w' in self._mode:
+                return _sys.stdout
+            else:
+                msg = _('argument "-" with mode %r' % self._mode)
+                raise ValueError(msg)
+
+        try:
+            # all other arguments are used as file names
+            if self._bufsize:
+                return open(string, self._mode, self._bufsize)
+            else:
+                return open(string, self._mode)
+        except IOError:
+            err = _sys.exc_info()[1]
+            message = _("can't open '%s': %s")
+            raise ArgumentTypeError(message % (string, err))
+
+    def __repr__(self):
+        args = [self._mode, self._bufsize]
+        args_str = ', '.join([repr(arg) for arg in args if arg is not None])
+        return '%s(%s)' % (type(self).__name__, args_str)
+
+# ===========================
+# Optional and Positional Parsing
+# ===========================
+
+class Namespace(_AttributeHolder):
+    """Simple object for storing attributes.
+
+    Implements equality by attribute names and values, and provides a simple
+    string representation.
+    """
+
+    def __init__(self, **kwargs):
+        for name in kwargs:
+            setattr(self, name, kwargs[name])
+
+    __hash__ = None
+
+    def __eq__(self, other):
+        return vars(self) == vars(other)
+
+    def __ne__(self, other):
+        return not (self == other)
+
+    def __contains__(self, key):
+        return key in self.__dict__
+
+
+class _ActionsContainer(object):
+
+    def __init__(self,
+                 description,
+                 prefix_chars,
+                 argument_default,
+                 conflict_handler):
+        super(_ActionsContainer, self).__init__()
+
+        self.description = description
+        self.argument_default = argument_default
+        self.prefix_chars = prefix_chars
+        self.conflict_handler = conflict_handler
+
+        # set up registries
+        self._registries = {}
+
+        # register actions
+        self.register('action', None, _StoreAction)
+        self.register('action', 'store', _StoreAction)
+        self.register('action', 'store_const', _StoreConstAction)
+        self.register('action', 'store_true', _StoreTrueAction)
+        self.register('action', 'store_false', _StoreFalseAction)
+        self.register('action', 'append', _AppendAction)
+        self.register('action', 'append_const', _AppendConstAction)
+        self.register('action', 'count', _CountAction)
+        self.register('action', 'help', _HelpAction)
+        self.register('action', 'version', _VersionAction)
+        self.register('action', 'parsers', _SubParsersAction)
+
+        # raise an exception if the conflict handler is invalid
+        self._get_handler()
+
+        # action storage
+        self._actions = []
+        self._option_string_actions = {}
+
+        # groups
+        self._action_groups = []
+        self._mutually_exclusive_groups = []
+
+        # defaults storage
+        self._defaults = {}
+
+        # determines whether an "option" looks like a negative number
+        self._negative_number_matcher = _re.compile(r'^-\d+$|^-\d*\.\d+$')
+
+        # whether or not there are any optionals that look like negative
+        # numbers -- uses a list so it can be shared and edited
+        self._has_negative_number_optionals = []
+
+    # ====================
+    # Registration methods
+    # ====================
+    def register(self, registry_name, value, object):
+        registry = self._registries.setdefault(registry_name, {})
+        registry[value] = object
+
+    def _registry_get(self, registry_name, value, default=None):
+        return self._registries[registry_name].get(value, default)
+
+    # ==================================
+    # Namespace default accessor methods
+    # ==================================
+    def set_defaults(self, **kwargs):
+        self._defaults.update(kwargs)
+
+        # if these defaults match any existing arguments, replace
+        # the previous default on the object with the new one
+        for action in self._actions:
+            if action.dest in kwargs:
+                action.default = kwargs[action.dest]
+
+    def get_default(self, dest):
+        for action in self._actions:
+            if action.dest == dest and action.default is not None:
+                return action.default
+        return self._defaults.get(dest, None)
+
+
+    # =======================
+    # Adding argument actions
+    # =======================
+    def add_argument(self, *args, **kwargs):
+        """
+        add_argument(dest, ..., name=value, ...)
+        add_argument(option_string, option_string, ..., name=value, ...)
+        """
+
+        # if no positional args are supplied or only one is supplied and
+        # it doesn't look like an option string, parse a positional
+        # argument
+        chars = self.prefix_chars
+        if not args or len(args) == 1 and args[0][0] not in chars:
+            if args and 'dest' in kwargs:
+                raise ValueError('dest supplied twice for positional argument')
+            kwargs = self._get_positional_kwargs(*args, **kwargs)
+
+        # otherwise, we're adding an optional argument
+        else:
+            kwargs = self._get_optional_kwargs(*args, **kwargs)
+
+        # if no default was supplied, use the parser-level default
+        if 'default' not in kwargs:
+            dest = kwargs['dest']
+            if dest in self._defaults:
+                kwargs['default'] = self._defaults[dest]
+            elif self.argument_default is not None:
+                kwargs['default'] = self.argument_default
+
+        # create the action object, and add it to the parser
+        action_class = self._pop_action_class(kwargs)
+        if not _callable(action_class):
+            raise ValueError('unknown action "%s"' % action_class)
+        action = action_class(**kwargs)
+
+        # raise an error if the action type is not callable
+        type_func = self._registry_get('type', action.type, action.type)
+        if not _callable(type_func):
+            raise ValueError('%r is not callable' % type_func)
+
+        return self._add_action(action)
+
+    def add_argument_group(self, *args, **kwargs):
+        group = _ArgumentGroup(self, *args, **kwargs)
+        self._action_groups.append(group)
+        return group
+
+    def add_mutually_exclusive_group(self, **kwargs):
+        group = _MutuallyExclusiveGroup(self, **kwargs)
+        self._mutually_exclusive_groups.append(group)
+        return group
+
+    def _add_action(self, action):
+        # resolve any conflicts
+        self._check_conflict(action)
+
+        # add to actions list
+        self._actions.append(action)
+        action.container = self
+
+        # index the action by any option strings it has
+        for option_string in action.option_strings:
+            self._option_string_actions[option_string] = action
+
+        # set the flag if any option strings look like negative numbers
+        for option_string in action.option_strings:
+            if self._negative_number_matcher.match(option_string):
+                if not self._has_negative_number_optionals:
+                    self._has_negative_number_optionals.append(True)
+
+        # return the created action
+        return action
+
+    def _remove_action(self, action):
+        self._actions.remove(action)
+
+    def _add_container_actions(self, container):
+        # collect groups by titles
+        title_group_map = {}
+        for group in self._action_groups:
+            if group.title in title_group_map:
+                msg = _('cannot merge actions - two groups are named %r')
+                raise ValueError(msg % (group.title))
+            title_group_map[group.title] = group
+
+        # map each action to its group
+        group_map = {}
+        for group in container._action_groups:
+
+            # if a group with the title exists, use that, otherwise
+            # create a new group matching the container's group
+            if group.title not in title_group_map:
+                title_group_map[group.title] = self.add_argument_group(
+                    title=group.title,
+                    description=group.description,
+                    conflict_handler=group.conflict_handler)
+
+            # map the actions to their new group
+            for action in group._group_actions:
+                group_map[action] = title_group_map[group.title]
+
+        # add container's mutually exclusive groups
+        # NOTE: if add_mutually_exclusive_group ever gains title= and
+        # description= then this code will need to be expanded as above
+        for group in container._mutually_exclusive_groups:
+            mutex_group = self.add_mutually_exclusive_group(
+                required=group.required)
+
+            # map the actions to their new mutex group
+            for action in group._group_actions:
+                group_map[action] = mutex_group
+
+        # add all actions to this container or their group
+        for action in container._actions:
+            group_map.get(action, self)._add_action(action)
+
+    def _get_positional_kwargs(self, dest, **kwargs):
+        # make sure required is not specified
+        if 'required' in kwargs:
+            msg = _("'required' is an invalid argument for positionals")
+            raise TypeError(msg)
+
+        # mark positional arguments as required if at least one is
+        # always required
+        if kwargs.get('nargs') not in [OPTIONAL, ZERO_OR_MORE]:
+            kwargs['required'] = True
+        if kwargs.get('nargs') == ZERO_OR_MORE and 'default' not in kwargs:
+            kwargs['required'] = True
+
+        # return the keyword arguments with no option strings
+        return dict(kwargs, dest=dest, option_strings=[])
+
+    def _get_optional_kwargs(self, *args, **kwargs):
+        # determine short and long option strings
+        option_strings = []
+        long_option_strings = []
+        for option_string in args:
+            # error on strings that don't start with an appropriate prefix
+            if not option_string[0] in self.prefix_chars:
+                msg = _('invalid option string %r: '
+                        'must start with a character %r')
+                tup = option_string, self.prefix_chars
+                raise ValueError(msg % tup)
+
+            # strings starting with two prefix characters are long options
+            option_strings.append(option_string)
+            if option_string[0] in self.prefix_chars:
+                if len(option_string) > 1:
+                    if option_string[1] in self.prefix_chars:
+                        long_option_strings.append(option_string)
+
+        # infer destination, '--foo-bar' -> 'foo_bar' and '-x' -> 'x'
+        dest = kwargs.pop('dest', None)
+        if dest is None:
+            if long_option_strings:
+                dest_option_string = long_option_strings[0]
+            else:
+                dest_option_string = option_strings[0]
+            dest = dest_option_string.lstrip(self.prefix_chars)
+            if not dest:
+                msg = _('dest= is required for options like %r')
+                raise ValueError(msg % option_string)
+            dest = dest.replace('-', '_')
+
+        # return the updated keyword arguments
+        return dict(kwargs, dest=dest, option_strings=option_strings)
+
+    def _pop_action_class(self, kwargs, default=None):
+        action = kwargs.pop('action', default)
+        return self._registry_get('action', action, action)
+
+    def _get_handler(self):
+        # determine function from conflict handler string
+        handler_func_name = '_handle_conflict_%s' % self.conflict_handler
+        try:
+            return getattr(self, handler_func_name)
+        except AttributeError:
+            msg = _('invalid conflict_resolution value: %r')
+            raise ValueError(msg % self.conflict_handler)
+
+    def _check_conflict(self, action):
+
+        # find all options that conflict with this option
+        confl_optionals = []
+        for option_string in action.option_strings:
+            if option_string in self._option_string_actions:
+                confl_optional = self._option_string_actions[option_string]
+                confl_optionals.append((option_string, confl_optional))
+
+        # resolve any conflicts
+        if confl_optionals:
+            conflict_handler = self._get_handler()
+            conflict_handler(action, confl_optionals)
+
+    def _handle_conflict_error(self, action, conflicting_actions):
+        message = _('conflicting option string(s): %s')
+        conflict_string = ', '.join([option_string
+                                     for option_string, action
+                                     in conflicting_actions])
+        raise ArgumentError(action, message % conflict_string)
+
+    def _handle_conflict_resolve(self, action, conflicting_actions):
+
+        # remove all conflicting options
+        for option_string, action in conflicting_actions:
+
+            # remove the conflicting option
+            action.option_strings.remove(option_string)
+            self._option_string_actions.pop(option_string, None)
+
+            # if the option now has no option string, remove it from the
+            # container holding it
+            if not action.option_strings:
+                action.container._remove_action(action)
+
+
+class _ArgumentGroup(_ActionsContainer):
+
+    def __init__(self, container, title=None, description=None, **kwargs):
+        # add any missing keyword arguments by checking the container
+        update = kwargs.setdefault
+        update('conflict_handler', container.conflict_handler)
+        update('prefix_chars', container.prefix_chars)
+        update('argument_default', container.argument_default)
+        super_init = super(_ArgumentGroup, self).__init__
+        super_init(description=description, **kwargs)
+
+        # group attributes
+        self.title = title
+        self._group_actions = []
+
+        # share most attributes with the container
+        self._registries = container._registries
+        self._actions = container._actions
+        self._option_string_actions = container._option_string_actions
+        self._defaults = container._defaults
+        self._has_negative_number_optionals = \
+            container._has_negative_number_optionals
+
+    def _add_action(self, action):
+        action = super(_ArgumentGroup, self)._add_action(action)
+        self._group_actions.append(action)
+        return action
+
+    def _remove_action(self, action):
+        super(_ArgumentGroup, self)._remove_action(action)
+        self._group_actions.remove(action)
+
+
+class _MutuallyExclusiveGroup(_ArgumentGroup):
+
+    def __init__(self, container, required=False):
+        super(_MutuallyExclusiveGroup, self).__init__(container)
+        self.required = required
+        self._container = container
+
+    def _add_action(self, action):
+        if action.required:
+            msg = _('mutually exclusive arguments must be optional')
+            raise ValueError(msg)
+        action = self._container._add_action(action)
+        self._group_actions.append(action)
+        return action
+
+    def _remove_action(self, action):
+        self._container._remove_action(action)
+        self._group_actions.remove(action)
+
+
+class ArgumentParser(_AttributeHolder, _ActionsContainer):
+    """Object for parsing command line strings into Python objects.
+
+    Keyword Arguments:
+        - prog -- The name of the program (default: sys.argv[0])
+        - usage -- A usage message (default: auto-generated from arguments)
+        - description -- A description of what the program does
+        - epilog -- Text following the argument descriptions
+        - parents -- Parsers whose arguments should be copied into this one
+        - formatter_class -- HelpFormatter class for printing help messages
+        - prefix_chars -- Characters that prefix optional arguments
+        - fromfile_prefix_chars -- Characters that prefix files containing
+            additional arguments
+        - argument_default -- The default value for all arguments
+        - conflict_handler -- String indicating how to handle conflicts
+        - add_help -- Add a -h/-help option
+    """
+
+    def __init__(self,
+                 prog=None,
+                 usage=None,
+                 description=None,
+                 epilog=None,
+                 version=None,
+                 parents=[],
+                 formatter_class=HelpFormatter,
+                 prefix_chars='-',
+                 fromfile_prefix_chars=None,
+                 argument_default=None,
+                 conflict_handler='error',
+                 add_help=True):
+
+        if version is not None:
+            import warnings
+            warnings.warn(
+                """The "version" argument to ArgumentParser is deprecated. """
+                """Please use """
+                """"add_argument(..., action='version', version="N", ...)" """
+                """instead""", DeprecationWarning)
+
+        superinit = super(ArgumentParser, self).__init__
+        superinit(description=description,
+                  prefix_chars=prefix_chars,
+                  argument_default=argument_default,
+                  conflict_handler=conflict_handler)
+
+        # default setting for prog
+        if prog is None:
+            prog = _os.path.basename(_sys.argv[0])
+
+        self.prog = prog
+        self.usage = usage
+        self.epilog = epilog
+        self.version = version
+        self.formatter_class = formatter_class
+        self.fromfile_prefix_chars = fromfile_prefix_chars
+        self.add_help = add_help
+
+        add_group = self.add_argument_group
+        self._positionals = add_group(_('positional arguments'))
+        self._optionals = add_group(_('optional arguments'))
+        self._subparsers = None
+
+        # register types
+        def identity(string):
+            return string
+        self.register('type', None, identity)
+
+        # add help and version arguments if necessary
+        # (using explicit default to override global argument_default)
+        if '-' in prefix_chars:
+            default_prefix = '-'
+        else:
+            default_prefix = prefix_chars[0]
+        if self.add_help:
+            self.add_argument(
+                default_prefix+'h', default_prefix*2+'help',
+                action='help', default=SUPPRESS,
+                help=_('show this help message and exit'))
+        if self.version:
+            self.add_argument(
+                default_prefix+'v', default_prefix*2+'version',
+                action='version', default=SUPPRESS,
+                version=self.version,
+                help=_("show program's version number and exit"))
+
+        # add parent arguments and defaults
+        for parent in parents:
+            self._add_container_actions(parent)
+            try:
+                defaults = parent._defaults
+            except AttributeError:
+                pass
+            else:
+                self._defaults.update(defaults)
+
+    # =======================
+    # Pretty __repr__ methods
+    # =======================
+    def _get_kwargs(self):
+        names = [
+            'prog',
+            'usage',
+            'description',
+            'version',
+            'formatter_class',
+            'conflict_handler',
+            'add_help',
+        ]
+        return [(name, getattr(self, name)) for name in names]
+
+    # ==================================
+    # Optional/Positional adding methods
+    # ==================================
+    def add_subparsers(self, **kwargs):
+        if self._subparsers is not None:
+            self.error(_('cannot have multiple subparser arguments'))
+
+        # add the parser class to the arguments if it's not present
+        kwargs.setdefault('parser_class', type(self))
+
+        if 'title' in kwargs or 'description' in kwargs:
+            title = _(kwargs.pop('title', 'subcommands'))
+            description = _(kwargs.pop('description', None))
+            self._subparsers = self.add_argument_group(title, description)
+        else:
+            self._subparsers = self._positionals
+
+        # prog defaults to the usage message of this parser, skipping
+        # optional arguments and with no "usage:" prefix
+        if kwargs.get('prog') is None:
+            formatter = self._get_formatter()
+            positionals = self._get_positional_actions()
+            groups = self._mutually_exclusive_groups
+            formatter.add_usage(self.usage, positionals, groups, '')
+            kwargs['prog'] = formatter.format_help().strip()
+
+        # create the parsers action and add it to the positionals list
+        parsers_class = self._pop_action_class(kwargs, 'parsers')
+        action = parsers_class(option_strings=[], **kwargs)
+        self._subparsers._add_action(action)
+
+        # return the created parsers action
+        return action
+
+    def _add_action(self, action):
+        if action.option_strings:
+            self._optionals._add_action(action)
+        else:
+            self._positionals._add_action(action)
+        return action
+
+    def _get_optional_actions(self):
+        return [action
+                for action in self._actions
+                if action.option_strings]
+
+    def _get_positional_actions(self):
+        return [action
+                for action in self._actions
+                if not action.option_strings]
+
+    # =====================================
+    # Command line argument parsing methods
+    # =====================================
+    def parse_args(self, args=None, namespace=None):
+        args, argv = self.parse_known_args(args, namespace)
+        if argv:
+            msg = _('unrecognized arguments: %s')
+            self.error(msg % ' '.join(argv))
+        return args
+
+    def parse_known_args(self, args=None, namespace=None):
+        # args default to the system args
+        if args is None:
+            args = _sys.argv[1:]
+
+        # default Namespace built from parser defaults
+        if namespace is None:
+            namespace = Namespace()
+
+        # add any action defaults that aren't present
+        for action in self._actions:
+            if action.dest is not SUPPRESS:
+                if not hasattr(namespace, action.dest):
+                    if action.default is not SUPPRESS:
+                        setattr(namespace, action.dest, action.default)
+
+        # add any parser defaults that aren't present
+        for dest in self._defaults:
+            if not hasattr(namespace, dest):
+                setattr(namespace, dest, self._defaults[dest])
+
+        # parse the arguments and exit if there are any errors
+        try:
+            namespace, args = self._parse_known_args(args, namespace)
+            if hasattr(namespace, _UNRECOGNIZED_ARGS_ATTR):
+                args.extend(getattr(namespace, _UNRECOGNIZED_ARGS_ATTR))
+                delattr(namespace, _UNRECOGNIZED_ARGS_ATTR)
+            return namespace, args
+        except ArgumentError:
+            err = _sys.exc_info()[1]
+            self.error(str(err))
+
+    def _parse_known_args(self, arg_strings, namespace):
+        # replace arg strings that are file references
+        if self.fromfile_prefix_chars is not None:
+            arg_strings = self._read_args_from_files(arg_strings)
+
+        # map all mutually exclusive arguments to the other arguments
+        # they can't occur with
+        action_conflicts = {}
+        for mutex_group in self._mutually_exclusive_groups:
+            group_actions = mutex_group._group_actions
+            for i, mutex_action in enumerate(mutex_group._group_actions):
+                conflicts = action_conflicts.setdefault(mutex_action, [])
+                conflicts.extend(group_actions[:i])
+                conflicts.extend(group_actions[i + 1:])
+
+        # find all option indices, and determine the arg_string_pattern
+        # which has an 'O' if there is an option at an index,
+        # an 'A' if there is an argument, or a '-' if there is a '--'
+        option_string_indices = {}
+        arg_string_pattern_parts = []
+        arg_strings_iter = iter(arg_strings)
+        for i, arg_string in enumerate(arg_strings_iter):
+
+            # all args after -- are non-options
+            if arg_string == '--':
+                arg_string_pattern_parts.append('-')
+                for arg_string in arg_strings_iter:
+                    arg_string_pattern_parts.append('A')
+
+            # otherwise, add the arg to the arg strings
+            # and note the index if it was an option
+            else:
+                option_tuple = self._parse_optional(arg_string)
+                if option_tuple is None:
+                    pattern = 'A'
+                else:
+                    option_string_indices[i] = option_tuple
+                    pattern = 'O'
+                arg_string_pattern_parts.append(pattern)
+
+        # join the pieces together to form the pattern
+        arg_strings_pattern = ''.join(arg_string_pattern_parts)
+
+        # converts arg strings to the appropriate and then takes the action
+        seen_actions = set()
+        seen_non_default_actions = set()
+
+        def take_action(action, argument_strings, option_string=None):
+            seen_actions.add(action)
+            argument_values = self._get_values(action, argument_strings)
+
+            # error if this argument is not allowed with other previously
+            # seen arguments, assuming that actions that use the default
+            # value don't really count as "present"
+            if argument_values is not action.default:
+                seen_non_default_actions.add(action)
+                for conflict_action in action_conflicts.get(action, []):
+                    if conflict_action in seen_non_default_actions:
+                        msg = _('not allowed with argument %s')
+                        action_name = _get_action_name(conflict_action)
+                        raise ArgumentError(action, msg % action_name)
+
+            # take the action if we didn't receive a SUPPRESS value
+            # (e.g. from a default)
+            if argument_values is not SUPPRESS:
+                action(self, namespace, argument_values, option_string)
+
+        # function to convert arg_strings into an optional action
+        def consume_optional(start_index):
+
+            # get the optional identified at this index
+            option_tuple = option_string_indices[start_index]
+            action, option_string, explicit_arg = option_tuple
+
+            # identify additional optionals in the same arg string
+            # (e.g. -xyz is the same as -x -y -z if no args are required)
+            match_argument = self._match_argument
+            action_tuples = []
+            while True:
+
+                # if we found no optional action, skip it
+                if action is None:
+                    extras.append(arg_strings[start_index])
+                    return start_index + 1
+
+                # if there is an explicit argument, try to match the
+                # optional's string arguments to only this
+                if explicit_arg is not None:
+                    arg_count = match_argument(action, 'A')
+
+                    # if the action is a single-dash option and takes no
+                    # arguments, try to parse more single-dash options out
+                    # of the tail of the option string
+                    chars = self.prefix_chars
+                    if arg_count == 0 and option_string[1] not in chars:
+                        action_tuples.append((action, [], option_string))
+                        char = option_string[0]
+                        option_string = char + explicit_arg[0]
+                        new_explicit_arg = explicit_arg[1:] or None
+                        optionals_map = self._option_string_actions
+                        if option_string in optionals_map:
+                            action = optionals_map[option_string]
+                            explicit_arg = new_explicit_arg
+                        else:
+                            msg = _('ignored explicit argument %r')
+                            raise ArgumentError(action, msg % explicit_arg)
+
+                    # if the action expect exactly one argument, we've
+                    # successfully matched the option; exit the loop
+                    elif arg_count == 1:
+                        stop = start_index + 1
+                        args = [explicit_arg]
+                        action_tuples.append((action, args, option_string))
+                        break
+
+                    # error if a double-dash option did not use the
+                    # explicit argument
+                    else:
+                        msg = _('ignored explicit argument %r')
+                        raise ArgumentError(action, msg % explicit_arg)
+
+                # if there is no explicit argument, try to match the
+                # optional's string arguments with the following strings
+                # if successful, exit the loop
+                else:
+                    start = start_index + 1
+                    selected_patterns = arg_strings_pattern[start:]
+                    arg_count = match_argument(action, selected_patterns)
+                    stop = start + arg_count
+                    args = arg_strings[start:stop]
+                    action_tuples.append((action, args, option_string))
+                    break
+
+            # add the Optional to the list and return the index at which
+            # the Optional's string args stopped
+            assert action_tuples
+            for action, args, option_string in action_tuples:
+                take_action(action, args, option_string)
+            return stop
+
+        # the list of Positionals left to be parsed; this is modified
+        # by consume_positionals()
+        positionals = self._get_positional_actions()
+
+        # function to convert arg_strings into positional actions
+        def consume_positionals(start_index):
+            # match as many Positionals as possible
+            match_partial = self._match_arguments_partial
+            selected_pattern = arg_strings_pattern[start_index:]
+            arg_counts = match_partial(positionals, selected_pattern)
+
+            # slice off the appropriate arg strings for each Positional
+            # and add the Positional and its args to the list
+            for action, arg_count in zip(positionals, arg_counts):
+                args = arg_strings[start_index: start_index + arg_count]
+                start_index += arg_count
+                take_action(action, args)
+
+            # slice off the Positionals that we just parsed and return the
+            # index at which the Positionals' string args stopped
+            positionals[:] = positionals[len(arg_counts):]
+            return start_index
+
+        # consume Positionals and Optionals alternately, until we have
+        # passed the last option string
+        extras = []
+        start_index = 0
+        if option_string_indices:
+            max_option_string_index = max(option_string_indices)
+        else:
+            max_option_string_index = -1
+        while start_index <= max_option_string_index:
+
+            # consume any Positionals preceding the next option
+            next_option_string_index = min([
+                index
+                for index in option_string_indices
+                if index >= start_index])
+            if start_index != next_option_string_index:
+                positionals_end_index = consume_positionals(start_index)
+
+                # only try to parse the next optional if we didn't consume
+                # the option string during the positionals parsing
+                if positionals_end_index > start_index:
+                    start_index = positionals_end_index
+                    continue
+                else:
+                    start_index = positionals_end_index
+
+            # if we consumed all the positionals we could and we're not
+            # at the index of an option string, there were extra arguments
+            if start_index not in option_string_indices:
+                strings = arg_strings[start_index:next_option_string_index]
+                extras.extend(strings)
+                start_index = next_option_string_index
+
+            # consume the next optional and any arguments for it
+            start_index = consume_optional(start_index)
+
+        # consume any positionals following the last Optional
+        stop_index = consume_positionals(start_index)
+
+        # if we didn't consume all the argument strings, there were extras
+        extras.extend(arg_strings[stop_index:])
+
+        # if we didn't use all the Positional objects, there were too few
+        # arg strings supplied.
+        if positionals:
+            self.error(_('too few arguments'))
+
+        # make sure all required actions were present, and convert defaults.
+        for action in self._actions:
+            if action not in seen_actions:
+                if action.required:
+                    name = _get_action_name(action)
+                    self.error(_('argument %s is required') % name)
+                else:
+                    # Convert action default now instead of doing it before
+                    # parsing arguments to avoid calling convert functions
+                    # twice (which may fail) if the argument was given, but
+                    # only if it was defined already in the namespace
+                    if (action.default is not None and
+                            isinstance(action.default, basestring) and
+                            hasattr(namespace, action.dest) and
+                            action.default is getattr(namespace, action.dest)):
+                        setattr(namespace, action.dest,
+                                self._get_value(action, action.default))
+
+        # make sure all required groups had one option present
+        for group in self._mutually_exclusive_groups:
+            if group.required:
+                for action in group._group_actions:
+                    if action in seen_non_default_actions:
+                        break
+
+                # if no actions were used, report the error
+                else:
+                    names = [_get_action_name(action)
+                             for action in group._group_actions
+                             if action.help is not SUPPRESS]
+                    msg = _('one of the arguments %s is required')
+                    self.error(msg % ' '.join(names))
+
+        # return the updated namespace and the extra arguments
+        return namespace, extras
+
+    def _read_args_from_files(self, arg_strings):
+        # expand arguments referencing files
+        new_arg_strings = []
+        for arg_string in arg_strings:
+
+            # for regular arguments, just add them back into the list
+            if arg_string[0] not in self.fromfile_prefix_chars:
+                new_arg_strings.append(arg_string)
+
+            # replace arguments referencing files with the file content
+            else:
+                try:
+                    args_file = open(arg_string[1:])
+                    try:
+                        arg_strings = []
+                        for arg_line in args_file.read().splitlines():
+                            for arg in self.convert_arg_line_to_args(arg_line):
+                                arg_strings.append(arg)
+                        arg_strings = self._read_args_from_files(arg_strings)
+                        new_arg_strings.extend(arg_strings)
+                    finally:
+                        args_file.close()
+                except IOError:
+                    err = _sys.exc_info()[1]
+                    self.error(str(err))
+
+        # return the modified argument list
+        return new_arg_strings
+
+    def convert_arg_line_to_args(self, arg_line):
+        return [arg_line]
+
+    def _match_argument(self, action, arg_strings_pattern):
+        # match the pattern for this action to the arg strings
+        nargs_pattern = self._get_nargs_pattern(action)
+        match = _re.match(nargs_pattern, arg_strings_pattern)
+
+        # raise an exception if we weren't able to find a match
+        if match is None:
+            nargs_errors = {
+                None: _('expected one argument'),
+                OPTIONAL: _('expected at most one argument'),
+                ONE_OR_MORE: _('expected at least one argument'),
+            }
+            default = _('expected %s argument(s)') % action.nargs
+            msg = nargs_errors.get(action.nargs, default)
+            raise ArgumentError(action, msg)
+
+        # return the number of arguments matched
+        return len(match.group(1))
+
+    def _match_arguments_partial(self, actions, arg_strings_pattern):
+        # progressively shorten the actions list by slicing off the
+        # final actions until we find a match
+        result = []
+        for i in range(len(actions), 0, -1):
+            actions_slice = actions[:i]
+            pattern = ''.join([self._get_nargs_pattern(action)
+                               for action in actions_slice])
+            match = _re.match(pattern, arg_strings_pattern)
+            if match is not None:
+                result.extend([len(string) for string in match.groups()])
+                break
+
+        # return the list of arg string counts
+        return result
+
+    def _parse_optional(self, arg_string):
+        # if it's an empty string, it was meant to be a positional
+        if not arg_string:
+            return None
+
+        # if it doesn't start with a prefix, it was meant to be positional
+        if not arg_string[0] in self.prefix_chars:
+            return None
+
+        # if the option string is present in the parser, return the action
+        if arg_string in self._option_string_actions:
+            action = self._option_string_actions[arg_string]
+            return action, arg_string, None
+
+        # if it's just a single character, it was meant to be positional
+        if len(arg_string) == 1:
+            return None
+
+        # if the option string before the "=" is present, return the action
+        if '=' in arg_string:
+            option_string, explicit_arg = arg_string.split('=', 1)
+            if option_string in self._option_string_actions:
+                action = self._option_string_actions[option_string]
+                return action, option_string, explicit_arg
+
+        # search through all possible prefixes of the option string
+        # and all actions in the parser for possible interpretations
+        option_tuples = self._get_option_tuples(arg_string)
+
+        # if multiple actions match, the option string was ambiguous
+        if len(option_tuples) > 1:
+            options = ', '.join([option_string
+                for action, option_string, explicit_arg in option_tuples])
+            tup = arg_string, options
+            self.error(_('ambiguous option: %s could match %s') % tup)
+
+        # if exactly one action matched, this segmentation is good,
+        # so return the parsed action
+        elif len(option_tuples) == 1:
+            option_tuple, = option_tuples
+            return option_tuple
+
+        # if it was not found as an option, but it looks like a negative
+        # number, it was meant to be positional
+        # unless there are negative-number-like options
+        if self._negative_number_matcher.match(arg_string):
+            if not self._has_negative_number_optionals:
+                return None
+
+        # if it contains a space, it was meant to be a positional
+        if ' ' in arg_string:
+            return None
+
+        # it was meant to be an optional but there is no such option
+        # in this parser (though it might be a valid option in a subparser)
+        return None, arg_string, None
+
+    def _get_option_tuples(self, option_string):
+        result = []
+
+        # option strings starting with two prefix characters are only
+        # split at the '='
+        chars = self.prefix_chars
+        if option_string[0] in chars and option_string[1] in chars:
+            if '=' in option_string:
+                option_prefix, explicit_arg = option_string.split('=', 1)
+            else:
+                option_prefix = option_string
+                explicit_arg = None
+            for option_string in self._option_string_actions:
+                if option_string.startswith(option_prefix):
+                    action = self._option_string_actions[option_string]
+                    tup = action, option_string, explicit_arg
+                    result.append(tup)
+
+        # single character options can be concatenated with their arguments
+        # but multiple character options always have to have their argument
+        # separate
+        elif option_string[0] in chars and option_string[1] not in chars:
+            option_prefix = option_string
+            explicit_arg = None
+            short_option_prefix = option_string[:2]
+            short_explicit_arg = option_string[2:]
+
+            for option_string in self._option_string_actions:
+                if option_string == short_option_prefix:
+                    action = self._option_string_actions[option_string]
+                    tup = action, option_string, short_explicit_arg
+                    result.append(tup)
+                elif option_string.startswith(option_prefix):
+                    action = self._option_string_actions[option_string]
+                    tup = action, option_string, explicit_arg
+                    result.append(tup)
+
+        # shouldn't ever get here
+        else:
+            self.error(_('unexpected option string: %s') % option_string)
+
+        # return the collected option tuples
+        return result
+
+    def _get_nargs_pattern(self, action):
+        # in all examples below, we have to allow for '--' args
+        # which are represented as '-' in the pattern
+        nargs = action.nargs
+
+        # the default (None) is assumed to be a single argument
+        if nargs is None:
+            nargs_pattern = '(-*A-*)'
+
+        # allow zero or one arguments
+        elif nargs == OPTIONAL:
+            nargs_pattern = '(-*A?-*)'
+
+        # allow zero or more arguments
+        elif nargs == ZERO_OR_MORE:
+            nargs_pattern = '(-*[A-]*)'
+
+        # allow one or more arguments
+        elif nargs == ONE_OR_MORE:
+            nargs_pattern = '(-*A[A-]*)'
+
+        # allow any number of options or arguments
+        elif nargs == REMAINDER:
+            nargs_pattern = '([-AO]*)'
+
+        # allow one argument followed by any number of options or arguments
+        elif nargs == PARSER:
+            nargs_pattern = '(-*A[-AO]*)'
+
+        # all others should be integers
+        else:
+            nargs_pattern = '(-*%s-*)' % '-*'.join('A' * nargs)
+
+        # if this is an optional action, -- is not allowed
+        if action.option_strings:
+            nargs_pattern = nargs_pattern.replace('-*', '')
+            nargs_pattern = nargs_pattern.replace('-', '')
+
+        # return the pattern
+        return nargs_pattern
+
+    # ========================
+    # Value conversion methods
+    # ========================
+    def _get_values(self, action, arg_strings):
+        # for everything but PARSER args, strip out '--'
+        if action.nargs not in [PARSER, REMAINDER]:
+            arg_strings = [s for s in arg_strings if s != '--']
+
+        # optional argument produces a default when not present
+        if not arg_strings and action.nargs == OPTIONAL:
+            if action.option_strings:
+                value = action.const
+            else:
+                value = action.default
+            if isinstance(value, basestring):
+                value = self._get_value(action, value)
+                self._check_value(action, value)
+
+        # when nargs='*' on a positional, if there were no command-line
+        # args, use the default if it is anything other than None
+        elif (not arg_strings and action.nargs == ZERO_OR_MORE and
+              not action.option_strings):
+            if action.default is not None:
+                value = action.default
+            else:
+                value = arg_strings
+            self._check_value(action, value)
+
+        # single argument or optional argument produces a single value
+        elif len(arg_strings) == 1 and action.nargs in [None, OPTIONAL]:
+            arg_string, = arg_strings
+            value = self._get_value(action, arg_string)
+            self._check_value(action, value)
+
+        # REMAINDER arguments convert all values, checking none
+        elif action.nargs == REMAINDER:
+            value = [self._get_value(action, v) for v in arg_strings]
+
+        # PARSER arguments convert all values, but check only the first
+        elif action.nargs == PARSER:
+            value = [self._get_value(action, v) for v in arg_strings]
+            self._check_value(action, value[0])
+
+        # all other types of nargs produce a list
+        else:
+            value = [self._get_value(action, v) for v in arg_strings]
+            for v in value:
+                self._check_value(action, v)
+
+        # return the converted value
+        return value
+
+    def _get_value(self, action, arg_string):
+        type_func = self._registry_get('type', action.type, action.type)
+        if not _callable(type_func):
+            msg = _('%r is not callable')
+            raise ArgumentError(action, msg % type_func)
+
+        # convert the value to the appropriate type
+        try:
+            result = type_func(arg_string)
+
+        # ArgumentTypeErrors indicate errors
+        except ArgumentTypeError:
+            name = getattr(action.type, '__name__', repr(action.type))
+            msg = str(_sys.exc_info()[1])
+            raise ArgumentError(action, msg)
+
+        # TypeErrors or ValueErrors also indicate errors
+        except (TypeError, ValueError):
+            name = getattr(action.type, '__name__', repr(action.type))
+            msg = _('invalid %s value: %r')
+            raise ArgumentError(action, msg % (name, arg_string))
+
+        # return the converted value
+        return result
+
+    def _check_value(self, action, value):
+        # converted value must be one of the choices (if specified)
+        if action.choices is not None and value not in action.choices:
+            tup = value, ', '.join(map(repr, action.choices))
+            msg = _('invalid choice: %r (choose from %s)') % tup
+            raise ArgumentError(action, msg)
+
+    # =======================
+    # Help-formatting methods
+    # =======================
+    def format_usage(self):
+        formatter = self._get_formatter()
+        formatter.add_usage(self.usage, self._actions,
+                            self._mutually_exclusive_groups)
+        return formatter.format_help()
+
+    def format_help(self):
+        formatter = self._get_formatter()
+
+        # usage
+        formatter.add_usage(self.usage, self._actions,
+                            self._mutually_exclusive_groups)
+
+        # description
+        formatter.add_tex

<TRUNCATED>

[07/50] [abbrv] phoenix git commit: PHOENIX-4342 - Surface QueryPlan in MutationPlan

Posted by td...@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/1d8a6bc3
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/1d8a6bc3
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/1d8a6bc3

Branch: refs/heads/system-catalog
Commit: 1d8a6bc3a6a277d9e3201066b753fa9fd7018545
Parents: 2a8e1c7
Author: Geoffrey Jacoby <gj...@apache.org>
Authored: Thu Nov 2 13:41:02 2017 -0700
Committer: Geoffrey Jacoby <gj...@apache.org>
Committed: Mon Nov 13 11:47:50 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/1d8a6bc3/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/1d8a6bc3/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/1d8a6bc3/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/1d8a6bc3/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/1d8a6bc3/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 6445894..3603ce7 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
@@ -694,173 +694,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);
         }
 
             
@@ -986,124 +826,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) {
@@ -1214,4 +939,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/1d8a6bc3/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() {


[05/50] [abbrv] phoenix git commit: Merge remote-tracking branch 'upstream/master' into PHOENIX-3534

Posted by td...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/39c0d1d4/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ViewFinder.java
----------------------------------------------------------------------
diff --cc phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ViewFinder.java
index 36a3f9c,0000000..cde7a01
mode 100644,000000..100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ViewFinder.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ViewFinder.java
@@@ -1,223 -1,0 +1,224 @@@
 +/**
 + * 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 static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.LINK_TYPE_BYTES;
 +import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.PARENT_TENANT_ID_BYTES;
 +import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES;
 +import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_TYPE_BYTES;
++import static org.apache.phoenix.query.QueryConstants.SEPARATOR_BYTE_ARRAY;
 +import static org.apache.phoenix.util.SchemaUtil.getVarChars;
 +
 +import java.io.IOException;
 +import java.util.LinkedHashSet;
 +import java.util.List;
 +import java.util.Set;
 +
 +import org.apache.hadoop.hbase.HConstants;
 +import org.apache.hadoop.hbase.client.Result;
 +import org.apache.hadoop.hbase.client.ResultScanner;
 +import org.apache.hadoop.hbase.client.Scan;
 +import org.apache.hadoop.hbase.client.Table;
 +import org.apache.hadoop.hbase.filter.CompareFilter;
 +import org.apache.hadoop.hbase.filter.FilterList;
 +import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
 +import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 +import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 +import org.apache.phoenix.schema.PTable;
 +import org.apache.phoenix.schema.tuple.ResultTuple;
 +import org.apache.phoenix.util.ByteUtil;
 +import org.apache.phoenix.util.SchemaUtil;
 +
 +import com.google.common.collect.LinkedHashMultimap;
 +import com.google.common.collect.Lists;
 +import com.google.common.collect.Sets;
 +
 +class ViewFinder {
 +
- 	// The PHYSICAL_TABLE link from view to the base table overwrites the PARENT_TABLE link 
++	// The PHYSICAL_TABLE link from view to the base table overwrites the PARENT_TABLE link (when namespace mapping is disabled)
 +    static TableViewFinderResult findBaseTable(Table systemCatalog, byte[] tenantId, byte[] schema, byte[] table)
 +        throws IOException {
 +        return findRelatedViews(systemCatalog, tenantId, schema, table, PTable.LinkType.PHYSICAL_TABLE,
 +            HConstants.LATEST_TIMESTAMP);
 +    }
 +    
 +    static TableViewFinderResult findParentViewofIndex(Table systemCatalog, byte[] tenantId, byte[] schema, byte[] table)
 +            throws IOException {
 +            return findRelatedViews(systemCatalog, tenantId, schema, table, PTable.LinkType.VIEW_INDEX_PARENT_TABLE,
 +                HConstants.LATEST_TIMESTAMP);
 +        }
 +
 +    static void findAllRelatives(Table systemTable, byte[] tenantId, byte[] schema, byte[] table,
 +        PTable.LinkType linkType, TableViewFinderResult result) throws IOException {
 +        findAllRelatives(systemTable, tenantId, schema, table, linkType, HConstants.LATEST_TIMESTAMP, result);
 +    }
 +
 +    static void findAllRelatives(Table systemCatalog, byte[] tenantId, byte[] schema, byte[] table,
 +        PTable.LinkType linkType, long timestamp, TableViewFinderResult result) throws IOException {
 +        TableViewFinderResult currentResult =
 +            findRelatedViews(systemCatalog, tenantId, schema, table, linkType, timestamp);
 +        result.addResult(currentResult);
 +        for (TableInfo viewInfo : currentResult.getResults()) {
 +            findAllRelatives(systemCatalog, viewInfo.getTenantId(), viewInfo.getSchemaName(), viewInfo.getTableName(), linkType, timestamp, result);
 +        }
 +    }
 +
 +    static TableViewFinderResult findRelatedViews(Table systemCatalog, byte[] tenantId, byte[] schema, byte[] table,
 +        PTable.LinkType linkType, long timestamp) throws IOException {
 +        if (linkType==PTable.LinkType.INDEX_TABLE || linkType==PTable.LinkType.EXCLUDED_COLUMN) {
 +            throw new IllegalArgumentException("findAllRelatives does not support link type "+linkType);
 +        }
 +        Scan scan = new Scan();
-         byte[] startRow = SchemaUtil.getTableKey(tenantId, schema, table);
++        byte[] startRow = ByteUtil.concat(SchemaUtil.getTableKey(tenantId, schema, table), SEPARATOR_BYTE_ARRAY);
 +        byte[] stopRow = ByteUtil.nextKey(startRow);
 +        scan.setStartRow(startRow);
 +        scan.setStopRow(stopRow);
 +        scan.setTimeRange(0, timestamp);
 +        SingleColumnValueFilter linkFilter =
 +            new SingleColumnValueFilter(TABLE_FAMILY_BYTES, LINK_TYPE_BYTES, CompareFilter.CompareOp.EQUAL,
 +                linkType.getSerializedValueAsByteArray());
 +        linkFilter.setFilterIfMissing(true);
 +        scan.setFilter(linkFilter);
 +        scan.addColumn(TABLE_FAMILY_BYTES, LINK_TYPE_BYTES);
 +        if (linkType==PTable.LinkType.PARENT_TABLE)
 +            scan.addColumn(TABLE_FAMILY_BYTES, PARENT_TENANT_ID_BYTES);
 +        if (linkType==PTable.LinkType.PHYSICAL_TABLE)
 +            scan.addColumn(TABLE_FAMILY_BYTES, TABLE_TYPE_BYTES);
 +        ResultScanner scanner = systemCatalog.getScanner(scan);
 +        List<TableInfo> tableInfoList = Lists.newArrayList();
 +        try {
 +            for (Result result = scanner.next(); (result != null); result = scanner.next()) {
 +                byte[][] rowKeyMetaData = new byte[5][];
 +                byte[] viewTenantId = null;
 +                getVarChars(result.getRow(), 5, rowKeyMetaData);
 +                if (linkType==PTable.LinkType.PARENT_TABLE) {
 +                    viewTenantId = result.getValue(TABLE_FAMILY_BYTES, PARENT_TENANT_ID_BYTES);
 +                } else if (linkType==PTable.LinkType.CHILD_TABLE) {
 +                    viewTenantId = rowKeyMetaData[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX];
 +                } else if (linkType==PTable.LinkType.VIEW_INDEX_PARENT_TABLE) {
 +                    viewTenantId = rowKeyMetaData[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
 +                } 
 +                else if (linkType==PTable.LinkType.PHYSICAL_TABLE && result.getValue(TABLE_FAMILY_BYTES, TABLE_TYPE_BYTES)!=null) {
 +                    // do not links from indexes to their physical table
 +                    continue;
 +                }
 +                viewTenantId = viewTenantId==null ? new byte[]{} : viewTenantId;
 +                byte[] viewSchemaName = SchemaUtil.getSchemaNameFromFullName(rowKeyMetaData[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX]).getBytes();
 +                byte[] viewName = SchemaUtil.getTableNameFromFullName(rowKeyMetaData[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX]).getBytes();
 +                tableInfoList.add(new TableInfo(result.getRow(), viewTenantId, viewSchemaName, viewName));
 +            }
 +            return new TableViewFinderResult(tableInfoList);
 +        } finally {
 +            scanner.close();
 +        }
 +    }
 +
 +    static Graph<TableInfo> findOrphans(Table systemCatalog, long timestamp) throws IOException {
 +        Graph<TableInfo> graph = new Graph<>();
 +        Scan scan = new Scan();
 +        scan.setTimeRange(0, timestamp);
 +        FilterList list = new FilterList(FilterList.Operator.MUST_PASS_ONE);
 +        SingleColumnValueFilter childFilter =
 +            new SingleColumnValueFilter(TABLE_FAMILY_BYTES, LINK_TYPE_BYTES, CompareFilter.CompareOp.EQUAL,
 +                PTable.LinkType.CHILD_TABLE.getSerializedValueAsByteArray());
 +        childFilter.setFilterIfMissing(true);
 +        SingleColumnValueFilter parentFilter =
 +            new SingleColumnValueFilter(TABLE_FAMILY_BYTES, LINK_TYPE_BYTES, CompareFilter.CompareOp.EQUAL,
 +                PTable.LinkType.PARENT_TABLE.getSerializedValueAsByteArray());
 +        parentFilter.setFilterIfMissing(true);
 +        SingleColumnValueFilter physicalTableFilter =
 +            new SingleColumnValueFilter(TABLE_FAMILY_BYTES, LINK_TYPE_BYTES, CompareFilter.CompareOp.EQUAL,
 +                PTable.LinkType.PHYSICAL_TABLE.getSerializedValueAsByteArray());
 +        physicalTableFilter.setFilterIfMissing(true);
 +        list.addFilter(childFilter);
 +        list.addFilter(parentFilter);
 +        list.addFilter(physicalTableFilter);
 +        scan.setFilter(list);
 +        scan.addColumn(TABLE_FAMILY_BYTES, LINK_TYPE_BYTES);
 +        scan.addColumn(TABLE_FAMILY_BYTES, PARENT_TENANT_ID_BYTES);
 +        ResultScanner scanner = systemCatalog.getScanner(scan);
 +        try {
 +            for (Result result = scanner.next(); (result != null); result = scanner.next()) {
 +                ImmutableBytesWritable ptr = new ImmutableBytesWritable();
 +                ResultTuple resultTuple = new ResultTuple(result);
 +                resultTuple.getKey(ptr);
 +                byte[][] rowKeyMetaData = new byte[5][];
 +                getVarChars(result.getRow(), 5, rowKeyMetaData);
 +                byte[] tenantId = rowKeyMetaData[0];
 +                byte[] schema = rowKeyMetaData[1];
 +                byte[] tableName = rowKeyMetaData[2];
 +                byte[] link = rowKeyMetaData[4];
 +                graph.addEdge(new TableInfo(tenantId, schema, tableName), new TableInfo(HConstants.EMPTY_BYTE_ARRAY, HConstants.EMPTY_BYTE_ARRAY, link));
 +            }
 +            return graph;
 +        } finally {
 +            scanner.close();
 +        }
 +    }
 +
 +
 +    @Deprecated
 +    public static class Graph<T> {
 +        private LinkedHashMultimap<T, T> map;
 +
 +        public Graph() {
 +            map = LinkedHashMultimap.create();
 +        }
 +
 +        public void addEdge(T leftNode, T rightNode) {
 +            Set<T> adjacent = map.get(leftNode);
 +            if (adjacent == null) {
 +                adjacent = new LinkedHashSet<T>();
 +                map.putAll(leftNode, adjacent);
 +            }
 +            adjacent.add(rightNode);
 +        }
 +
 +        public void addTwoWayVertex(T leftNode, T rightNode) {
 +            addEdge(leftNode, rightNode);
 +            addEdge(rightNode, leftNode);
 +        }
 +
 +        public boolean isConnected(T leftNode, T rightNode) {
 +            Set<T> adjacent = map.get(leftNode);
 +            return adjacent != null && adjacent.contains(rightNode);
 +        }
 +
 +        public Set<T> adjacentNodes(T last) {
 +            return map.get(last);
 +        }
 +
 +        public Set<T> findAllOrphans() {
 +            Set<T> results = Sets.newHashSet();
 +            for (T t : map.keySet() ) {
 +                for (T value : map.get(t)) {
 +                    if (!map.containsKey(value)) {
 +                        results.add(t);
 +                    }
 +                }
 +            }
 +            return results;
 +        }
 +
 +        @Override
 +        public String toString() {
 +            return "Graph{" +
 +                "map=" + map +
 +                '}';
 +        }
 +    }
 +
 +}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/39c0d1d4/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixConnection.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/39c0d1d4/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/phoenix/blob/39c0d1d4/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java
----------------------------------------------------------------------
diff --cc phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java
index f860d68,2a0c8f0..3961656
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java
@@@ -65,8 -59,8 +65,9 @@@ import org.apache.phoenix.schema.PColum
  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.PTable.ViewType;
  import org.apache.phoenix.schema.PTableType;
  import org.apache.phoenix.schema.SequenceKey;
  import org.apache.phoenix.schema.SortOrder;
@@@ -300,25 -227,17 +301,35 @@@ 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 ViewType getViewType(List<Mutation> tableMetaData, KeyValueBuilder builder,
 +    	      ImmutableBytesWritable value) {
 +    	        if (getMutationValue(getPutOnlyTableHeaderRow(tableMetaData),
 +    	            PhoenixDatabaseMetaData.VIEW_TYPE_BYTES, builder, value)) {
 +    	            return ViewType.fromSerializedValue(value.get()[value.getOffset()]);
 +    	        }
 +    	        return null;
 +    	    }
 +    
 +    public static int getSaltBuckets(List<Mutation> tableMetaData, KeyValueBuilder builder,
 +      ImmutableBytesWritable value) {
 +        if (getMutationValue(getPutOnlyTableHeaderRow(tableMetaData),
 +            PhoenixDatabaseMetaData.SALT_BUCKETS_BYTES, builder, value)) {
 +            return PInteger.INSTANCE.getCodec().decodeInt(value, SortOrder.getDefault());
 +        }
 +        return 0;
 +    }
 +    
      public static long getParentSequenceNumber(List<Mutation> tableMetaData) {
          return getSequenceNumber(getParentTableHeaderRow(tableMetaData));
      }
@@@ -795,24 -682,10 +806,31 @@@
          return SchemaUtil.getTableKey(ByteUtil.EMPTY_BYTE_ARRAY, physicalTableSchemaName, physicalTableName);
      }
      
 -    public static IndexType getIndexType(List<Mutation> tableMetaData, KeyValueBuilder builder,
 +	public static List<Mutation> removeChildLinks(List<Mutation> catalogMutations) {
 +		List<Mutation> childLinks = Lists.newArrayList();
 +		Iterator<Mutation> iter = catalogMutations.iterator();
 +		while (iter.hasNext()) {
 +			Mutation m = iter.next();
 +			for (KeyValue kv : m.getFamilyMap().get(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES)) {
 +				// remove mutations of link type LinkType.CHILD_TABLE
 +				if ((Bytes.compareTo(kv.getQualifierArray(), kv.getQualifierOffset(), kv.getQualifierLength(),
 +						PhoenixDatabaseMetaData.LINK_TYPE_BYTES, 0,
 +						PhoenixDatabaseMetaData.LINK_TYPE_BYTES.length) == 0)
 +						&& ((Bytes.compareTo(kv.getValueArray(), kv.getValueOffset(), kv.getValueLength(),
 +								LinkType.CHILD_TABLE.getSerializedValueAsByteArray(), 0,
 +								LinkType.CHILD_TABLE.getSerializedValueAsByteArray().length) == 0))) {
 +					childLinks.add(m);
 +					iter.remove();
 +				}
 +			}
 +		}
 +		return childLinks;
 +	}
++
++	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/39c0d1d4/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
----------------------------------------------------------------------
diff --cc phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
index 0060993,47b4b43..30fa0ac
--- 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,12 -20,11 +20,14 @@@ 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_CHILD_LINK_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.query.QueryConstants.SEPARATOR_BYTE;
 +import static org.apache.phoenix.query.QueryConstants.SEPARATOR_BYTE_ARRAY;
+ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES;
  
  import java.sql.Connection;
  import java.sql.DatabaseMetaData;


[31/50] [abbrv] phoenix git commit: PHOENIX-4446 Sequence table region opening failing because of property setting attempt on read-only configuration-addendum(Rajeshbabu)

Posted by td...@apache.org.
PHOENIX-4446 Sequence table region opening failing because of property setting attempt on read-only configuration-addendum(Rajeshbabu)


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

Branch: refs/heads/system-catalog
Commit: 1a19d1ecbd38f2b7ee406df8efa05d29f685ef57
Parents: 334eb15
Author: Rajeshbabu Chintaguntla <ra...@apache.org>
Authored: Tue Dec 12 15:29:20 2017 +0530
Committer: Rajeshbabu Chintaguntla <ra...@apache.org>
Committed: Tue Dec 12 15:29:20 2017 +0530

----------------------------------------------------------------------
 .../wal/WALReplayWithIndexWritesAndCompressedWALIT.java            | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/1a19d1ec/phoenix-core/src/it/java/org/apache/hadoop/hbase/regionserver/wal/WALReplayWithIndexWritesAndCompressedWALIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/hadoop/hbase/regionserver/wal/WALReplayWithIndexWritesAndCompressedWALIT.java b/phoenix-core/src/it/java/org/apache/hadoop/hbase/regionserver/wal/WALReplayWithIndexWritesAndCompressedWALIT.java
index 042dd88..e2ddd4e 100644
--- a/phoenix-core/src/it/java/org/apache/hadoop/hbase/regionserver/wal/WALReplayWithIndexWritesAndCompressedWALIT.java
+++ b/phoenix-core/src/it/java/org/apache/hadoop/hbase/regionserver/wal/WALReplayWithIndexWritesAndCompressedWALIT.java
@@ -60,6 +60,7 @@ import org.apache.phoenix.hbase.index.covered.ColumnGroup;
 import org.apache.phoenix.hbase.index.covered.CoveredColumn;
 import org.apache.phoenix.hbase.index.covered.CoveredColumnIndexSpecifierBuilder;
 import org.apache.phoenix.hbase.index.util.TestIndexManagementUtil;
+import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.util.ConfigUtil;
 import org.junit.After;
 import org.junit.Before;
@@ -100,6 +101,7 @@ public class WALReplayWithIndexWritesAndCompressedWALIT {
     setupCluster();
     Path hbaseRootDir = UTIL.getDataTestDir();
     this.conf = HBaseConfiguration.create(UTIL.getConfiguration());
+    this.conf.setBoolean(QueryServices.INDEX_FAILURE_THROW_EXCEPTION_ATTRIB, false);
     this.fs = UTIL.getDFSCluster().getFileSystem();
     this.hbaseRootDir = new Path(this.conf.get(HConstants.HBASE_DIR));
     this.oldLogDir = new Path(this.hbaseRootDir, HConstants.HREGION_OLDLOGDIR_NAME);


[35/50] [abbrv] phoenix git commit: PHOENIX-4449 Bundle a copy of Argparse-1.4.0 for installations that need it

Posted by td...@apache.org.
PHOENIX-4449 Bundle a copy of Argparse-1.4.0 for installations that need it


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

Branch: refs/heads/system-catalog
Commit: 5cb02da74c15b0ae7c0fb4c880d60a2d1b6d18aa
Parents: 90c7241
Author: Josh Elser <el...@apache.org>
Authored: Mon Dec 11 19:18:25 2017 -0500
Committer: Josh Elser <el...@apache.org>
Committed: Wed Dec 13 15:39:47 2017 -0500

----------------------------------------------------------------------
 NOTICE                         |    2 +
 bin/argparse-1.4.0/argparse.py | 2392 +++++++++++++++++++++++++++++++++++
 bin/sqlline-thin.py            |    9 +-
 bin/sqlline.py                 |   14 +-
 4 files changed, 2413 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/5cb02da7/NOTICE
----------------------------------------------------------------------
diff --git a/NOTICE b/NOTICE
index eb2eef5..8b2b647 100644
--- a/NOTICE
+++ b/NOTICE
@@ -19,3 +19,5 @@ The file bin/daemon.py is based on the file of the same name in python-daemon 2.
 # Copyright © 2003 Clark Evans
 # Copyright © 2002 Noah Spurrier
 # Copyright © 2001 Jürgen Hermann
+
+The file bin/argparse-1.4.0/argparse.py is (c) 2006-2009 Steven J. Bethard <st...@gmail.com>.


[48/50] [abbrv] phoenix git commit: fix test

Posted by td...@apache.org.
fix test


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

Branch: refs/heads/system-catalog
Commit: 338c650e879604ac6670625852dd88e8a4ee04dd
Parents: cb47eae
Author: Thomas D'Silva <td...@apache.org>
Authored: Thu Dec 28 23:41:30 2017 -0800
Committer: Thomas D'Silva <td...@apache.org>
Committed: Thu Dec 28 23:42:20 2017 -0800

----------------------------------------------------------------------
 .../it/java/org/apache/phoenix/end2end/TableDDLPermissionsIT.java  | 2 --
 1 file changed, 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/338c650e/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 dc55400..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
@@ -201,8 +201,6 @@ public class TableDDLPermissionsIT extends BasePermissionsIT {
             verifyAllowed(readTable(phoenixTableName, indexName1), unprivilegedUser);
             verifyAllowed(readTable(phoenixTableName, indexName2), unprivilegedUser);
             verifyAllowed(rebuildIndex(indexName2, phoenixTableName), unprivilegedUser);
-            verifyAllowed(dropView(viewName3), regularUser);
-            verifyAllowed(dropView(viewName4), regularUser);
 
             // data table user should be able to read new index
             verifyAllowed(rebuildIndex(indexName2, phoenixTableName), regularUser1);


[19/50] [abbrv] phoenix git commit: PHOENIX-672 Add GRANT and REVOKE commands using HBase AccessController

Posted by td...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/88038a2d/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/88038a2d/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/88038a2d/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/88038a2d/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/88038a2d/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/88038a2d/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/88038a2d/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/88038a2d/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/88038a2d/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/88038a2d/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/88038a2d/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/88038a2d/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/88038a2d/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/88038a2d/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/88038a2d/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" +


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

Posted by td...@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/25359a95
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/25359a95
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/25359a95

Branch: refs/heads/system-catalog
Commit: 25359a95f1a8c5aa6e852e750fd6da6e10249387
Parents: 071fbce
Author: Thomas D'Silva <td...@apache.org>
Authored: Mon Nov 20 19:13:53 2017 -0800
Committer: Thomas D'Silva <td...@apache.org>
Committed: Wed Dec 6 12:26:15 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/25359a95/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/25359a95/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/25359a95/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/25359a95/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 b5a55b8..7462baa 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/25359a95/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/25359a95/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);


[23/50] [abbrv] phoenix git commit: PHOENIX-3050 Handle DESC columns in child/parent join optimization

Posted by td...@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/071fbce4
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/071fbce4
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/071fbce4

Branch: refs/heads/system-catalog
Commit: 071fbce499c5303ffdcd4bbe25c1cda788aced0c
Parents: d77c237
Author: maryannxue <ma...@gmail.com>
Authored: Wed Dec 6 12:07:16 2017 -0800
Committer: maryannxue <ma...@gmail.com>
Committed: Wed Dec 6 12:07:16 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/071fbce4/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/071fbce4/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/071fbce4/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/071fbce4/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; 


[39/50] [abbrv] phoenix git commit: PHOENIX-4437 Make QueryPlan.getEstimatedBytesToScan() independent of getExplainPlan() and pull optimize() out of getExplainPlan()

Posted by td...@apache.org.
PHOENIX-4437 Make QueryPlan.getEstimatedBytesToScan() independent of getExplainPlan() and pull optimize() out of getExplainPlan()


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

Branch: refs/heads/system-catalog
Commit: 412329a7415302831954891285d291055328c28b
Parents: 9355a4d
Author: maryannxue <ma...@gmail.com>
Authored: Thu Dec 21 10:31:04 2017 -0800
Committer: maryannxue <ma...@gmail.com>
Committed: Thu Dec 21 10:31:04 2017 -0800

----------------------------------------------------------------------
 .../end2end/ExplainPlanWithStatsEnabledIT.java  |  2 +-
 .../apache/phoenix/execute/BaseQueryPlan.java   | 45 ++++++--------
 .../apache/phoenix/execute/HashJoinPlan.java    | 59 +++++++++---------
 .../phoenix/execute/SortMergeJoinPlan.java      | 63 ++++++++++----------
 .../org/apache/phoenix/execute/UnionPlan.java   | 53 ++++++++--------
 .../apache/phoenix/jdbc/PhoenixStatement.java   |  9 ++-
 6 files changed, 119 insertions(+), 112 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/412329a7/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
index 49efa97..f13510b 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExplainPlanWithStatsEnabledIT.java
@@ -298,7 +298,7 @@ public class ExplainPlanWithStatsEnabledIT extends ParallelStatsEnabledIT {
         try (Connection conn = DriverManager.getConnection(getUrl())) {
             conn.setAutoCommit(false);
             Estimate info = getByteRowEstimates(conn, sql, binds);
-            assertEquals((Long) 200l, info.estimatedBytes);
+            assertEquals((Long) 176l, info.estimatedBytes);
             assertEquals((Long) 2l, info.estimatedRows);
             assertTrue(info.estimateInfoTs > 0);
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/412329a7/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 31f67b7..380037f 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
@@ -117,7 +117,7 @@ public abstract class BaseQueryPlan implements QueryPlan {
     protected Long estimatedRows;
     protected Long estimatedSize;
     protected Long estimateInfoTimestamp;
-    private boolean explainPlanCalled;
+    private boolean getEstimatesCalled;
     
 
     protected BaseQueryPlan(
@@ -498,32 +498,17 @@ public abstract class BaseQueryPlan implements QueryPlan {
 
     @Override
     public ExplainPlan getExplainPlan() throws SQLException {
-        explainPlanCalled = true;
         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();
-        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;
+        ResultIterator iterator = iterator();
+        ExplainPlan explainPlan = new ExplainPlan(getPlanSteps(iterator));
+        iterator.close();
+        return explainPlan;
     }
 
-    private List<String> getPlanSteps(ResultIterator iterator){
+    private List<String> getPlanSteps(ResultIterator iterator) {
         List<String> planSteps = Lists.newArrayListWithExpectedSize(5);
         iterator.explain(planSteps);
         return planSteps;
@@ -536,26 +521,32 @@ public abstract class BaseQueryPlan implements QueryPlan {
     
     @Override
     public Long getEstimatedRowsToScan() throws SQLException {
-        if (!explainPlanCalled) {
-            getExplainPlan();
+        if (!getEstimatesCalled) {
+            getEstimates();
         }
         return estimatedRows;
     }
 
     @Override
     public Long getEstimatedBytesToScan() throws SQLException {
-        if (!explainPlanCalled) {
-            getExplainPlan();
+        if (!getEstimatesCalled) {
+            getEstimates();
         }
         return estimatedSize;
     }
 
     @Override
     public Long getEstimateInfoTimestamp() throws SQLException {
-        if (!explainPlanCalled) {
-            getExplainPlan();
+        if (!getEstimatesCalled) {
+            getEstimates();
         }
         return estimateInfoTimestamp;
     }
 
+    private void getEstimates() throws SQLException {
+        getEstimatesCalled = true;
+        // Initialize a dummy iterator to get the estimates based on stats.
+        ResultIterator iterator = iterator();
+        iterator.close();
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/412329a7/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 2d2ff4e..23a0da6 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
@@ -99,7 +99,7 @@ public class HashJoinPlan extends DelegateQueryPlan {
     private Long estimatedRows;
     private Long estimatedBytes;
     private Long estimateInfoTs;
-    private boolean explainPlanCalled;
+    private boolean getEstimatesCalled;
     
     public static HashJoinPlan create(SelectStatement statement, 
             QueryPlan plan, HashJoinInfo joinInfo, SubPlan[] subPlans) throws SQLException {
@@ -247,7 +247,6 @@ public class HashJoinPlan extends DelegateQueryPlan {
 
     @Override
     public ExplainPlan getExplainPlan() throws SQLException {
-        explainPlanCalled = true;
         List<String> planSteps = Lists.newArrayList(delegate.getExplainPlan().getPlanSteps());
         int count = subPlans.length;
         for (int i = 0; i < count; i++) {
@@ -263,26 +262,6 @@ public class HashJoinPlan extends DelegateQueryPlan {
         if (joinInfo != null && joinInfo.getLimit() != null) {
             planSteps.add("    JOIN-SCANNER " + joinInfo.getLimit() + " ROW LIMIT");
         }
-        for (SubPlan subPlan : subPlans) {
-            if (subPlan.getInnerPlan().getEstimatedBytesToScan() == null
-                    || subPlan.getInnerPlan().getEstimatedRowsToScan() == null
-                    || subPlan.getInnerPlan().getEstimateInfoTimestamp() == null) {
-                /*
-                 * If any of the sub plans doesn't have the estimate info available, then we don't
-                 * provide estimate for the overall plan
-                 */
-                estimatedBytes = null;
-                estimatedRows = null;
-                estimateInfoTs = null;
-                break;
-            } else {
-                estimatedBytes =
-                        add(estimatedBytes, subPlan.getInnerPlan().getEstimatedBytesToScan());
-                estimatedRows = add(estimatedRows, subPlan.getInnerPlan().getEstimatedRowsToScan());
-                estimateInfoTs =
-                        getMin(estimateInfoTs, subPlan.getInnerPlan().getEstimateInfoTimestamp());
-            }
-        }
         return new ExplainPlan(planSteps);
     }
 
@@ -520,27 +499,51 @@ public class HashJoinPlan extends DelegateQueryPlan {
 
     @Override
     public Long getEstimatedRowsToScan() throws SQLException {
-        if (!explainPlanCalled) {
-            getExplainPlan();
+        if (!getEstimatesCalled) {
+            getEstimates();
         }
         return estimatedRows;
     }
 
     @Override
     public Long getEstimatedBytesToScan() throws SQLException {
-        if (!explainPlanCalled) {
-            getExplainPlan();
+        if (!getEstimatesCalled) {
+            getEstimates();
         }
         return estimatedBytes;
     }
 
     @Override
     public Long getEstimateInfoTimestamp() throws SQLException {
-        if (!explainPlanCalled) {
-            getExplainPlan();
+        if (!getEstimatesCalled) {
+            getEstimates();
         }
         return estimateInfoTs;
     }
+
+    private void getEstimates() throws SQLException {
+        getEstimatesCalled = true;
+        for (SubPlan subPlan : subPlans) {
+            if (subPlan.getInnerPlan().getEstimatedBytesToScan() == null
+                    || subPlan.getInnerPlan().getEstimatedRowsToScan() == null
+                    || subPlan.getInnerPlan().getEstimateInfoTimestamp() == null) {
+                /*
+                 * If any of the sub plans doesn't have the estimate info available, then we don't
+                 * provide estimate for the overall plan
+                 */
+                estimatedBytes = null;
+                estimatedRows = null;
+                estimateInfoTs = null;
+                break;
+            } else {
+                estimatedBytes =
+                        add(estimatedBytes, subPlan.getInnerPlan().getEstimatedBytesToScan());
+                estimatedRows = add(estimatedRows, subPlan.getInnerPlan().getEstimatedRowsToScan());
+                estimateInfoTs =
+                        getMin(estimateInfoTs, subPlan.getInnerPlan().getEstimateInfoTimestamp());
+            }
+        }
+    }
 }
 
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/412329a7/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 3e380da..2436d1e 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
@@ -95,7 +95,7 @@ public class SortMergeJoinPlan implements QueryPlan {
     private Long estimatedBytes;
     private Long estimatedRows;
     private Long estimateInfoTs;
-    private boolean explainPlanCalled;
+    private boolean getEstimatesCalled;
 
     public SortMergeJoinPlan(StatementContext context, FilterableStatement statement, TableRef table, 
             JoinType type, QueryPlan lhsPlan, QueryPlan rhsPlan, List<Expression> lhsKeyExpressions, List<Expression> rhsKeyExpressions,
@@ -157,7 +157,6 @@ public class SortMergeJoinPlan implements QueryPlan {
 
     @Override
     public ExplainPlan getExplainPlan() throws SQLException {
-        explainPlanCalled = true;
         List<String> steps = Lists.newArrayList();
         steps.add("SORT-MERGE-JOIN (" + type.toString().toUpperCase() + ") TABLES");
         for (String step : lhsPlan.getExplainPlan().getPlanSteps()) {
@@ -167,28 +166,6 @@ public class SortMergeJoinPlan implements QueryPlan {
         for (String step : rhsPlan.getExplainPlan().getPlanSteps()) {
             steps.add("    " + step);            
         }
-        if ((lhsPlan.getEstimatedBytesToScan() == null || rhsPlan.getEstimatedBytesToScan() == null)
-                || (lhsPlan.getEstimatedRowsToScan() == null
-                        || rhsPlan.getEstimatedRowsToScan() == null)
-                || (lhsPlan.getEstimateInfoTimestamp() == null
-                        || rhsPlan.getEstimateInfoTimestamp() == null)) {
-            /*
-             * If any of the sub plans doesn't have the estimate info available, then we don't
-             * provide estimate for the overall plan
-             */
-            estimatedBytes = null;
-            estimatedRows = null;
-            estimateInfoTs = null;
-        } else {
-            estimatedBytes =
-                    add(add(estimatedBytes, lhsPlan.getEstimatedBytesToScan()),
-                        rhsPlan.getEstimatedBytesToScan());
-            estimatedRows =
-                    add(add(estimatedRows, lhsPlan.getEstimatedRowsToScan()),
-                        rhsPlan.getEstimatedRowsToScan());
-            estimateInfoTs =
-                    getMin(lhsPlan.getEstimateInfoTimestamp(), rhsPlan.getEstimateInfoTimestamp());
-        }
         return new ExplainPlan(steps);
     }
 
@@ -754,25 +731,51 @@ public class SortMergeJoinPlan implements QueryPlan {
 
     @Override
     public Long getEstimatedRowsToScan() throws SQLException {
-        if (!explainPlanCalled) {
-            getExplainPlan();
+        if (!getEstimatesCalled) {
+            getEstimates();
         }
         return estimatedRows;
     }
 
     @Override
     public Long getEstimatedBytesToScan() throws SQLException {
-        if (!explainPlanCalled) {
-            getExplainPlan();
+        if (!getEstimatesCalled) {
+            getEstimates();
         }
         return estimatedBytes;
     }
 
     @Override
     public Long getEstimateInfoTimestamp() throws SQLException {
-        if (!explainPlanCalled) {
-            getExplainPlan();
+        if (!getEstimatesCalled) {
+            getEstimates();
         }
         return estimateInfoTs;
     }
+
+    private void getEstimates() throws SQLException {
+        getEstimatesCalled = true;
+        if ((lhsPlan.getEstimatedBytesToScan() == null || rhsPlan.getEstimatedBytesToScan() == null)
+                || (lhsPlan.getEstimatedRowsToScan() == null
+                || rhsPlan.getEstimatedRowsToScan() == null)
+                || (lhsPlan.getEstimateInfoTimestamp() == null
+                || rhsPlan.getEstimateInfoTimestamp() == null)) {
+            /*
+             * If any of the sub plans doesn't have the estimate info available, then we don't
+             * provide estimate for the overall plan
+             */
+            estimatedBytes = null;
+            estimatedRows = null;
+            estimateInfoTs = null;
+        } else {
+            estimatedBytes =
+                    add(add(estimatedBytes, lhsPlan.getEstimatedBytesToScan()),
+                            rhsPlan.getEstimatedBytesToScan());
+            estimatedRows =
+                    add(add(estimatedRows, lhsPlan.getEstimatedRowsToScan()),
+                            rhsPlan.getEstimatedRowsToScan());
+            estimateInfoTs =
+                    getMin(lhsPlan.getEstimateInfoTimestamp(), rhsPlan.getEstimateInfoTimestamp());
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/412329a7/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 e6bf654..3b5168c 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
@@ -69,7 +69,7 @@ public class UnionPlan implements QueryPlan {
     private Long estimatedRows;
     private Long estimatedBytes;
     private Long estimateInfoTs;
-    private boolean explainPlanCalled;
+    private boolean getEstimatesCalled;
 
     public UnionPlan(StatementContext context, FilterableStatement statement, TableRef table, RowProjector projector,
             Integer limit, Integer offset, OrderBy orderBy, GroupBy groupBy, List<QueryPlan> plans, ParameterMetaData paramMetaData) throws SQLException {
@@ -174,7 +174,6 @@ public class UnionPlan implements QueryPlan {
 
     @Override
     public ExplainPlan getExplainPlan() throws SQLException {
-        explainPlanCalled = true;
         List<String> steps = new ArrayList<String>();
         steps.add("UNION ALL OVER " + this.plans.size() + " QUERIES");
         ResultIterator iterator = iterator();
@@ -184,23 +183,6 @@ public class UnionPlan implements QueryPlan {
         for (int i = 1 ; i < steps.size()-offset; i++) {
             steps.set(i, "    " + steps.get(i));
         }
-        for (QueryPlan plan : plans) {
-            if (plan.getEstimatedBytesToScan() == null || plan.getEstimatedRowsToScan() == null
-                    || plan.getEstimateInfoTimestamp() == null) {
-                /*
-                 * If any of the sub plans doesn't have the estimate info available, then we don't
-                 * provide estimate for the overall plan
-                 */
-                estimatedBytes = null;
-                estimatedRows = null;
-                estimateInfoTs = null;
-                break;
-            } else {
-                estimatedBytes = add(estimatedBytes, plan.getEstimatedBytesToScan());
-                estimatedRows = add(estimatedRows, plan.getEstimatedRowsToScan());
-                estimateInfoTs = getMin(estimateInfoTs, plan.getEstimateInfoTimestamp());
-            }
-        }
         return new ExplainPlan(steps);
     }
 
@@ -265,25 +247,46 @@ public class UnionPlan implements QueryPlan {
 
     @Override
     public Long getEstimatedRowsToScan() throws SQLException {
-        if (!explainPlanCalled) {
-            getExplainPlan();
+        if (!getEstimatesCalled) {
+            getEstimates();
         }
         return estimatedRows;
     }
 
     @Override
     public Long getEstimatedBytesToScan() throws SQLException {
-        if (!explainPlanCalled) {
-            getExplainPlan();
+        if (!getEstimatesCalled) {
+            getEstimates();
         }
         return estimatedBytes;
     }
 
     @Override
     public Long getEstimateInfoTimestamp() throws SQLException {
-        if (!explainPlanCalled) {
-            getExplainPlan();
+        if (!getEstimatesCalled) {
+            getEstimates();
         }
         return estimateInfoTs;
     }
+
+    private void getEstimates() throws SQLException {
+        getEstimatesCalled = true;
+        for (QueryPlan plan : plans) {
+            if (plan.getEstimatedBytesToScan() == null || plan.getEstimatedRowsToScan() == null
+                    || plan.getEstimateInfoTimestamp() == null) {
+                /*
+                 * If any of the sub plans doesn't have the estimate info available, then we don't
+                 * provide estimate for the overall plan
+                 */
+                estimatedBytes = null;
+                estimatedRows = null;
+                estimateInfoTs = null;
+                break;
+            } else {
+                estimatedBytes = add(estimatedBytes, plan.getEstimatedBytesToScan());
+                estimatedRows = add(estimatedRows, plan.getEstimatedRowsToScan());
+                estimateInfoTs = getMin(estimateInfoTs, plan.getEstimateInfoTimestamp());
+            }
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/412329a7/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 c699088..b637173 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
@@ -581,7 +581,14 @@ public class PhoenixStatement implements Statement, SQLCloseable {
         @Override
         public QueryPlan compilePlan(PhoenixStatement stmt, Sequence.ValueOp seqAction) throws SQLException {
             CompilableStatement compilableStmt = getStatement();
-            final StatementPlan plan = compilableStmt.compilePlan(stmt, Sequence.ValueOp.VALIDATE_SEQUENCE);
+            StatementPlan compilePlan = compilableStmt.compilePlan(stmt, Sequence.ValueOp.VALIDATE_SEQUENCE);
+            // For a QueryPlan, we need to get its optimized plan; for a MutationPlan, its enclosed QueryPlan
+            // has already been optimized during compilation.
+            if (compilePlan instanceof QueryPlan) {
+                QueryPlan dataPlan = (QueryPlan) compilePlan;
+                compilePlan = stmt.getConnection().getQueryServices().getOptimizer().optimize(stmt, dataPlan);
+            }
+            final StatementPlan plan = compilePlan;
             List<String> planSteps = plan.getExplainPlan().getPlanSteps();
             List<Tuple> tuples = Lists.newArrayListWithExpectedSize(planSteps.size());
             Long estimatedBytesToScan = plan.getEstimatedBytesToScan();


[15/50] [abbrv] phoenix git commit: PHOENIX-4360 Prevent System.Catalog from splitting.

Posted by td...@apache.org.
PHOENIX-4360 Prevent System.Catalog from splitting.


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

Branch: refs/heads/system-catalog
Commit: c216b667a8da568f768c0d26f46fa1a9c0994a04
Parents: ca1e17b
Author: Lars Hofhansl <la...@apache.org>
Authored: Mon Nov 20 14:08:44 2017 -0800
Committer: Lars Hofhansl <la...@apache.org>
Committed: Mon Nov 20 14:08:44 2017 -0800

----------------------------------------------------------------------
 .../apache/phoenix/end2end/SystemCatalogIT.java | 79 ++++++++++++++++++++
 .../phoenix/schema/MetaDataSplitPolicy.java     | 23 +-----
 2 files changed, 82 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/c216b667/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
new file mode 100644
index 0000000..8a9bca2
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SystemCatalogIT.java
@@ -0,0 +1,79 @@
+package org.apache.phoenix.end2end;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Properties;
+
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.RegionLocator;
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.junit.After;
+import org.junit.Test;
+
+public class SystemCatalogIT {
+    private HBaseTestingUtility testUtil = null;
+
+    @After
+    public void cleanup() throws Exception {
+        if (null != testUtil) {
+          testUtil.shutdownMiniCluster();
+          testUtil = null;
+        }
+    }
+
+    /**
+     * Make sure that SYSTEM.CATALOG cannot be split, even with schemas and multi-tenant views
+     */
+    @Test
+    public void testSystemTableSplit() throws Exception {
+        testUtil = new HBaseTestingUtility();
+        testUtil.startMiniCluster(1);
+        for (int i=0; i<10; i++) {
+            createTable("schema"+i+".table_"+i);
+        }
+        TableName systemCatalog = TableName.valueOf("SYSTEM.CATALOG");
+        RegionLocator rl = testUtil.getConnection().getRegionLocator(systemCatalog);
+        assertEquals(rl.getAllRegionLocations().size(), 1);
+
+        // now attempt to split SYSTEM.CATALOG
+        testUtil.getHBaseAdmin().split(systemCatalog);
+
+        // make sure the split finishes (there's no synchronous splitting before HBase 2.x)
+        testUtil.getHBaseAdmin().disableTable(systemCatalog);
+        testUtil.getHBaseAdmin().enableTable(systemCatalog);
+
+        // test again... Must still be exactly one region.
+        rl = testUtil.getConnection().getRegionLocator(systemCatalog);
+        assertEquals(1, rl.getAllRegionLocations().size());
+    }
+
+    private void createTable(String tableName) throws Exception {
+        try (Connection conn = DriverManager.getConnection(getJdbcUrl());
+            Statement stmt = conn.createStatement();) {
+            stmt.execute("DROP TABLE IF EXISTS " + tableName);
+            stmt.execute("CREATE TABLE " + tableName
+                + " (TENANT_ID VARCHAR NOT NULL, PK1 VARCHAR NOT NULL, V1 VARCHAR CONSTRAINT PK PRIMARY KEY(TENANT_ID, PK1)) MULTI_TENANT=true");
+            try (Connection tenant1Conn = getTenantConnection("tenant1")) {
+                String view1DDL = "CREATE VIEW " + tableName + "_view AS SELECT * FROM " + tableName;
+                tenant1Conn.createStatement().execute(view1DDL);
+            }
+            conn.commit();
+        }
+    }
+
+    private String getJdbcUrl() {
+        return "jdbc:phoenix:localhost:" + testUtil.getZkCluster().getClientPort() + ":/hbase";
+    }
+
+    private Connection getTenantConnection(String tenantId) throws SQLException {
+        Properties tenantProps = new Properties();
+        tenantProps.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, tenantId);
+        return DriverManager.getConnection(getJdbcUrl(), tenantProps);
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/c216b667/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataSplitPolicy.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataSplitPolicy.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataSplitPolicy.java
index 90fb8fb..154a9c2 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataSplitPolicy.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataSplitPolicy.java
@@ -25,25 +25,8 @@ import org.apache.phoenix.util.SchemaUtil;
 public class MetaDataSplitPolicy extends ConstantSizeRegionSplitPolicy {
 
     @Override
-    protected byte[] getSplitPoint() {
-        byte[] splitPoint = super.getSplitPoint();
-        int offset = SchemaUtil.getVarCharLength(splitPoint, 0, splitPoint.length);
-        // Split only on Phoenix schema name, so this is ok b/c we won't be splitting
-        // in the middle of a Phoenix table.
-        if (offset == splitPoint.length) {
-            return splitPoint;
-        }
-//        offset = SchemaUtil.getVarCharLength(splitPoint, offset+1, splitPoint.length-offset-1);
-//        // Split only on Phoenix schema and table name, so this is ok b/c we won't be splitting
-//        // in the middle of a Phoenix table.
-//        if (offset == splitPoint.length) {
-//            return splitPoint;
-//        }
-        // Otherwise, an attempt is being made to split in the middle of a table.
-        // Just return a split point at the schema boundary instead
-        byte[] newSplitPoint = new byte[offset + 1];
-        System.arraycopy(splitPoint, 0, newSplitPoint, 0, offset+1);
-        return newSplitPoint;
+    protected boolean shouldSplit() {
+        // never split SYSTEM.CATALOG
+        return false;
     }
-
 }


[21/50] [abbrv] phoenix git commit: PHOENIX-4288 Indexes not used when ordering by primary key

Posted by td...@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/6807dacc
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/6807dacc
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/6807dacc

Branch: refs/heads/system-catalog
Commit: 6807dacce7d063e14f06bc57888e7d2a5f78863a
Parents: 88038a2
Author: maryannxue <ma...@gmail.com>
Authored: Tue Dec 5 10:52:46 2017 -0800
Committer: maryannxue <ma...@gmail.com>
Committed: Tue Dec 5 10:52:46 2017 -0800

----------------------------------------------------------------------
 .../phoenix/end2end/CostBasedDecisionIT.java    | 466 +++++++++++++++++++
 .../apache/phoenix/end2end/MutationStateIT.java |  17 +
 .../apache/phoenix/end2end/SystemCatalogIT.java |  17 +
 .../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, 988 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/6807dacc/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/6807dacc/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/6807dacc/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 8a9bca2..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
@@ -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/6807dacc/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/6807dacc/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/6807dacc/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/6807dacc/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/6807dacc/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/6807dacc/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/6807dacc/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/6807dacc/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/6807dacc/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/6807dacc/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/6807dacc/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/6807dacc/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/6807dacc/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/6807dacc/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/6807dacc/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/6807dacc/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/6807dacc/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/6807dacc/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/6807dacc/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/6807dacc/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/6807dacc/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();


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

Posted by td...@apache.org.
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/c075a178
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/c075a178
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/c075a178

Branch: refs/heads/system-catalog
Commit: c075a17879309fe4def5a621951d72929fb10c3a
Parents: 72bc819
Author: Karan Mehta <ka...@gmail.com>
Authored: Mon Dec 4 10:44:13 2017 -0800
Committer: Thomas D'Silva <td...@apache.org>
Committed: Fri Dec 8 14:52:33 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/c075a178/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/c075a178/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/c075a178/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/c075a178/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/c075a178/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/c075a178/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/c075a178/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/c075a178/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'";


[13/50] [abbrv] phoenix git commit: PHOENIX-4384 Phoenix server jar doesn't include icu4j jars (Shehzaad Nakhoda)

Posted by td...@apache.org.
PHOENIX-4384 Phoenix server jar doesn't include icu4j jars (Shehzaad Nakhoda)


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

Branch: refs/heads/system-catalog
Commit: 03fc3314219f364bf91dd09921e51ecf94a41aff
Parents: 4944081
Author: James Taylor <jt...@salesforce.com>
Authored: Sat Nov 18 11:31:46 2017 -0800
Committer: James Taylor <jt...@salesforce.com>
Committed: Sat Nov 18 11:31:46 2017 -0800

----------------------------------------------------------------------
 phoenix-core/pom.xml   | 1 +
 phoenix-server/pom.xml | 3 +++
 pom.xml                | 3 ++-
 3 files changed, 6 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/03fc3314/phoenix-core/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-core/pom.xml b/phoenix-core/pom.xml
index fbda221..c90946a 100644
--- a/phoenix-core/pom.xml
+++ b/phoenix-core/pom.xml
@@ -474,6 +474,7 @@
     <dependency>
       <groupId>com.salesforce.i18n</groupId>
       <artifactId>i18n-util</artifactId>
+      <version>${i18n-util.version}</version>
     </dependency>
   </dependencies>
 </project>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/03fc3314/phoenix-server/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-server/pom.xml b/phoenix-server/pom.xml
index e67927e..e0baeb1 100644
--- a/phoenix-server/pom.xml
+++ b/phoenix-server/pom.xml
@@ -135,6 +135,9 @@
                   <include>org.apache.thrift:libthrift</include>
                   <include>com.clearspring.analytics:stream</include>
                   <include>com.salesforce.i18n:i18n-util</include>
+                  <include>com.ibm.icu:icu4j</include>
+                  <include>com.ibm.icu:icu4j-charset</include>
+                  <include>com.ibm.icu:icu4j-localespi</include>
                 </includes>
                   <excludes>
                     <exclude>org.apache.phoenix:phoenix-server</exclude>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/03fc3314/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 2327c80..7b6b052 100644
--- a/pom.xml
+++ b/pom.xml
@@ -105,6 +105,7 @@
     <scala.version>2.11.8</scala.version>
     <scala.binary.version>2.11</scala.binary.version>
     <stream.version>2.9.5</stream.version>
+    <i18n-util.version>1.0.1</i18n-util.version>
     <!-- Test Dependencies -->
     <mockito-all.version>1.8.5</mockito-all.version>
     <junit.version>4.12</junit.version>
@@ -930,7 +931,7 @@
       <dependency>
         <groupId>com.salesforce.i18n</groupId>
         <artifactId>i18n-util</artifactId>
-        <version>1.0.1</version>
+        <version>${i18n-util.version}</version>
       </dependency>
     </dependencies>
   </dependencyManagement>


[43/50] [abbrv] phoenix git commit: Immutable table SINGLE_CELL_ARRAY_WITH_OFFSETS values starting with separator byte return null in query results

Posted by td...@apache.org.
Immutable table SINGLE_CELL_ARRAY_WITH_OFFSETS values starting with separator byte return null in query results


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

Branch: refs/heads/system-catalog
Commit: edc9d12dc056640526e388d7ccb1a6e2c6d3c51c
Parents: 2759727
Author: Vincent Poon <vi...@apache.org>
Authored: Wed Dec 27 15:34:32 2017 -0800
Committer: Vincent Poon <vi...@apache.org>
Committed: Wed Dec 27 15:34:32 2017 -0800

----------------------------------------------------------------------
 .../apache/phoenix/end2end/StoreNullsIT.java    |   6 +-
 .../phoenix/end2end/UpsertBigValuesIT.java      |  64 ++++-
 .../phoenix/end2end/index/DropColumnIT.java     |  19 +-
 .../phoenix/compile/ProjectionCompiler.java     |   6 +-
 .../expression/SingleCellColumnExpression.java  |  35 ++-
 .../apache/phoenix/index/IndexMaintainer.java   |   2 +-
 .../NonAggregateRegionScannerFactory.java       |   2 +-
 .../org/apache/phoenix/schema/ColumnRef.java    |   4 +-
 .../java/org/apache/phoenix/schema/PTable.java  |  21 +-
 .../phoenix/schema/types/PArrayDataType.java    |  18 +-
 .../schema/types/PArrayDataTypeDecoder.java     |  79 +++++-
 .../schema/types/PArrayDataTypeEncoder.java     |  10 +-
 .../java/org/apache/phoenix/util/IndexUtil.java |   3 +-
 .../schema/ImmutableStorageSchemeTest.java      | 241 +++++++++++++++++--
 14 files changed, 452 insertions(+), 58 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/edc9d12d/phoenix-core/src/it/java/org/apache/phoenix/end2end/StoreNullsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/StoreNullsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/StoreNullsIT.java
index 378a9ed..d1aee9f 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/StoreNullsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/StoreNullsIT.java
@@ -145,7 +145,11 @@ public class StoreNullsIT extends ParallelStatsDisabledIT {
         byte[] qualifier = table.getImmutableStorageScheme()== ImmutableStorageScheme.SINGLE_CELL_ARRAY_WITH_OFFSETS ? QueryConstants.SINGLE_KEYVALUE_COLUMN_QUALIFIER_BYTES : nameColumn.getColumnQualifierBytes();
         assertTrue(rs.containsColumn(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, qualifier));
         assertTrue(rs.size() == 2); // 2 because it also includes the empty key value column
-        KeyValueColumnExpression colExpression = table.getImmutableStorageScheme() == ImmutableStorageScheme.SINGLE_CELL_ARRAY_WITH_OFFSETS ? new SingleCellColumnExpression(nameColumn, "NAME", table.getEncodingScheme()) : new KeyValueColumnExpression(nameColumn);
+        KeyValueColumnExpression colExpression =
+                table.getImmutableStorageScheme() == ImmutableStorageScheme.SINGLE_CELL_ARRAY_WITH_OFFSETS
+                        ? new SingleCellColumnExpression(nameColumn, "NAME",
+                                table.getEncodingScheme(), table.getImmutableStorageScheme())
+                        : new KeyValueColumnExpression(nameColumn);
         ImmutableBytesPtr ptr = new ImmutableBytesPtr();
         colExpression.evaluate(new ResultTuple(rs), ptr);
         assertEquals(new ImmutableBytesPtr(PVarchar.INSTANCE.toBytes("v1")), ptr);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/edc9d12d/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertBigValuesIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertBigValuesIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertBigValuesIT.java
index ceb76d0..9842434 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertBigValuesIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertBigValuesIT.java
@@ -25,10 +25,16 @@ import java.sql.Connection;
 import java.sql.DriverManager;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
+import java.util.Arrays;
+import java.util.List;
 import java.util.Properties;
 
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PLong;
+import org.apache.phoenix.schema.types.PSmallint;
 import org.junit.Test;
 
+import com.google.common.collect.Lists;
 
 public class UpsertBigValuesIT extends ParallelStatsDisabledIT {
 
@@ -374,4 +380,60 @@ public class UpsertBigValuesIT extends ParallelStatsDisabledIT {
         */
         conn.close();
     }
-}
+
+    @Test
+    public void testShort() throws Exception {
+        List<Short> testData =
+                Arrays.asList(Short.MIN_VALUE, Short.MAX_VALUE, (short) (Short.MIN_VALUE + 1),
+                    (short) (Short.MAX_VALUE - 1), (short) 0, (short) 1, (short) -1);
+        testValues(false, PSmallint.INSTANCE, testData);
+        testValues(true, PSmallint.INSTANCE, testData);
+    }
+
+    @Test
+    public void testBigInt() throws Exception {
+        List<Long> testData =
+                Arrays.asList(Long.MIN_VALUE, Long.MAX_VALUE, Long.MIN_VALUE + 1L,
+                    Long.MAX_VALUE - 1L, 0L, 1L, -1L);
+        testValues(false, PLong.INSTANCE, testData);
+        testValues(true, PLong.INSTANCE, testData);
+    }
+
+    private <T extends Number> void testValues(boolean immutable, PDataType<?> dataType, List<T> testData) throws Exception {
+        String tableName = generateUniqueName();
+        String ddl =
+                String.format("CREATE %s TABLE %s (K INTEGER PRIMARY KEY, V1 %s)",
+                    immutable ? "IMMUTABLE" : "", tableName, dataType.getSqlTypeName());
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            conn.createStatement().execute(ddl);
+            String upsert = "UPSERT INTO " + tableName + " VALUES(?, ?)";
+            PreparedStatement stmt = conn.prepareStatement(upsert);
+            int id = 1;
+            for (T testVal : testData) {
+                stmt.setInt(1, id++);
+                stmt.setObject(2, testVal, dataType.getSqlType());
+                stmt.execute();
+            }
+            conn.commit();
+            String query = String.format("SELECT K,V1 FROM %s ORDER BY K ASC", tableName);
+            ResultSet rs = conn.createStatement().executeQuery(query);
+            int index = 0;
+            boolean failed = false;
+            List<String> errors = Lists.newArrayList();
+            while (rs.next()) {
+                Number resultVal = rs.getObject(2, testData.get(0).getClass());
+                T testVal = testData.get(index++);
+                if (!testVal.equals(resultVal)) {
+                    errors.add(String.format("[expected=%s actual=%s] ",
+                        testVal, resultVal));
+                    failed = true;
+                }
+            }
+            String errorMsg =
+                    String.format("Data in table didn't match input: immutable=%s, dataType=%s, %s",
+                        immutable, dataType.getSqlTypeName(), errors);
+            assertFalse(errorMsg, failed);
+        }
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/edc9d12d/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/DropColumnIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/DropColumnIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/DropColumnIT.java
index badb2a6..28aa1e9 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/DropColumnIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/DropColumnIT.java
@@ -254,7 +254,9 @@ public class DropColumnIT extends ParallelStatsDisabledIT {
         assertNotNull(result);
         byte[] colValue;
         if (!mutable && columnEncoded) {
-            KeyValueColumnExpression colExpression = new SingleCellColumnExpression(dataColumn, "V2", dataTable.getEncodingScheme());
+            KeyValueColumnExpression colExpression =
+                    new SingleCellColumnExpression(dataColumn, "V2", dataTable.getEncodingScheme(),
+                            dataTable.getImmutableStorageScheme());
             ImmutableBytesPtr ptr = new ImmutableBytesPtr();
             colExpression.evaluate(new ResultTuple(result), ptr);
             colValue = ptr.copyBytesIfNecessary();
@@ -273,7 +275,10 @@ public class DropColumnIT extends ParallelStatsDisabledIT {
         result = results.next();
         assertNotNull(result);
         if (!mutable && columnEncoded) {
-            KeyValueColumnExpression colExpression = new SingleCellColumnExpression(glovalIndexCol, "0:V2", globalIndexTable.getEncodingScheme());
+            KeyValueColumnExpression colExpression =
+                    new SingleCellColumnExpression(glovalIndexCol, "0:V2",
+                            globalIndexTable.getEncodingScheme(),
+                            globalIndexTable.getImmutableStorageScheme());
             ImmutableBytesPtr ptr = new ImmutableBytesPtr();
             colExpression.evaluate(new ResultTuple(result), ptr);
             colValue = ptr.copyBytesIfNecessary();
@@ -293,7 +298,10 @@ public class DropColumnIT extends ParallelStatsDisabledIT {
         result = results.next();
         assertNotNull(result);
         if (!mutable && columnEncoded) {
-            KeyValueColumnExpression colExpression = new SingleCellColumnExpression(localIndexCol, "0:V2", localIndexTable.getEncodingScheme());
+            KeyValueColumnExpression colExpression =
+                    new SingleCellColumnExpression(localIndexCol, "0:V2",
+                            localIndexTable.getEncodingScheme(),
+                            localIndexTable.getImmutableStorageScheme());
             ImmutableBytesPtr ptr = new ImmutableBytesPtr();
             assertTrue(colExpression.evaluate(new ResultTuple(result), ptr));
             colValue = ptr.copyBytesIfNecessary();
@@ -387,7 +395,10 @@ public class DropColumnIT extends ParallelStatsDisabledIT {
             PColumn localIndexCol = localIndex2.getColumnForColumnName(indexColumnName);
             byte[] colValue;
             if (!mutable && columnEncoded) {
-                KeyValueColumnExpression colExpression = new SingleCellColumnExpression(localIndexCol, indexColumnName, localIndex2.getEncodingScheme());
+                KeyValueColumnExpression colExpression =
+                        new SingleCellColumnExpression(localIndexCol, indexColumnName,
+                                localIndex2.getEncodingScheme(),
+                                localIndex2.getImmutableStorageScheme());
                 ImmutableBytesPtr ptr = new ImmutableBytesPtr();
                 colExpression.evaluate(new ResultTuple(result), ptr);
                 colValue = ptr.copyBytesIfNecessary();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/edc9d12d/phoenix-core/src/main/java/org/apache/phoenix/compile/ProjectionCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/ProjectionCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/ProjectionCompiler.java
index a147882..f85b5a8 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/ProjectionCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/ProjectionCompiler.java
@@ -668,7 +668,11 @@ public class ProjectionCompiler {
                              PTable table = context.getCurrentTable().getTable();
                              KeyValueColumnExpression keyValueColumnExpression;
                              if (table.getImmutableStorageScheme() != ImmutableStorageScheme.ONE_CELL_PER_COLUMN) {
-                                 keyValueColumnExpression = new SingleCellColumnExpression(col, col.getName().getString(), table.getEncodingScheme());
+                                keyValueColumnExpression =
+                                        new SingleCellColumnExpression(col,
+                                                col.getName().getString(),
+                                                table.getEncodingScheme(),
+                                                table.getImmutableStorageScheme());
                              } else {
                                  keyValueColumnExpression = new KeyValueColumnExpression(col);
                              }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/edc9d12d/phoenix-core/src/main/java/org/apache/phoenix/expression/SingleCellColumnExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/SingleCellColumnExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/SingleCellColumnExpression.java
index 8c1e0b6..e46e1fa 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/SingleCellColumnExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/SingleCellColumnExpression.java
@@ -25,7 +25,6 @@ import java.io.DataOutput;
 import java.io.IOException;
 
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.phoenix.compile.CreateTableCompiler.ViewWhereExpressionVisitor;
 import org.apache.phoenix.expression.visitor.ExpressionVisitor;
@@ -54,12 +53,19 @@ public class SingleCellColumnExpression extends KeyValueColumnExpression {
     private String arrayColDisplayName;
     private KeyValueColumnExpression keyValueColumnExpression;
     private QualifierEncodingScheme encodingScheme;
-    
+    private ImmutableStorageScheme immutableStorageScheme;
+
     public SingleCellColumnExpression() {
     }
-    
-    public SingleCellColumnExpression(PDatum column, byte[] cf, byte[] cq, QualifierEncodingScheme encodingScheme) {
+
+    public SingleCellColumnExpression(ImmutableStorageScheme immutableStorageScheme) {
+        this.immutableStorageScheme = immutableStorageScheme;
+    }
+
+    public SingleCellColumnExpression(PDatum column, byte[] cf, byte[] cq,
+            QualifierEncodingScheme encodingScheme, ImmutableStorageScheme immutableStorageScheme) {
         super(column, cf, SINGLE_KEYVALUE_COLUMN_QUALIFIER_BYTES);
+        this.immutableStorageScheme = immutableStorageScheme;
         Preconditions.checkNotNull(encodingScheme);
         Preconditions.checkArgument(encodingScheme != NON_ENCODED_QUALIFIERS);
         this.decodedColumnQualifier = encodingScheme.decode(cq);
@@ -67,8 +73,9 @@ public class SingleCellColumnExpression extends KeyValueColumnExpression {
         setKeyValueExpression();
     }
     
-    public SingleCellColumnExpression(PColumn column, String displayName, QualifierEncodingScheme encodingScheme) {
+    public SingleCellColumnExpression(PColumn column, String displayName, QualifierEncodingScheme encodingScheme, ImmutableStorageScheme immutableStorageScheme) {
         super(column, column.getFamilyName().getBytes(), SINGLE_KEYVALUE_COLUMN_QUALIFIER_BYTES);
+        this.immutableStorageScheme = immutableStorageScheme;
         Preconditions.checkNotNull(encodingScheme);
         Preconditions.checkArgument(encodingScheme != NON_ENCODED_QUALIFIERS);
         this.arrayColDisplayName = displayName;
@@ -86,8 +93,6 @@ public class SingleCellColumnExpression extends KeyValueColumnExpression {
         }
     	// the first position is reserved and we offset maxEncodedColumnQualifier by ENCODED_CQ_COUNTER_INITIAL_VALUE (which is the minimum encoded column qualifier)
     	int index = decodedColumnQualifier-QueryConstants.ENCODED_CQ_COUNTER_INITIAL_VALUE+1;
-    	byte serializedImmutableStorageScheme = ptr.get()[ptr.getOffset() + ptr.getLength() - Bytes.SIZEOF_BYTE];
-    	ImmutableStorageScheme immutableStorageScheme = ImmutableStorageScheme.fromSerializedValue(serializedImmutableStorageScheme);
         // Given a ptr to the entire array, set ptr to point to a particular element within that array
     	ColumnValueDecoder encoderDecoder = immutableStorageScheme.getDecoder();
     	return encoderDecoder.decode(ptr, index);
@@ -97,7 +102,18 @@ public class SingleCellColumnExpression extends KeyValueColumnExpression {
     public void readFields(DataInput input) throws IOException {
         super.readFields(input);
         this.decodedColumnQualifier = WritableUtils.readVInt(input);
-        this.encodingScheme = QualifierEncodingScheme.values()[WritableUtils.readVInt(input)];
+        int serializedEncodingScheme = WritableUtils.readVInt(input);
+        // prior to PHOENIX-4432 we weren't writing out the immutableStorageScheme in write(),
+        // so we use the decodedColumnQualifier sign to determine whether it's there
+        if (Integer.signum(serializedEncodingScheme) == -1) {
+            this.immutableStorageScheme =
+                    ImmutableStorageScheme
+                            .fromSerializedValue((byte) WritableUtils.readVInt(input));
+            serializedEncodingScheme = -serializedEncodingScheme;
+        } else {
+            this.immutableStorageScheme = ImmutableStorageScheme.SINGLE_CELL_ARRAY_WITH_OFFSETS;
+        }
+        this.encodingScheme = QualifierEncodingScheme.values()[serializedEncodingScheme];
         setKeyValueExpression();
     }
 
@@ -105,7 +121,8 @@ public class SingleCellColumnExpression extends KeyValueColumnExpression {
     public void write(DataOutput output) throws IOException {
         super.write(output);
         WritableUtils.writeVInt(output, decodedColumnQualifier);
-        WritableUtils.writeVInt(output, encodingScheme.ordinal());
+        WritableUtils.writeVInt(output, -encodingScheme.ordinal()); //negative since PHOENIX-4432
+        WritableUtils.writeVInt(output, immutableStorageScheme.getSerializedMetadataValue());
     }
     
     public KeyValueColumnExpression getKeyValueExpression() {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/edc9d12d/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java b/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java
index 500ac4b..fa60679 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java
@@ -1010,7 +1010,7 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
                         public PDataType getDataType() {
                             return null;
                         }
-                    }, dataColRef.getFamily(), dataColRef.getQualifier(), encodingScheme);
+                    }, dataColRef.getFamily(), dataColRef.getQualifier(), encodingScheme, immutableStorageScheme);
                     ImmutableBytesPtr ptr = new ImmutableBytesPtr();
                     expression.evaluate(new ValueGetterTuple(valueGetter, ts), ptr);
                     byte[] value = ptr.copyBytesIfNecessary();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/edc9d12d/phoenix-core/src/main/java/org/apache/phoenix/iterate/NonAggregateRegionScannerFactory.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/NonAggregateRegionScannerFactory.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/NonAggregateRegionScannerFactory.java
index ded33cc..c097d0d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/iterate/NonAggregateRegionScannerFactory.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/NonAggregateRegionScannerFactory.java
@@ -209,7 +209,7 @@ public class NonAggregateRegionScannerFactory extends RegionScannerFactory {
       int arrayKVRefSize = WritableUtils.readVInt(input);
       for (int i = 0; i < arrayKVRefSize; i++) {
         PTable.ImmutableStorageScheme scheme = EncodedColumnsUtil.getImmutableStorageScheme(scan);
-        KeyValueColumnExpression kvExp = scheme != PTable.ImmutableStorageScheme.ONE_CELL_PER_COLUMN ? new SingleCellColumnExpression()
+        KeyValueColumnExpression kvExp = scheme != PTable.ImmutableStorageScheme.ONE_CELL_PER_COLUMN ? new SingleCellColumnExpression(scheme)
             : new KeyValueColumnExpression();
         kvExp.readFields(input);
         arrayKVRefs.add(kvExp);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/edc9d12d/phoenix-core/src/main/java/org/apache/phoenix/schema/ColumnRef.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/ColumnRef.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/ColumnRef.java
index c73b860..fa3156d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/ColumnRef.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/ColumnRef.java
@@ -126,7 +126,9 @@ public class ColumnRef {
         }
 
         Expression expression = table.getImmutableStorageScheme() == ImmutableStorageScheme.SINGLE_CELL_ARRAY_WITH_OFFSETS ? 
-        		new SingleCellColumnExpression(column, displayName, table.getEncodingScheme()) : new KeyValueColumnExpression(column, displayName);
+                        new SingleCellColumnExpression(column, displayName,
+                                table.getEncodingScheme(), table.getImmutableStorageScheme())
+                        : new KeyValueColumnExpression(column, displayName);
 
         if (column.getExpressionStr() != null) {
             String url = PhoenixRuntime.JDBC_PROTOCOL

http://git-wip-us.apache.org/repos/asf/phoenix/blob/edc9d12d/phoenix-core/src/main/java/org/apache/phoenix/schema/PTable.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTable.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTable.java
index ec931b7..7e186ad 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTable.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTable.java
@@ -190,14 +190,14 @@ public interface PTable extends PMetaDataEntity {
             }
         },
         // stores a single cell per column family that contains all serialized column values
-        SINGLE_CELL_ARRAY_WITH_OFFSETS((byte)2) {
+        SINGLE_CELL_ARRAY_WITH_OFFSETS((byte)2, PArrayDataType.IMMUTABLE_SERIALIZATION_V2) {
             @Override
             public ColumnValueEncoder getEncoder(int numElements) {
                 PDataType type = PVarbinary.INSTANCE;
                 int estimatedSize = PArrayDataType.estimateSize(numElements, type);
                 TrustedByteArrayOutputStream byteStream = new TrustedByteArrayOutputStream(estimatedSize);
                 DataOutputStream oStream = new DataOutputStream(byteStream);
-                return new PArrayDataTypeEncoder(byteStream, oStream, numElements, type, SortOrder.ASC, false, PArrayDataType.IMMUTABLE_SERIALIZATION_VERSION);
+                return new PArrayDataTypeEncoder(byteStream, oStream, numElements, type, SortOrder.ASC, false, getSerializationVersion());
             }
             
             @Override
@@ -207,15 +207,30 @@ public interface PTable extends PMetaDataEntity {
         };
 
         private final byte serializedValue;
-        
+        private byte serializationVersion;
+
         private ImmutableStorageScheme(byte serializedValue) {
             this.serializedValue = serializedValue;
         }
 
+        private ImmutableStorageScheme(byte serializedValue, byte serializationVersion) {
+            this.serializedValue = serializedValue;
+            this.serializationVersion = serializationVersion;
+        }
+
         public byte getSerializedMetadataValue() {
             return this.serializedValue;
         }
 
+        public byte getSerializationVersion() {
+            return this.serializationVersion;
+        }
+
+        @VisibleForTesting
+        void setSerializationVersion(byte serializationVersion) {
+            this.serializationVersion = serializationVersion;
+        }
+
         public static ImmutableStorageScheme fromSerializedValue(byte serializedValue) {
             if (serializedValue < 1 || serializedValue > ImmutableStorageScheme.values().length) {
                 return null;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/edc9d12d/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PArrayDataType.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PArrayDataType.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PArrayDataType.java
index 7d742e2..162b235 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PArrayDataType.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PArrayDataType.java
@@ -75,7 +75,12 @@ public abstract class PArrayDataType<T> extends PDataType<T> {
     // array serialization format where bytes can be used as part of the row key
     public static final byte SORTABLE_SERIALIZATION_VERSION = 1;
     // array serialization format where bytes are immutable (does not support prepend/append or sorting)
+    @Deprecated
     public static final byte IMMUTABLE_SERIALIZATION_VERSION = 2;
+    // array serialization format where bytes are immutable (does not support prepend/append or sorting)
+    // differs from V1 in that nulls are not serialized
+    // we rely only on offsets to determine the presence of nulls
+    public static final byte IMMUTABLE_SERIALIZATION_V2 = 3;
     
     protected PArrayDataType(String sqlTypeName, int sqlType, Class clazz, PDataCodec codec, int ordinal) {
         super(sqlTypeName, sqlType, clazz, codec, ordinal);
@@ -217,7 +222,7 @@ public abstract class PArrayDataType<T> extends PDataType<T> {
     }
     
     public static boolean useShortForOffsetArray(int maxoffset, byte serializationVersion) {
-    	if (serializationVersion == IMMUTABLE_SERIALIZATION_VERSION) {
+    if (serializationVersion == IMMUTABLE_SERIALIZATION_VERSION || serializationVersion == IMMUTABLE_SERIALIZATION_V2) {
     		 return (maxoffset <= Short.MAX_VALUE && maxoffset >= Short.MIN_VALUE );
     	}
     	// If the max offset is less than Short.MAX_VALUE then offset array can use short
@@ -383,7 +388,10 @@ public abstract class PArrayDataType<T> extends PDataType<T> {
 		int offset;
         if (useShort) {
             offset = indexOffset + (Bytes.SIZEOF_SHORT * arrayIndex);
-            return Bytes.toShort(bytes, offset, Bytes.SIZEOF_SHORT) + (serializationVersion == PArrayDataType.IMMUTABLE_SERIALIZATION_VERSION ? 0 : Short.MAX_VALUE);
+            return Bytes.toShort(bytes, offset, Bytes.SIZEOF_SHORT)
+                    + (serializationVersion == PArrayDataType.IMMUTABLE_SERIALIZATION_VERSION
+                            || serializationVersion == PArrayDataType.IMMUTABLE_SERIALIZATION_V2 ? 0
+                                    : Short.MAX_VALUE);
         } else {
             offset = indexOffset + (Bytes.SIZEOF_INT * arrayIndex);
             return Bytes.toInt(bytes, offset, Bytes.SIZEOF_INT);
@@ -964,7 +972,11 @@ public abstract class PArrayDataType<T> extends PDataType<T> {
             }
         } else {
             for (int pos : offsetPos) {
-                short val = serializationVersion == PArrayDataType.IMMUTABLE_SERIALIZATION_VERSION ? (short)pos : (short)(pos - Short.MAX_VALUE);
+                short val =
+                        serializationVersion == PArrayDataType.IMMUTABLE_SERIALIZATION_VERSION
+                                || serializationVersion == PArrayDataType.IMMUTABLE_SERIALIZATION_V2
+                                        ? (short) pos
+                                        : (short) (pos - Short.MAX_VALUE);
 				Bytes.putShort(offsetArr, off, val);
                 off += Bytes.SIZEOF_SHORT;
             }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/edc9d12d/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PArrayDataTypeDecoder.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PArrayDataTypeDecoder.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PArrayDataTypeDecoder.java
index 7a6ea91..22fa46c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PArrayDataTypeDecoder.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PArrayDataTypeDecoder.java
@@ -22,6 +22,7 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.ColumnValueDecoder;
+import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.tuple.Tuple;
 import org.apache.phoenix.util.ByteUtil;
 
@@ -78,13 +79,45 @@ public class PArrayDataTypeDecoder implements ColumnValueDecoder {
             }
             int elementLength = 0;
             if (arrayIndex == (noOfElements - 1)) {
-                int separatorBytes =  serializationVersion == PArrayDataType.SORTABLE_SERIALIZATION_VERSION ? 3 : 0;
-                elementLength = (bytes[currOffset + initPos] == QueryConstants.SEPARATOR_BYTE || bytes[currOffset + initPos] == QueryConstants.DESC_SEPARATOR_BYTE) ? 0 : indexOffset
-                        - (currOffset + initPos) - separatorBytes;
+                // in the original IMMUTABLE_SERIALIZATION_VERSION (v1), for nulls we store
+                // (separatorByte, #_of_nulls) in the data. Because of the separatorByte, we can't
+                // distinguish between nulls and actual data values that start with the separator
+                // byte. We do a hack here to limit the damage by checking offsets - if the prior
+                // offset had a length of 0, then we know we're storing 2 or more nulls. However, we
+                // still can't fix the case distinguishing a single null from a short value. There
+                // are two kinds of separatorByte, so the results will be potentially incorrect for
+                // 2 short values that correspond to (separatorByte, 1)
+                if (serializationVersion == PArrayDataType.IMMUTABLE_SERIALIZATION_VERSION) {
+                    elementLength = indexOffset - (currOffset + initPos);
+                    if (isNullValue(arrayIndex, bytes, initPos, serializationVersion, useShort, indexOffset, currOffset, elementLength)) {
+                        elementLength = 0;
+                    }
+                } else {
+                    int separatorBytes =  serializationVersion == PArrayDataType.SORTABLE_SERIALIZATION_VERSION ? 3 : 0;
+                    elementLength = isSeparatorByte(bytes, initPos, currOffset) && serializationVersion == PArrayDataType.SORTABLE_SERIALIZATION_VERSION ? 0 : indexOffset
+                            - (currOffset + initPos) - separatorBytes;
+                }
             } else {
-                int separatorByte =  serializationVersion == PArrayDataType.SORTABLE_SERIALIZATION_VERSION ? 1 : 0;
-                elementLength = (bytes[currOffset + initPos] == QueryConstants.SEPARATOR_BYTE || bytes[currOffset + initPos] == QueryConstants.DESC_SEPARATOR_BYTE) ? 0 : PArrayDataType.getOffset(bytes,
-                        arrayIndex + 1, useShort, indexOffset, serializationVersion) - currOffset - separatorByte;
+                if (serializationVersion == PArrayDataType.IMMUTABLE_SERIALIZATION_VERSION) {
+                    elementLength = PArrayDataType.getOffset(bytes, arrayIndex + 1,
+                        useShort, indexOffset, serializationVersion)
+                            - currOffset;
+                    if (isNullValue(arrayIndex, bytes, initPos, serializationVersion, useShort, indexOffset, currOffset, elementLength)) {
+                        elementLength = 0;
+                    }
+                } else {
+                    int separatorByte =
+                            serializationVersion == PArrayDataType.SORTABLE_SERIALIZATION_VERSION
+                                    ? 1
+                                    : 0;
+                    elementLength =
+                            isSeparatorByte(bytes, initPos, currOffset)
+                                    && serializationVersion == PArrayDataType.SORTABLE_SERIALIZATION_VERSION
+                                            ? 0
+                                            : PArrayDataType.getOffset(bytes, arrayIndex + 1,
+                                                useShort, indexOffset, serializationVersion)
+                                                    - currOffset - separatorByte;
+                }
             }
             ptr.set(bytes, currOffset + initPos, elementLength);
         } else {
@@ -99,4 +132,38 @@ public class PArrayDataTypeDecoder implements ColumnValueDecoder {
         return true;
     }
 
+    // returns true if the prior element in the array is a null
+    private static boolean isNullValue(int arrayIndex, byte[] bytes, int initPos,
+            byte serializationVersion, boolean useShort, int indexOffset, int currOffset,
+            int elementLength) {
+        if (isSeparatorByte(bytes, initPos, currOffset)) {
+            if (isPriorValueZeroLength(arrayIndex, bytes,
+                serializationVersion, useShort, indexOffset, currOffset)) {
+                return true;
+            } else {
+                // if there's no prior null, there can be at most 1 null
+                if (elementLength == 2) {
+                    // nullByte calculation comes from the encoding of one null
+                    // see PArrayDataType#serializeNulls
+                    byte nullByte = SortOrder.invert((byte)(0));
+                    if (bytes[initPos+currOffset+1] == nullByte) {
+                        return true;
+                    }
+                }
+            }
+        }
+        return false;
+    }
+
+    // checks prior value length by subtracting offset of the previous item from the current offset
+    private static boolean isPriorValueZeroLength(int arrayIndex, byte[] bytes, byte serializationVersion,
+            boolean useShort, int indexOffset, int currOffset) {
+        return arrayIndex > 0 && currOffset - PArrayDataType.getOffset(bytes, arrayIndex - 1,
+            useShort, indexOffset, serializationVersion) == 0;
+    }
+
+    private static boolean isSeparatorByte(byte[] bytes, int initPos, int currOffset) {
+        return bytes[currOffset + initPos] == QueryConstants.SEPARATOR_BYTE || bytes[currOffset + initPos] == QueryConstants.DESC_SEPARATOR_BYTE;
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/edc9d12d/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PArrayDataTypeEncoder.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PArrayDataTypeEncoder.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PArrayDataTypeEncoder.java
index 3dad6c3..7467981 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PArrayDataTypeEncoder.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PArrayDataTypeEncoder.java
@@ -92,7 +92,9 @@ public class PArrayDataTypeEncoder implements ColumnValueEncoder {
     // used to represent the absence of a value 
     @Override
     public void appendAbsentValue() {
-        if (serializationVersion == PArrayDataType.IMMUTABLE_SERIALIZATION_VERSION && !baseType.isFixedWidth()) {
+        if ((serializationVersion == PArrayDataType.IMMUTABLE_SERIALIZATION_VERSION
+                || serializationVersion == PArrayDataType.IMMUTABLE_SERIALIZATION_V2)
+                && !baseType.isFixedWidth()) {
             offsetPos.add(-byteStream.size());
             nulls++;
         }
@@ -125,7 +127,11 @@ public class PArrayDataTypeEncoder implements ColumnValueEncoder {
                     offsetPos.add(byteStream.size());
                     nulls++;
                 } else {
-                    nulls = PArrayDataType.serializeNulls(oStream, nulls);
+                    // we don't serialize nulls for IMMUTABLE_SERIALIZATION_V2
+                    if (serializationVersion == PArrayDataType.SORTABLE_SERIALIZATION_VERSION
+                            || serializationVersion == PArrayDataType.IMMUTABLE_SERIALIZATION_VERSION) {
+                        nulls = PArrayDataType.serializeNulls(oStream, nulls);
+                    }
                     offsetPos.add(byteStream.size());
                     if (sortOrder == SortOrder.DESC) {
                         SortOrder.invert(bytes, offset, bytes, offset, len);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/edc9d12d/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..33b7383 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
@@ -468,13 +468,14 @@ public class IndexUtil {
             KeyValueSchema keyValueSchema = deserializeLocalIndexJoinSchemaFromScan(scan); 
             boolean storeColsInSingleCell = scan.getAttribute(BaseScannerRegionObserver.COLUMNS_STORED_IN_SINGLE_CELL) != null;
             QualifierEncodingScheme encodingScheme = EncodedColumnsUtil.getQualifierEncodingScheme(scan);
+            ImmutableStorageScheme immutableStorageScheme = EncodedColumnsUtil.getImmutableStorageScheme(scan);
             Expression[] colExpressions = storeColsInSingleCell ? new SingleCellColumnExpression[dataColumns.length] : new KeyValueColumnExpression[dataColumns.length];
             for (int i = 0; i < dataColumns.length; i++) {
                 byte[] family = dataColumns[i].getFamily();
                 byte[] qualifier = dataColumns[i].getQualifier();
                 Field field = keyValueSchema.getField(i);
                 Expression dataColumnExpr =
-                        storeColsInSingleCell ? new SingleCellColumnExpression(field, family, qualifier, encodingScheme)
+                        storeColsInSingleCell ? new SingleCellColumnExpression(field, family, qualifier, encodingScheme, immutableStorageScheme)
                             : new KeyValueColumnExpression(field, family, qualifier);
                 colExpressions[i] = dataColumnExpr;
             }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/edc9d12d/phoenix-core/src/test/java/org/apache/phoenix/schema/ImmutableStorageSchemeTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/schema/ImmutableStorageSchemeTest.java b/phoenix-core/src/test/java/org/apache/phoenix/schema/ImmutableStorageSchemeTest.java
index d8c5cdb..3d1b176 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/schema/ImmutableStorageSchemeTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/schema/ImmutableStorageSchemeTest.java
@@ -17,9 +17,15 @@
  */
 package org.apache.phoenix.schema;
 
+import static org.apache.phoenix.schema.PTable.ImmutableStorageScheme.SINGLE_CELL_ARRAY_WITH_OFFSETS;
+import static org.apache.phoenix.schema.types.PArrayDataType.IMMUTABLE_SERIALIZATION_V2;
+import static org.apache.phoenix.schema.types.PArrayDataType.IMMUTABLE_SERIALIZATION_VERSION;
+import static org.apache.phoenix.util.ByteUtil.EMPTY_BYTE_ARRAY;
 import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 import java.util.Arrays;
 import java.util.List;
@@ -33,6 +39,11 @@ import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.PTable.ImmutableStorageScheme;
 import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PInteger;
+import org.apache.phoenix.schema.types.PSmallint;
+import org.apache.phoenix.schema.types.PTinyint;
+import org.apache.phoenix.schema.types.PUnsignedTinyint;
 import org.apache.phoenix.schema.types.PVarbinary;
 import org.apache.phoenix.util.ByteUtil;
 import org.junit.Test;
@@ -55,16 +66,22 @@ public class ImmutableStorageSchemeTest {
         }
     };
     private ImmutableStorageScheme immutableStorageScheme;
+    byte serializationVersion;
     
-    @Parameters(name="ImmutableStorageSchemeTest_immutableStorageScheme={0}}") // name is used by failsafe as file name in reports
-    public static ImmutableStorageScheme[] data() {
-        ImmutableStorageScheme[] values = ImmutableStorageScheme.values();
-        // skip ONE_CELL_PER_COLUMN
-        return Arrays.copyOfRange(values, 1, values.length);
+    @Parameters(name="ImmutableStorageSchemeTest_immutableStorageScheme={0},serializationVersion={1}}") // name is used by failsafe as file name in reports
+    public static List<Object[]> data() {
+        return Arrays.asList(new Object[][] {
+                { SINGLE_CELL_ARRAY_WITH_OFFSETS,
+                        IMMUTABLE_SERIALIZATION_VERSION },
+                { SINGLE_CELL_ARRAY_WITH_OFFSETS,
+                        IMMUTABLE_SERIALIZATION_V2 }
+                        });
     }
     
-    public ImmutableStorageSchemeTest(ImmutableStorageScheme immutableStorageScheme) {
+    public ImmutableStorageSchemeTest(ImmutableStorageScheme immutableStorageScheme, byte serializationVersion) {
         this.immutableStorageScheme = immutableStorageScheme;
+        this.immutableStorageScheme.setSerializationVersion(serializationVersion);
+        this.serializationVersion = serializationVersion;
     }
 
     @Test
@@ -75,9 +92,7 @@ public class ImmutableStorageSchemeTest {
         children.add(LiteralExpression.newConstant(BYTE_ARRAY1, PVarbinary.INSTANCE));
         children.add(FALSE_EVAL_EXPRESSION);
         children.add(LiteralExpression.newConstant(BYTE_ARRAY2, PVarbinary.INSTANCE));
-        SingleCellConstructorExpression singleCellConstructorExpression = new SingleCellConstructorExpression(immutableStorageScheme, children);
-        ImmutableBytesPtr ptr = new ImmutableBytesPtr();
-        singleCellConstructorExpression.evaluate(null, ptr);
+        ImmutableBytesPtr ptr = evaluate(children);
         
         ImmutableBytesPtr ptrCopy = new ImmutableBytesPtr(ptr);
         ColumnValueDecoder decoder = immutableStorageScheme.getDecoder();
@@ -160,23 +175,201 @@ public class ImmutableStorageSchemeTest {
         children.add(nullExpression);
         children.add(LiteralExpression.newConstant(BYTE_ARRAY1, PVarbinary.INSTANCE));
         children.add(LiteralExpression.newConstant(BYTE_ARRAY2, PVarbinary.INSTANCE));
-        SingleCellConstructorExpression singleCellConstructorExpression = new SingleCellConstructorExpression(immutableStorageScheme, children);
+        ImmutableBytesPtr ptr = evaluate(children);
+
+        assertDecodedContents(ptr, new byte[][] {EMPTY_BYTE_ARRAY, EMPTY_BYTE_ARRAY, BYTE_ARRAY1, BYTE_ARRAY2});
+    }
+
+    @Test
+    public void testTrailingNulls() throws Exception {
+        List<Expression> children = Lists.newArrayListWithExpectedSize(4);
+        LiteralExpression nullExpression = LiteralExpression.newConstant(null);
+        children.add(LiteralExpression.newConstant(BYTE_ARRAY1, PVarbinary.INSTANCE));
+        children.add(LiteralExpression.newConstant(BYTE_ARRAY2, PVarbinary.INSTANCE));
+        children.add(nullExpression);
+        children.add(nullExpression);
+        ImmutableBytesPtr ptr = evaluate(children);
+
+        assertDecodedContents(ptr, new byte[][] {BYTE_ARRAY1, BYTE_ARRAY2, EMPTY_BYTE_ARRAY, EMPTY_BYTE_ARRAY});
+    }
+
+    @Test
+    public void testManyNulls() throws Exception {
+        List<Expression> children = Lists.newArrayListWithExpectedSize(4);
+        LiteralExpression nullExpression = LiteralExpression.newConstant(null);
+        byte[][] testData = new byte[300][];
+        children.add(LiteralExpression.newConstant(BYTE_ARRAY1, PVarbinary.INSTANCE));
+        testData[0] = BYTE_ARRAY1;
+        for (int i = 1; i < testData.length - 1; i++) {
+            children.add(nullExpression);
+            testData[i] = EMPTY_BYTE_ARRAY;
+        }
+        children.add(LiteralExpression.newConstant(BYTE_ARRAY2, PVarbinary.INSTANCE));
+        testData[299] = BYTE_ARRAY2;
+        ImmutableBytesPtr ptr = evaluate(children);
+
+        assertDecodedContents(ptr, testData);
+    }
+
+    @Test
+    public void testSingleLeadingTrailingNull() throws Exception {
+        List<Expression> children = Lists.newArrayListWithExpectedSize(4);
+        LiteralExpression nullExpression = LiteralExpression.newConstant(null);
+        children.add(nullExpression);
+        children.add(LiteralExpression.newConstant(BYTE_ARRAY1, PVarbinary.INSTANCE));
+        children.add(nullExpression);
+        ImmutableBytesPtr ptr = evaluate(children);
+
+        assertDecodedContents(ptr,
+            new byte[][] { EMPTY_BYTE_ARRAY, BYTE_ARRAY1, EMPTY_BYTE_ARRAY });
+    }
+
+    @Test
+    public void testSingleMiddleNull() throws Exception {
+        List<Expression> children = Lists.newArrayListWithExpectedSize(4);
+        LiteralExpression nullExpression = LiteralExpression.newConstant(null);
+        children.add(LiteralExpression.newConstant(BYTE_ARRAY1, PVarbinary.INSTANCE));
+        children.add(nullExpression);
+        children.add(LiteralExpression.newConstant(BYTE_ARRAY2, PVarbinary.INSTANCE));
+        ImmutableBytesPtr ptr = evaluate(children);
+
+        assertDecodedContents(ptr, new byte[][] { BYTE_ARRAY1, EMPTY_BYTE_ARRAY, BYTE_ARRAY2 });
+    }
+
+    @Test
+    public void testAllShortValues() throws Exception {
+        int curr = Short.MIN_VALUE;
+        List<Expression> children = Lists.newArrayListWithExpectedSize(1);
+        List<Integer> failedValues = Lists.newArrayList();
+        while (curr <= Short.MAX_VALUE) {
+            children.add(LiteralExpression.newConstant(curr, PSmallint.INSTANCE));
+            ImmutableBytesPtr ptr = evaluate(children);
+            ColumnValueDecoder decoder = immutableStorageScheme.getDecoder();
+            assertTrue(decoder.decode(ptr, 0));
+            if (ptr.getLength() == 0) {
+                failedValues.add(curr);
+            } else {
+                if (curr != PSmallint.INSTANCE.getCodec().decodeShort(ptr.copyBytesIfNecessary(), 0,
+                    SortOrder.ASC)) {
+                    failedValues.add(curr);
+                }
+            }
+            children.remove(0);
+            curr++;
+        }
+        // in v1, we can't distinguish a null from two short values
+        if (serializationVersion == IMMUTABLE_SERIALIZATION_VERSION) {
+            assertTrue(failedValues.size() + " values were not properly decoded: " + failedValues,
+                failedValues.size() == 2);
+        } else {
+            assertTrue(failedValues.size() + " values were not properly decoded: " + failedValues,
+                failedValues.size() == 0);
+        }
+    }
+
+    @Test
+    public void testSingleByteValues() throws Exception {
+        List<Expression> children = Lists.newArrayListWithExpectedSize(4);
+        LiteralExpression nullExpression = LiteralExpression.newConstant(null);
+        children.add(nullExpression);
+        children.add(LiteralExpression.newConstant((byte) -128, PTinyint.INSTANCE));
+        children.add(nullExpression);
+        children.add(LiteralExpression.newConstant((byte) 0, PUnsignedTinyint.INSTANCE));
+        children.add(nullExpression);
+        children.add(LiteralExpression.newConstant((byte) 127, PUnsignedTinyint.INSTANCE));
+        ImmutableBytesPtr ptr = evaluate(children);
+
+        assertNullAtIndex(ptr, 0);
+        assertValueAtIndex(ptr, 1, (byte) -128, PTinyint.INSTANCE);
+        assertNullAtIndex(ptr, 2);
+        assertValueAtIndex(ptr, 3, (byte) 0, PUnsignedTinyint.INSTANCE);
+        assertNullAtIndex(ptr, 4);
+        assertValueAtIndex(ptr, 5, (byte) 127, PUnsignedTinyint.INSTANCE);
+    }
+
+    @Test
+    public void testSeparatorByteValues() throws Exception {
+        List<Expression> children = Lists.newArrayListWithExpectedSize(4);
+        LiteralExpression nullExpression = LiteralExpression.newConstant(null);
+        children.add(nullExpression);
+        children.add(LiteralExpression.newConstant((short) -32513, PSmallint.INSTANCE));
+        children.add(nullExpression);
+        children.add(LiteralExpression.newConstant((short) 32767, PSmallint.INSTANCE));
+        children.add(nullExpression);
+        children.add(LiteralExpression.newConstant(Integer.MAX_VALUE, PInteger.INSTANCE));
+        children.add(nullExpression);
+        children.add(LiteralExpression.newConstant(Integer.MIN_VALUE, PInteger.INSTANCE));
+        // see if we can differentiate two nulls and {separatorByte, 2}
+        children.add(nullExpression);
+        children.add(nullExpression);
+        children.add(LiteralExpression.newConstant((short) -32514, PSmallint.INSTANCE));
+
+        ImmutableBytesPtr ptr = evaluate(children);
+
+        assertNullAtIndex(ptr, 0);
+        try {
+            assertValueAtIndex(ptr, 1, (short) -32513, PSmallint.INSTANCE);
+        } catch (Exception e) {
+            if (serializationVersion != IMMUTABLE_SERIALIZATION_VERSION) {
+                fail("Failed on exception " + e);
+            }
+        }
+        assertNullAtIndex(ptr, 2);
+        try {
+            assertValueAtIndex(ptr, 3, (short) 32767, PSmallint.INSTANCE);
+        } catch (Exception e) {
+            if (serializationVersion != IMMUTABLE_SERIALIZATION_VERSION) {
+                fail("Failed on exception " + e);
+            }
+        }
+        assertNullAtIndex(ptr, 4);
+        assertValueAtIndex(ptr, 5, Integer.MAX_VALUE, PInteger.INSTANCE);
+        assertNullAtIndex(ptr, 6);
+        assertValueAtIndex(ptr, 7, Integer.MIN_VALUE, PInteger.INSTANCE);
+        assertNullAtIndex(ptr, 8);
+        assertNullAtIndex(ptr, 9);
+        assertValueAtIndex(ptr, 10, (short) -32514, PSmallint.INSTANCE);
+    }
+
+    private void assertNullAtIndex(ImmutableBytesPtr ptr, int index) {
+        assertValueAtIndex(ptr, index, null, null);
+    }
+
+    private void assertValueAtIndex(ImmutableBytesPtr ptr, int index, Object value,
+            PDataType type) {
+        ImmutableBytesPtr ptrCopy = new ImmutableBytesPtr(ptr);
+        ColumnValueDecoder decoder = immutableStorageScheme.getDecoder();
+        assertTrue(decoder.decode(ptrCopy, index));
+        if (value == null) {
+            assertArrayEquals(EMPTY_BYTE_ARRAY, ptrCopy.copyBytesIfNecessary());
+            return;
+        }
+        Object decoded;
+        if (type.equals(PSmallint.INSTANCE)) {
+            decoded = type.getCodec().decodeShort(ptrCopy.copyBytesIfNecessary(), 0, SortOrder.ASC);
+        } else if (type.equals(PInteger.INSTANCE)) {
+            decoded = type.getCodec().decodeInt(ptrCopy.copyBytesIfNecessary(), 0, SortOrder.ASC);
+        } else { // assume byte for all other types
+            decoded = type.getCodec().decodeByte(ptrCopy.copyBytesIfNecessary(), 0, SortOrder.ASC);
+        }
+        assertEquals(value, decoded);
+    }
+
+    private ImmutableBytesPtr evaluate(List<Expression> children) {
+        SingleCellConstructorExpression singleCellConstructorExpression =
+                new SingleCellConstructorExpression(immutableStorageScheme, children);
         ImmutableBytesPtr ptr = new ImmutableBytesPtr();
         singleCellConstructorExpression.evaluate(null, ptr);
-        
-        ImmutableBytesPtr ptrCopy = new ImmutableBytesPtr(ptr);
+        return ptr;
+    }
+
+    private void assertDecodedContents(ImmutableBytesPtr ptr, byte[]... contents) {
         ColumnValueDecoder decoder = immutableStorageScheme.getDecoder();
-        assertTrue(decoder.decode(ptrCopy, 0));
-        assertArrayEquals(ByteUtil.EMPTY_BYTE_ARRAY, ptrCopy.copyBytesIfNecessary());
-        ptrCopy = new ImmutableBytesPtr(ptr);
-        assertTrue(decoder.decode(ptrCopy, 1));
-        assertArrayEquals(ByteUtil.EMPTY_BYTE_ARRAY, ptrCopy.copyBytesIfNecessary());
-        ptrCopy = new ImmutableBytesPtr(ptr);
-        assertTrue(decoder.decode(ptrCopy, 2));
-        assertArrayEquals(BYTE_ARRAY1, ptrCopy.copyBytesIfNecessary());
-        ptrCopy = new ImmutableBytesPtr(ptr);
-        assertTrue(decoder.decode(ptrCopy, 3));
-        assertArrayEquals(BYTE_ARRAY2, ptrCopy.copyBytesIfNecessary());
+        for (int i = 0; i < contents.length; i++) {
+            ImmutableBytesPtr ptrCopy = new ImmutableBytesPtr(ptr);
+            assertTrue(decoder.decode(ptrCopy, i));
+            assertArrayEquals(contents[i], ptrCopy.copyBytesIfNecessary());
+        }
     }
-    
+
 }


[17/50] [abbrv] phoenix git commit: PHOENIX-4389 Flapping tests SystemTablePermissionsIT and MigrateSystemTablesToSystemNamespaceIT

Posted by td...@apache.org.
PHOENIX-4389 Flapping tests SystemTablePermissionsIT and MigrateSystemTablesToSystemNamespaceIT


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

Branch: refs/heads/system-catalog
Commit: d46d4e564d94077b637ad8f34d8785a4ac3486a5
Parents: 4e0c0a3
Author: Karan Mehta <ka...@gmail.com>
Authored: Tue Nov 28 15:39:03 2017 -0800
Committer: Thomas D'Silva <td...@apache.org>
Committed: Tue Nov 28 18:33:15 2017 -0800

----------------------------------------------------------------------
 .../end2end/MigrateSystemTablesToSystemNamespaceIT.java      | 8 ++++++++
 1 file changed, 8 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/d46d4e56/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java
index 91e34be..c36cdd3 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MigrateSystemTablesToSystemNamespaceIT.java
@@ -18,6 +18,7 @@ package org.apache.phoenix.end2end;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.phoenix.coprocessor.MetaDataProtocol;
@@ -87,6 +88,7 @@ public class MigrateSystemTablesToSystemNamespaceIT extends BaseTest {
         testUtil = new HBaseTestingUtility();
         Configuration conf = testUtil.getConfiguration();
         enableNamespacesOnServer(conf);
+        configureRandomHMasterPort(conf);
         testUtil.startMiniCluster(1);
     }
 
@@ -289,6 +291,12 @@ public class MigrateSystemTablesToSystemNamespaceIT extends BaseTest {
         conf.set(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, Boolean.TRUE.toString());
     }
 
+    // For PHOENIX-4389 (Flapping tests SystemTablePermissionsIT and MigrateSystemTablesToSystemNamespaceIT)
+    private void configureRandomHMasterPort(Configuration conf) {
+        // Avoid multiple clusters trying to bind the master's info port (16010)
+        conf.setInt(HConstants.MASTER_INFO_PORT, -1);
+    }
+
     private Properties getClientPropertiesWithSystemMappingEnabled() {
         Properties clientProps = new Properties();
         clientProps.setProperty(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, Boolean.TRUE.toString());


[20/50] [abbrv] phoenix git commit: PHOENIX-672 Add GRANT and REVOKE commands using HBase AccessController

Posted by td...@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/88038a2d
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/88038a2d
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/88038a2d

Branch: refs/heads/system-catalog
Commit: 88038a2dacb7aa1a90015163d4d75d04793e4e11
Parents: 355ee52
Author: Karan Mehta <ka...@gmail.com>
Authored: Wed Nov 29 12:04:06 2017 -0800
Committer: Thomas D'Silva <td...@apache.org>
Committed: Wed Nov 29 20:39:19 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/88038a2d/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/88038a2d/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/88038a2d/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);
     }
 }


[27/50] [abbrv] phoenix git commit: PHOENIX-4415 Ignore CURRENT_SCN property if set in Pig Storer

Posted by td...@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/d6e61af8
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/d6e61af8
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/d6e61af8

Branch: refs/heads/system-catalog
Commit: d6e61af807f7a4e605c61217bac556ffe00ea237
Parents: 1c3387d
Author: James Taylor <jt...@salesforce.com>
Authored: Thu Dec 7 19:13:25 2017 -0800
Committer: James Taylor <jt...@salesforce.com>
Committed: Thu Dec 7 19:13:25 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/d6e61af8/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/d6e61af8/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/d6e61af8/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/d6e61af8/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/d6e61af8/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/d6e61af8/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
+}


[14/50] [abbrv] phoenix git commit: PHOENIX-4387 DefaultColumnValueIT failing in non-US build environments (Pedro Boado)

Posted by td...@apache.org.
PHOENIX-4387 DefaultColumnValueIT failing in non-US build environments (Pedro Boado)


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

Branch: refs/heads/system-catalog
Commit: ca1e17b75ced1e618869ad2c2ada019fcc336c02
Parents: 03fc331
Author: James Taylor <jt...@salesforce.com>
Authored: Sat Nov 18 11:36:53 2017 -0800
Committer: James Taylor <jt...@salesforce.com>
Committed: Sat Nov 18 11:36:53 2017 -0800

----------------------------------------------------------------------
 .../org/apache/phoenix/end2end/DefaultColumnValueIT.java    | 9 ++++++---
 1 file changed, 6 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/ca1e17b7/phoenix-core/src/it/java/org/apache/phoenix/end2end/DefaultColumnValueIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DefaultColumnValueIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DefaultColumnValueIT.java
index 62d79bc..59b15d8 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DefaultColumnValueIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DefaultColumnValueIT.java
@@ -32,6 +32,7 @@ import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.Time;
 import java.sql.Timestamp;
+import java.text.DecimalFormatSymbols;
 
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.util.ByteUtil;
@@ -44,6 +45,8 @@ public class DefaultColumnValueIT extends ParallelStatsDisabledIT {
     private String sharedTable1;
     private String sharedTable2;
 
+    private String DEFAULT_CURRENCY_SYMBOL = DecimalFormatSymbols.getInstance().getCurrencySymbol();
+
     @Before
     public void init() {
         sharedTable1 = generateUniqueName();
@@ -583,7 +586,7 @@ public class DefaultColumnValueIT extends ParallelStatsDisabledIT {
                 "pk INTEGER PRIMARY KEY,"
                 + "c1 INTEGER DEFAULT 1 + 9,"
                 + "c2 DOUBLE DEFAULT SQRT(91506.25),"
-                + "c3 DECIMAL DEFAULT TO_NUMBER('$123.33', '\u00A4###.##'),"
+                + "c3 DECIMAL DEFAULT TO_NUMBER('" + DEFAULT_CURRENCY_SYMBOL + "123.33', '\u00A4###.##'),"
                 + "c4 VARCHAR DEFAULT 'AB' || 'CD',"
                 + "c5 CHAR(5) DEFAULT 'E' || 'F',"
                 + "c6 INTEGER DEFAULT \"MONTH\"(TO_TIMESTAMP('2015-6-05'))"
@@ -599,7 +602,7 @@ public class DefaultColumnValueIT extends ParallelStatsDisabledIT {
                 "pk INTEGER NOT NULL,"
                 + "c1 INTEGER NOT NULL DEFAULT 1 + 9,"
                 + "c2 DOUBLE NOT NULL DEFAULT SQRT(91506.25),"
-                + "c3 DECIMAL NOT NULL DEFAULT TO_NUMBER('$123.33', '\u00A4###.##'),"
+                + "c3 DECIMAL NOT NULL DEFAULT TO_NUMBER('" + DEFAULT_CURRENCY_SYMBOL + "123.33', '\u00A4###.##'),"
                 + "c4 VARCHAR NOT NULL DEFAULT 'AB' || 'CD',"
                 + "c5 CHAR(5) NOT NULL DEFAULT 'E' || 'F',"
                 + "c6 INTEGER NOT NULL DEFAULT \"MONTH\"(TO_TIMESTAMP('2015-6-05')),"
@@ -1045,7 +1048,7 @@ public class DefaultColumnValueIT extends ParallelStatsDisabledIT {
                 "pk INTEGER PRIMARY KEY,"
                 + "c1 INTEGER DEFAULT 1 + 9,"
                 + "c2 DOUBLE DEFAULT SQRT(91506.25),"
-                + "c3 DECIMAL DEFAULT TO_NUMBER('$123.33', '\u00A4###.##'),"
+                + "c3 DECIMAL DEFAULT TO_NUMBER('" + DEFAULT_CURRENCY_SYMBOL + "123.33', '\u00A4###.##'),"
                 + "c4 VARCHAR DEFAULT 'AB' || 'CD',"
                 + "c5 CHAR(5) DEFAULT 'E' || 'F',"
                 + "c6 INTEGER DEFAULT \"MONTH\"(TO_TIMESTAMP('2015-6-05'))"


[42/50] [abbrv] phoenix git commit: PHOENIX-4487 Missing SYSTEM.MUTEX table upgrading from 4.7 to 4.13

Posted by td...@apache.org.
PHOENIX-4487 Missing SYSTEM.MUTEX table upgrading from 4.7 to 4.13


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

Branch: refs/heads/system-catalog
Commit: 2759727e444ef8ae7475d25979be84bfe89895f5
Parents: 1636f61
Author: James Taylor <jt...@salesforce.com>
Authored: Fri Dec 22 10:58:55 2017 -0800
Committer: James Taylor <jt...@salesforce.com>
Committed: Tue Dec 26 13:21:09 2017 -0800

----------------------------------------------------------------------
 .../query/ConnectionQueryServicesImpl.java      | 29 ++++++++++++++++----
 1 file changed, 24 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/2759727e/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 072bf28..38be6af 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
@@ -2640,11 +2640,23 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             } catch (TableAlreadyExistsException e) {
                 long currentServerSideTableTimeStamp = e.getTable().getTimeStamp();
                 sysCatalogTableName = e.getTable().getPhysicalName().getString();
-                if (currentServerSideTableTimeStamp < MIN_SYSTEM_TABLE_TIMESTAMP
-                        && (acquiredMutexLock = acquireUpgradeMutex(currentServerSideTableTimeStamp, mutexRowKey))) {
-                    snapshotName = getSysCatalogSnapshotName(currentServerSideTableTimeStamp);
-                    createSnapshot(snapshotName, sysCatalogTableName);
-                    snapshotCreated = true;
+                if (currentServerSideTableTimeStamp < MIN_SYSTEM_TABLE_TIMESTAMP) {
+                    // Ensure that the SYSTEM.MUTEX table has been created prior
+                    // to attempting to acquire the upgrade mutex. If namespace
+                    // mapping is enabled, we've already done this earlier in the
+                    // upgrade, so no need for a bunch of wasted RPCs.
+                    if (currentServerSideTableTimeStamp <= MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_10_0 &&
+                            !SchemaUtil.isNamespaceMappingEnabled(PTableType.SYSTEM,
+                                    ConnectionQueryServicesImpl.this.getProps())) {
+                        try (HBaseAdmin admin = getAdmin()) {
+                            createSysMutexTable(admin, this.getProps());
+                        }
+                    }
+                    if (acquiredMutexLock = acquireUpgradeMutex(currentServerSideTableTimeStamp, mutexRowKey)) {
+                        snapshotName = getSysCatalogSnapshotName(currentServerSideTableTimeStamp);
+                        createSnapshot(snapshotName, sysCatalogTableName);
+                        snapshotCreated = true;
+                    }
                 }
                 String columnsToAdd = "";
                 // This will occur if we have an older SYSTEM.CATALOG and we need to update it to
@@ -3191,6 +3203,13 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             // Try acquiring a lock in SYSMUTEX table before migrating the tables since it involves disabling the table
             // If we cannot acquire lock, it means some old client is either migrating SYSCAT or trying to upgrade the
             // schema of SYSCAT table and hence it should not be interrupted
+            // Create mutex if not already created
+            if (!tableNames.contains(PhoenixDatabaseMetaData.SYSTEM_MUTEX_HBASE_TABLE_NAME)) {
+                TableName mutexName = SchemaUtil.getPhysicalTableName(PhoenixDatabaseMetaData.SYSTEM_MUTEX_NAME, props);
+                if (PhoenixDatabaseMetaData.SYSTEM_MUTEX_HBASE_TABLE_NAME.equals(mutexName) || !tableNames.contains(mutexName)) {
+                    createSysMutexTable(admin, props);
+                }
+            }
             acquiredMutexLock = acquireUpgradeMutex(MetaDataProtocol.MIN_SYSTEM_TABLE_MIGRATION_TIMESTAMP, mutexRowKey);
             if(acquiredMutexLock) {
                 logger.debug("Acquired lock in SYSMUTEX table for migrating SYSTEM tables to SYSTEM namespace");