http://git-wip-us.apache.org/repos/asf/accumulo/blob/92613388/server/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthorizor.java
----------------------------------------------------------------------
diff --cc 
server/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthorizor.java
index 36bd86a,0000000..8873938
mode 100644,000000..100644
--- 
a/server/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthorizor.java
+++ 
b/server/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthorizor.java
@@@ -1,157 -1,0 +1,156 @@@
 +/*
 + * 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.accumulo.server.security.handler;
 +
 +import java.util.Collections;
 +import java.util.HashMap;
 +import java.util.Map;
 +import java.util.Set;
 +import java.util.TreeSet;
 +
 +import org.apache.accumulo.core.Constants;
 +import org.apache.accumulo.core.client.AccumuloSecurityException;
 +import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode;
 +import org.apache.accumulo.core.security.Authorizations;
 +import org.apache.accumulo.core.security.SystemPermission;
 +import org.apache.accumulo.core.security.TablePermission;
 +import org.apache.accumulo.core.security.thrift.TCredentials;
 +import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 +import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
 +import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy;
 +import org.apache.accumulo.server.zookeeper.ZooCache;
 +import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 +import org.apache.log4j.Logger;
 +import org.apache.zookeeper.KeeperException;
 +
 +public class ZKAuthorizor implements Authorizor {
 +  private static final Logger log = Logger.getLogger(ZKAuthorizor.class);
 +  private static Authorizor zkAuthorizorInstance = null;
 +  
 +  private final String ZKUserAuths = "/Authorizations";
 +  
 +  private String ZKUserPath;
 +  private final ZooCache zooCache;
 +  
 +  public static synchronized Authorizor getInstance() {
 +    if (zkAuthorizorInstance == null)
 +      zkAuthorizorInstance = new ZKAuthorizor();
 +    return zkAuthorizorInstance;
 +  }
 +  
 +  public ZKAuthorizor() {
 +    zooCache = new ZooCache();
 +  }
 +  
++  @Override
 +  public void initialize(String instanceId, boolean initialize) {
 +    ZKUserPath = ZKSecurityTool.getInstancePath(instanceId) + "/users";
 +  }
 +  
++  @Override
 +  public Authorizations getCachedUserAuthorizations(String user) {
 +    byte[] authsBytes = zooCache.get(ZKUserPath + "/" + user + ZKUserAuths);
 +    if (authsBytes != null)
 +      return ZKSecurityTool.convertAuthorizations(authsBytes);
 +    return Constants.NO_AUTHS;
 +  }
 +  
 +  @Override
 +  public boolean validSecurityHandlers(Authenticator auth, PermissionHandler 
pm) {
 +    return true;
 +  }
 +  
 +  @Override
 +  public void initializeSecurity(TCredentials itw, String rootuser) throws 
AccumuloSecurityException {
 +    IZooReaderWriter zoo = ZooReaderWriter.getRetryingInstance();
 +    
 +    // create the root user with all system privileges, no table privileges, 
and no record-level authorizations
 +    Set<SystemPermission> rootPerms = new TreeSet<SystemPermission>();
 +    for (SystemPermission p : SystemPermission.values())
 +      rootPerms.add(p);
 +    Map<String,Set<TablePermission>> tablePerms = new 
HashMap<String,Set<TablePermission>>();
 +    // Allow the root user to flush the !METADATA table
 +    tablePerms.put(Constants.METADATA_TABLE_ID, 
Collections.singleton(TablePermission.ALTER_TABLE));
 +    
 +    try {
 +      // prep parent node of users with root username
 +      if (!zoo.exists(ZKUserPath))
 +        zoo.putPersistentData(ZKUserPath, rootuser.getBytes(Constants.UTF8), 
NodeExistsPolicy.FAIL);
 +      
 +      initUser(rootuser);
 +      zoo.putPersistentData(ZKUserPath + "/" + rootuser + ZKUserAuths, 
ZKSecurityTool.convertAuthorizations(Constants.NO_AUTHS), 
NodeExistsPolicy.FAIL);
 +    } catch (KeeperException e) {
 +      log.error(e, e);
 +      throw new RuntimeException(e);
 +    } catch (InterruptedException e) {
 +      log.error(e, e);
 +      throw new RuntimeException(e);
 +    }
 +  }
 +  
-   /**
-    * @param user
-    * @throws AccumuloSecurityException
-    */
++  @Override
 +  public void initUser(String user) throws AccumuloSecurityException {
 +    IZooReaderWriter zoo = ZooReaderWriter.getRetryingInstance();
 +    try {
 +      zoo.putPersistentData(ZKUserPath + "/" + user, new byte[0], 
NodeExistsPolicy.SKIP);
 +    } catch (KeeperException e) {
 +      log.error(e, e);
 +      throw new AccumuloSecurityException(user, 
SecurityErrorCode.CONNECTION_ERROR, e);
 +    } catch (InterruptedException e) {
 +      log.error(e, e);
 +      throw new RuntimeException(e);
 +    }
 +  }
 +  
 +  @Override
 +  public void dropUser(String user) throws AccumuloSecurityException {
 +    try {
 +      synchronized (zooCache) {
 +        IZooReaderWriter zoo = ZooReaderWriter.getRetryingInstance();
 +        zoo.recursiveDelete(ZKUserPath + "/" + user + ZKUserAuths, 
NodeMissingPolicy.SKIP);
 +        zooCache.clear(ZKUserPath + "/" + user);
 +      }
 +    } catch (InterruptedException e) {
 +      log.error(e, e);
 +      throw new RuntimeException(e);
 +    } catch (KeeperException e) {
 +      log.error(e, e);
 +      if (e.code().equals(KeeperException.Code.NONODE))
 +        throw new AccumuloSecurityException(user, 
SecurityErrorCode.USER_DOESNT_EXIST, e);
 +      throw new AccumuloSecurityException(user, 
SecurityErrorCode.CONNECTION_ERROR, e);
 +      
 +    }
 +  }
 +  
 +  @Override
 +  public void changeAuthorizations(String user, Authorizations 
authorizations) throws AccumuloSecurityException {
 +    try {
 +      synchronized (zooCache) {
 +        zooCache.clear();
 +        ZooReaderWriter.getRetryingInstance().putPersistentData(ZKUserPath + 
"/" + user + ZKUserAuths, ZKSecurityTool.convertAuthorizations(authorizations),
 +            NodeExistsPolicy.OVERWRITE);
 +      }
 +    } catch (KeeperException e) {
 +      log.error(e, e);
 +      throw new AccumuloSecurityException(user, 
SecurityErrorCode.CONNECTION_ERROR, e);
 +    } catch (InterruptedException e) {
 +      log.error(e, e);
 +      throw new RuntimeException(e);
 +    }
 +  }
 +  
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/92613388/server/src/main/java/org/apache/accumulo/server/security/handler/ZKPermHandler.java
----------------------------------------------------------------------
diff --cc 
server/src/main/java/org/apache/accumulo/server/security/handler/ZKPermHandler.java
index 4a05658,0000000..d802eb9
mode 100644,000000..100644
--- 
a/server/src/main/java/org/apache/accumulo/server/security/handler/ZKPermHandler.java
+++ 
b/server/src/main/java/org/apache/accumulo/server/security/handler/ZKPermHandler.java
@@@ -1,365 -1,0 +1,363 @@@
 +/*
 + * 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.accumulo.server.security.handler;
 +
 +import java.util.Collections;
 +import java.util.HashMap;
 +import java.util.Map;
 +import java.util.Map.Entry;
 +import java.util.Set;
 +import java.util.TreeSet;
 +
 +import org.apache.accumulo.core.Constants;
 +import org.apache.accumulo.core.client.AccumuloSecurityException;
 +import org.apache.accumulo.core.client.TableNotFoundException;
 +import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode;
 +import org.apache.accumulo.core.security.SystemPermission;
 +import org.apache.accumulo.core.security.TablePermission;
 +import org.apache.accumulo.core.security.thrift.TCredentials;
 +import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 +import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
 +import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy;
 +import org.apache.accumulo.server.zookeeper.ZooCache;
 +import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 +import org.apache.log4j.Logger;
 +import org.apache.zookeeper.KeeperException;
 +import org.apache.zookeeper.KeeperException.Code;
 +
 +/**
 + * 
 + */
 +public class ZKPermHandler implements PermissionHandler {
 +  private static final Logger log = Logger.getLogger(ZKAuthorizor.class);
 +  private static PermissionHandler zkPermHandlerInstance = null;
 +  
 +  private String ZKUserPath;
 +  private String ZKTablePath;
 +  private final ZooCache zooCache;
 +  private static final String ZKUserSysPerms = "/System";
 +  private static final String ZKUserTablePerms = "/Tables";
 +  
 +  public static synchronized PermissionHandler getInstance() {
 +    if (zkPermHandlerInstance == null)
 +      zkPermHandlerInstance = new ZKPermHandler();
 +    return zkPermHandlerInstance;
 +  }
 +  
++  @Override
 +  public void initialize(String instanceId, boolean initialize) {
 +    ZKUserPath = ZKSecurityTool.getInstancePath(instanceId) + "/users";
 +    ZKTablePath = ZKSecurityTool.getInstancePath(instanceId) + "/tables";
 +  }
 +  
 +  public ZKPermHandler() {
 +    zooCache = new ZooCache();
 +  }
 +  
 +  @Override
 +  public boolean hasTablePermission(String user, String table, 
TablePermission permission) throws TableNotFoundException {
 +    byte[] serializedPerms;
 +    try {
 +      String path = ZKUserPath + "/" + user + ZKUserTablePerms + "/" + table;
 +      ZooReaderWriter.getRetryingInstance().sync(path);
 +      serializedPerms = ZooReaderWriter.getRetryingInstance().getData(path, 
null);
 +    } catch (KeeperException e) {
 +      if (e.code() == Code.NONODE) {
 +        // maybe the table was just deleted?
 +        try {
 +          // check for existence:
 +          ZooReaderWriter.getRetryingInstance().getData(ZKTablePath + "/" + 
table, null);
 +          // it's there, you don't have permission
 +          return false;
 +        } catch (InterruptedException ex) {
 +          log.warn("Unhandled InterruptedException, failing closed for table 
permission check", e);
 +          return false;
 +        } catch (KeeperException ex) {
 +          // not there, throw an informative exception
 +          if (e.code() == Code.NONODE) {
 +            throw new TableNotFoundException(null, table, "while checking 
permissions");
 +          }
 +          log.warn("Unhandled InterruptedException, failing closed for table 
permission check", e);
 +        }
 +        return false;
 +      }
 +      log.warn("Unhandled KeeperException, failing closed for table 
permission check", e);
 +      return false;
 +    } catch (InterruptedException e) {
 +      log.warn("Unhandled InterruptedException, failing closed for table 
permission check", e);
 +      return false;
 +    }
 +    if (serializedPerms != null) {
 +      return 
ZKSecurityTool.convertTablePermissions(serializedPerms).contains(permission);
 +    }
 +    return false;
 +  }
 +  
 +  @Override
 +  public boolean hasCachedTablePermission(String user, String table, 
TablePermission permission) throws AccumuloSecurityException, 
TableNotFoundException {
 +    byte[] serializedPerms = zooCache.get(ZKUserPath + "/" + user + 
ZKUserTablePerms + "/" + table);
 +    if (serializedPerms != null) {
 +      return 
ZKSecurityTool.convertTablePermissions(serializedPerms).contains(permission);
 +    }
 +    return false;
 +  }
 +  
 +  @Override
 +  public void grantSystemPermission(String user, SystemPermission permission) 
throws AccumuloSecurityException {
 +    try {
 +      byte[] permBytes = zooCache.get(ZKUserPath + "/" + user + 
ZKUserSysPerms);
 +      Set<SystemPermission> perms;
 +      if (permBytes == null) {
 +        perms = new TreeSet<SystemPermission>();
 +      } else {
 +        perms = ZKSecurityTool.convertSystemPermissions(permBytes);
 +      }
 +      
 +      if (perms.add(permission)) {
 +        synchronized (zooCache) {
 +          zooCache.clear();
 +          ZooReaderWriter.getRetryingInstance().putPersistentData(ZKUserPath 
+ "/" + user + ZKUserSysPerms, ZKSecurityTool.convertSystemPermissions(perms),
 +              NodeExistsPolicy.OVERWRITE);
 +        }
 +      }
 +    } catch (KeeperException e) {
 +      log.error(e, e);
 +      throw new AccumuloSecurityException(user, 
SecurityErrorCode.CONNECTION_ERROR, e);
 +    } catch (InterruptedException e) {
 +      log.error(e, e);
 +      throw new RuntimeException(e);
 +    }
 +  }
 +  
 +  @Override
 +  public void grantTablePermission(String user, String table, TablePermission 
permission) throws AccumuloSecurityException {
 +    Set<TablePermission> tablePerms;
 +    byte[] serializedPerms = zooCache.get(ZKUserPath + "/" + user + 
ZKUserTablePerms + "/" + table);
 +    if (serializedPerms != null)
 +      tablePerms = ZKSecurityTool.convertTablePermissions(serializedPerms);
 +    else
 +      tablePerms = new TreeSet<TablePermission>();
 +    
 +    try {
 +      if (tablePerms.add(permission)) {
 +        synchronized (zooCache) {
 +          zooCache.clear(ZKUserPath + "/" + user + ZKUserTablePerms + "/" + 
table);
 +          IZooReaderWriter zoo = ZooReaderWriter.getRetryingInstance();
 +          zoo.putPersistentData(ZKUserPath + "/" + user + ZKUserTablePerms + 
"/" + table, ZKSecurityTool.convertTablePermissions(tablePerms),
 +              NodeExistsPolicy.OVERWRITE);
 +        }
 +      }
 +    } catch (KeeperException e) {
 +      log.error(e, e);
 +      throw new AccumuloSecurityException(user, 
SecurityErrorCode.CONNECTION_ERROR, e);
 +    } catch (InterruptedException e) {
 +      log.error(e, e);
 +      throw new RuntimeException(e);
 +    }
 +  }
 +  
 +  @Override
 +  public void revokeSystemPermission(String user, SystemPermission 
permission) throws AccumuloSecurityException {
 +    byte[] sysPermBytes = zooCache.get(ZKUserPath + "/" + user + 
ZKUserSysPerms);
 +    
 +    // User had no system permission, nothing to revoke.
 +    if (sysPermBytes == null)
 +      return;
 +    
 +    Set<SystemPermission> sysPerms = 
ZKSecurityTool.convertSystemPermissions(sysPermBytes);
 +    
 +    try {
 +      if (sysPerms.remove(permission)) {
 +        synchronized (zooCache) {
 +          zooCache.clear();
 +          ZooReaderWriter.getRetryingInstance().putPersistentData(ZKUserPath 
+ "/" + user + ZKUserSysPerms, 
ZKSecurityTool.convertSystemPermissions(sysPerms),
 +              NodeExistsPolicy.OVERWRITE);
 +        }
 +      }
 +    } catch (KeeperException e) {
 +      log.error(e, e);
 +      throw new AccumuloSecurityException(user, 
SecurityErrorCode.CONNECTION_ERROR, e);
 +    } catch (InterruptedException e) {
 +      log.error(e, e);
 +      throw new RuntimeException(e);
 +    }
 +  }
 +  
 +  @Override
 +  public void revokeTablePermission(String user, String table, 
TablePermission permission) throws AccumuloSecurityException {
 +    byte[] serializedPerms = zooCache.get(ZKUserPath + "/" + user + 
ZKUserTablePerms + "/" + table);
 +    
 +    // User had no table permission, nothing to revoke.
 +    if (serializedPerms == null)
 +      return;
 +    
 +    Set<TablePermission> tablePerms = 
ZKSecurityTool.convertTablePermissions(serializedPerms);
 +    try {
 +      if (tablePerms.remove(permission)) {
 +        zooCache.clear();
 +        IZooReaderWriter zoo = ZooReaderWriter.getRetryingInstance();
 +        if (tablePerms.size() == 0)
 +          zoo.recursiveDelete(ZKUserPath + "/" + user + ZKUserTablePerms + 
"/" + table, NodeMissingPolicy.SKIP);
 +        else
 +          zoo.putPersistentData(ZKUserPath + "/" + user + ZKUserTablePerms + 
"/" + table, ZKSecurityTool.convertTablePermissions(tablePerms),
 +              NodeExistsPolicy.OVERWRITE);
 +      }
 +    } catch (KeeperException e) {
 +      log.error(e, e);
 +      throw new AccumuloSecurityException(user, 
SecurityErrorCode.CONNECTION_ERROR, e);
 +    } catch (InterruptedException e) {
 +      log.error(e, e);
 +      throw new RuntimeException(e);
 +    }
 +  }
 +  
 +  @Override
 +  public void cleanTablePermissions(String table) throws 
AccumuloSecurityException {
 +    try {
 +      synchronized (zooCache) {
 +        zooCache.clear();
 +        IZooReaderWriter zoo = ZooReaderWriter.getRetryingInstance();
 +        for (String user : zooCache.getChildren(ZKUserPath))
 +          zoo.recursiveDelete(ZKUserPath + "/" + user + ZKUserTablePerms + 
"/" + table, NodeMissingPolicy.SKIP);
 +      }
 +    } catch (KeeperException e) {
 +      log.error(e, e);
 +      throw new AccumuloSecurityException("unknownUser", 
SecurityErrorCode.CONNECTION_ERROR, e);
 +    } catch (InterruptedException e) {
 +      log.error(e, e);
 +      throw new RuntimeException(e);
 +    }
 +  }
 +  
 +  @Override
 +  public void initializeSecurity(TCredentials itw, String rootuser) throws 
AccumuloSecurityException {
 +    IZooReaderWriter zoo = ZooReaderWriter.getRetryingInstance();
 +    
 +    // create the root user with all system privileges, no table privileges, 
and no record-level authorizations
 +    Set<SystemPermission> rootPerms = new TreeSet<SystemPermission>();
 +    for (SystemPermission p : SystemPermission.values())
 +      rootPerms.add(p);
 +    Map<String,Set<TablePermission>> tablePerms = new 
HashMap<String,Set<TablePermission>>();
 +    // Allow the root user to flush the !METADATA table
 +    tablePerms.put(Constants.METADATA_TABLE_ID, 
Collections.singleton(TablePermission.ALTER_TABLE));
 +    
 +    try {
 +      // prep parent node of users with root username
 +      if (!zoo.exists(ZKUserPath))
 +        zoo.putPersistentData(ZKUserPath, rootuser.getBytes(Constants.UTF8), 
NodeExistsPolicy.FAIL);
 +      
 +      initUser(rootuser);
 +      zoo.putPersistentData(ZKUserPath + "/" + rootuser + ZKUserSysPerms, 
ZKSecurityTool.convertSystemPermissions(rootPerms), NodeExistsPolicy.FAIL);
 +      for (Entry<String,Set<TablePermission>> entry : tablePerms.entrySet())
 +        createTablePerm(rootuser, entry.getKey(), entry.getValue());
 +    } catch (KeeperException e) {
 +      log.error(e, e);
 +      throw new RuntimeException(e);
 +    } catch (InterruptedException e) {
 +      log.error(e, e);
 +      throw new RuntimeException(e);
 +    }
 +  }
 +  
-   /**
-    * @param user
-    * @throws AccumuloSecurityException
-    */
++  @Override
 +  public void initUser(String user) throws AccumuloSecurityException {
 +    IZooReaderWriter zoo = ZooReaderWriter.getRetryingInstance();
 +    try {
 +      zoo.putPersistentData(ZKUserPath + "/" + user, new byte[0], 
NodeExistsPolicy.SKIP);
 +      zoo.putPersistentData(ZKUserPath + "/" + user + ZKUserTablePerms, new 
byte[0], NodeExistsPolicy.SKIP);
 +    } catch (KeeperException e) {
 +      log.error(e, e);
 +      throw new AccumuloSecurityException(user, 
SecurityErrorCode.CONNECTION_ERROR, e);
 +    } catch (InterruptedException e) {
 +      log.error(e, e);
 +      throw new RuntimeException(e);
 +    }
 +  }
 +  
 +  /**
 +   * Sets up a new table configuration for the provided user/table. No 
checking for existence is done here, it should be done before calling.
 +   */
 +  private void createTablePerm(String user, String table, 
Set<TablePermission> perms) throws KeeperException, InterruptedException {
 +    synchronized (zooCache) {
 +      zooCache.clear();
 +      ZooReaderWriter.getRetryingInstance().putPersistentData(ZKUserPath + 
"/" + user + ZKUserTablePerms + "/" + table,
 +          ZKSecurityTool.convertTablePermissions(perms), 
NodeExistsPolicy.FAIL);
 +    }
 +  }
 +  
 +  @Override
 +  public void cleanUser(String user) throws AccumuloSecurityException {
 +    try {
 +      synchronized (zooCache) {
 +        IZooReaderWriter zoo = ZooReaderWriter.getRetryingInstance();
 +        zoo.recursiveDelete(ZKUserPath + "/" + user + ZKUserSysPerms, 
NodeMissingPolicy.SKIP);
 +        zoo.recursiveDelete(ZKUserPath + "/" + user + ZKUserTablePerms, 
NodeMissingPolicy.SKIP);
 +        zooCache.clear(ZKUserPath + "/" + user);
 +      }
 +    } catch (InterruptedException e) {
 +      log.error(e, e);
 +      throw new RuntimeException(e);
 +    } catch (KeeperException e) {
 +      log.error(e, e);
 +      if (e.code().equals(KeeperException.Code.NONODE))
 +        throw new AccumuloSecurityException(user, 
SecurityErrorCode.USER_DOESNT_EXIST, e);
 +      throw new AccumuloSecurityException(user, 
SecurityErrorCode.CONNECTION_ERROR, e);
 +      
 +    }
 +  }
 +  
 +  @Override
 +  public boolean hasSystemPermission(String user, SystemPermission 
permission) throws AccumuloSecurityException {
 +    byte[] perms;
 +    try {
 +      String path = ZKUserPath + "/" + user + ZKUserSysPerms;
 +      ZooReaderWriter.getRetryingInstance().sync(path);
 +      perms = ZooReaderWriter.getRetryingInstance().getData(path, null);
 +    } catch (KeeperException e) {
 +      if (e.code() == Code.NONODE) {
 +        return false;
 +      }
 +      log.warn("Unhandled KeeperException, failing closed for table 
permission check", e);
 +      return false;
 +    } catch (InterruptedException e) {
 +      log.warn("Unhandled InterruptedException, failing closed for table 
permission check", e);
 +      return false;
 +    }
 +    
 +    if (perms == null)
 +      return false;
 +    return 
ZKSecurityTool.convertSystemPermissions(perms).contains(permission);
 +  }
 +  
 +  @Override
 +  public boolean hasCachedSystemPermission(String user, SystemPermission 
permission) throws AccumuloSecurityException {
 +    byte[] perms = zooCache.get(ZKUserPath + "/" + user + ZKUserSysPerms);
 +    if (perms == null)
 +      return false;
 +    return 
ZKSecurityTool.convertSystemPermissions(perms).contains(permission);
 +  }
 +  
 +  @Override
 +  public boolean validSecurityHandlers(Authenticator authent, Authorizor 
author) {
 +    return true;
 +  }
 +  
 +  @Override
 +  public void initTable(String table) throws AccumuloSecurityException {
 +    // All proper housekeeping is done on delete and permission granting, no 
work needs to be done here
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/92613388/server/src/main/java/org/apache/accumulo/server/tabletserver/MemValue.java
----------------------------------------------------------------------
diff --cc 
server/src/main/java/org/apache/accumulo/server/tabletserver/MemValue.java
index 735bf20,0000000..f68859a
mode 100644,000000..100644
--- a/server/src/main/java/org/apache/accumulo/server/tabletserver/MemValue.java
+++ b/server/src/main/java/org/apache/accumulo/server/tabletserver/MemValue.java
@@@ -1,93 -1,0 +1,95 @@@
 +/*
 + * 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.accumulo.server.tabletserver;
 +
 +import java.io.DataOutput;
 +import java.io.IOException;
 +
 +import org.apache.accumulo.core.data.Value;
 +
 +/**
 + * 
 + */
 +public class MemValue extends Value {
 +  int kvCount;
 +  boolean merged = false;
 +  
 +  /**
 +   * @param value
 +   *          Value
 +   * @param kv
 +   *          kv count
 +   */
 +  public MemValue(byte[] value, int kv) {
 +    super(value);
 +    this.kvCount = kv;
 +  }
 +  
 +  public MemValue() {
 +    super();
 +    this.kvCount = Integer.MAX_VALUE;
 +  }
 +  
 +  public MemValue(Value value, int kv) {
 +    super(value);
 +    this.kvCount = kv;
 +  }
 +  
 +  // Override
++  @Override
 +  public void write(final DataOutput out) throws IOException {
 +    if (!merged) {
 +      byte[] combinedBytes = new byte[getSize() + 4];
 +      System.arraycopy(value, 0, combinedBytes, 4, getSize());
 +      combinedBytes[0] = (byte) (kvCount >>> 24);
 +      combinedBytes[1] = (byte) (kvCount >>> 16);
 +      combinedBytes[2] = (byte) (kvCount >>> 8);
 +      combinedBytes[3] = (byte) (kvCount);
 +      value = combinedBytes;
 +      merged = true;
 +    }
 +    super.write(out);
 +  }
 +  
++  @Override
 +  public void set(final byte[] b) {
 +    super.set(b);
 +    merged = false;
 +  }
 +
++  @Override
 +  public void copy(byte[] b) {
 +    super.copy(b);
 +    merged = false;
 +  }
 +  
 +  /**
 +   * Takes a Value and will take out the embedded kvCount, and then return 
that value while replacing the Value with the original unembedded version
 +   * 
-    * @param v
 +   * @return The kvCount embedded in v.
 +   */
 +  public static int splitKVCount(Value v) {
 +    if (v instanceof MemValue)
 +      return ((MemValue) v).kvCount;
 +    
 +    byte[] originalBytes = new byte[v.getSize() - 4];
 +    byte[] combined = v.get();
 +    System.arraycopy(combined, 4, originalBytes, 0, originalBytes.length);
 +    v.set(originalBytes);
 +    return (combined[0] << 24) + ((combined[1] & 0xFF) << 16) + ((combined[2] 
& 0xFF) << 8) + (combined[3] & 0xFF);
 +  }
 +}

Reply via email to