You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by mm...@apache.org on 2017/02/09 13:38:12 UTC

[1/3] accumulo git commit: ACCUMULO-4577: modified exception thrown by Tables.getNamespaceId

Repository: accumulo
Updated Branches:
  refs/heads/1.8 04847d97d -> f4120422b


ACCUMULO-4577: modified exception thrown by Tables.getNamespaceId


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

Branch: refs/heads/1.8
Commit: 78176e5948bccb2501184bbffad31a8d7d7e6603
Parents: 15ae69c
Author: Mike Miller <mm...@apache.org>
Authored: Thu Feb 2 09:02:57 2017 -0500
Committer: Mike Miller <mm...@apache.org>
Committed: Wed Feb 8 12:24:48 2017 -0500

----------------------------------------------------------------------
 .../accumulo/core/client/impl/Tables.java       |  4 +-
 .../server/client/ClientServiceHandler.java     | 22 +++++---
 .../server/conf/TableParentConfiguration.java   |  7 ++-
 .../accumulo/master/FateServiceHandler.java     | 36 ++++++++-----
 .../master/MasterClientServiceHandler.java      | 16 ++++--
 .../accumulo/master/tableOps/CloneTable.java    | 19 +------
 .../accumulo/master/tableOps/ExportTable.java   |  7 ++-
 .../apache/accumulo/tserver/TabletServer.java   | 54 ++++++++++++++++----
 8 files changed, 110 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/78176e59/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java b/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java
index 18971ad..fcf838f 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java
@@ -272,7 +272,7 @@ public class Tables {
    * @throws IllegalArgumentException
    *           if the table doesn't exist in ZooKeeper
    */
-  public static String getNamespaceId(Instance instance, String tableId) throws IllegalArgumentException {
+  public static String getNamespaceId(Instance instance, String tableId) throws TableNotFoundException {
     checkArgument(instance != null, "instance is null");
     checkArgument(tableId != null, "tableId is null");
 
@@ -281,7 +281,7 @@ public class Tables {
 
     // We might get null out of ZooCache if this tableID doesn't exist
     if (null == n) {
-      throw new IllegalArgumentException("Table with id " + tableId + " does not exist");
+      throw new TableNotFoundException(tableId, null, null);
     }
 
     return new String(n, UTF_8);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/78176e59/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java b/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java
index fbfb492..3b34f71 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java
@@ -194,10 +194,14 @@ public class ClientServiceHandler implements ClientService.Iface {
   }
 
   @Override
-  public void grantTablePermission(TInfo tinfo, TCredentials credentials, String user, String tableName, byte permission) throws ThriftSecurityException,
-      ThriftTableOperationException {
+  public void grantTablePermission(TInfo tinfo, TCredentials credentials, String user, String tableName, byte permission) throws TException {
     String tableId = checkTableId(instance, tableName, TableOperation.PERMISSION);
-    String namespaceId = Tables.getNamespaceId(instance, tableId);
+    String namespaceId;
+    try {
+      namespaceId = Tables.getNamespaceId(instance, tableId);
+    } catch (TableNotFoundException e) {
+      throw new TException(e);
+    }
 
     security.grantTablePermission(credentials, user, tableId, TablePermission.getPermissionById(permission), namespaceId);
   }
@@ -215,10 +219,14 @@ public class ClientServiceHandler implements ClientService.Iface {
   }
 
   @Override
-  public void revokeTablePermission(TInfo tinfo, TCredentials credentials, String user, String tableName, byte permission) throws ThriftSecurityException,
-      ThriftTableOperationException {
+  public void revokeTablePermission(TInfo tinfo, TCredentials credentials, String user, String tableName, byte permission) throws TException {
     String tableId = checkTableId(instance, tableName, TableOperation.PERMISSION);
-    String namespaceId = Tables.getNamespaceId(instance, tableId);
+    String namespaceId;
+    try {
+      namespaceId = Tables.getNamespaceId(instance, tableId);
+    } catch (TableNotFoundException e) {
+      throw new TException(e);
+    }
 
     security.revokeTablePermission(credentials, user, tableId, TablePermission.getPermissionById(permission), namespaceId);
   }
@@ -437,6 +445,8 @@ public class ClientServiceHandler implements ClientService.Iface {
       throw new TException(e);
     } catch (IOException e) {
       throw new TException(e);
+    } catch (TableNotFoundException e) {
+      throw new TException(e);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/78176e59/server/base/src/main/java/org/apache/accumulo/server/conf/TableParentConfiguration.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/TableParentConfiguration.java b/server/base/src/main/java/org/apache/accumulo/server/conf/TableParentConfiguration.java
index 26fee7c..70649be 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/TableParentConfiguration.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/TableParentConfiguration.java
@@ -17,6 +17,7 @@
 package org.apache.accumulo.server.conf;
 
 import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 
@@ -35,6 +36,10 @@ public class TableParentConfiguration extends NamespaceConfiguration {
 
   @Override
   protected String getNamespaceId() {
-    return Tables.getNamespaceId(inst, tableId);
+    try {
+      return Tables.getNamespaceId(inst, tableId);
+    } catch (TableNotFoundException e) {
+      throw new RuntimeException(e);
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/78176e59/server/master/src/main/java/org/apache/accumulo/master/FateServiceHandler.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/FateServiceHandler.java b/server/master/src/main/java/org/apache/accumulo/master/FateServiceHandler.java
index 5af612c..ee0f089 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/FateServiceHandler.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/FateServiceHandler.java
@@ -171,7 +171,7 @@ class FateServiceHandler implements FateService.Iface {
         });
 
         String tableId = ClientServiceHandler.checkTableId(master.getInstance(), oldTableName, tableOp);
-        String namespaceId = Tables.getNamespaceId(master.getInstance(), tableId);
+        String namespaceId = getNamespaceIdFromTableId(tableOp, tableId);
 
         final boolean canRename;
         try {
@@ -232,8 +232,8 @@ class FateServiceHandler implements FateService.Iface {
           propertiesToSet.put(entry.getKey(), entry.getValue());
         }
 
-        master.fate.seedTransaction(opid, new TraceRepo<>(new CloneTable(c.getPrincipal(), srcTableId, tableName, propertiesToSet, propertiesToExclude)),
-            autoCleanup);
+        master.fate.seedTransaction(opid, new TraceRepo<>(new CloneTable(c.getPrincipal(), namespaceId, srcTableId, tableName, propertiesToSet,
+            propertiesToExclude)), autoCleanup);
 
         break;
       }
@@ -242,7 +242,7 @@ class FateServiceHandler implements FateService.Iface {
         String tableName = validateTableNameArgument(arguments.get(0), tableOp, NOT_SYSTEM);
 
         final String tableId = ClientServiceHandler.checkTableId(master.getInstance(), tableName, tableOp);
-        String namespaceId = Tables.getNamespaceId(master.getInstance(), tableId);
+        String namespaceId = getNamespaceIdFromTableId(tableOp, tableId);
 
         final boolean canDeleteTable;
         try {
@@ -260,7 +260,7 @@ class FateServiceHandler implements FateService.Iface {
       case TABLE_ONLINE: {
         TableOperation tableOp = TableOperation.ONLINE;
         final String tableId = validateTableIdArgument(arguments.get(0), tableOp, NOT_ROOT_ID);
-        String namespaceId = Tables.getNamespaceId(master.getInstance(), tableId);
+        String namespaceId = getNamespaceIdFromTableId(tableOp, tableId);
 
         final boolean canOnlineOfflineTable;
         try {
@@ -279,7 +279,7 @@ class FateServiceHandler implements FateService.Iface {
       case TABLE_OFFLINE: {
         TableOperation tableOp = TableOperation.OFFLINE;
         final String tableId = validateTableIdArgument(arguments.get(0), tableOp, NOT_ROOT_ID);
-        String namespaceId = Tables.getNamespaceId(master.getInstance(), tableId);
+        String namespaceId = getNamespaceIdFromTableId(tableOp, tableId);
 
         final boolean canOnlineOfflineTable;
         try {
@@ -302,7 +302,7 @@ class FateServiceHandler implements FateService.Iface {
         Text endRow = ByteBufferUtil.toText(arguments.get(2));
 
         final String tableId = ClientServiceHandler.checkTableId(master.getInstance(), tableName, tableOp);
-        String namespaceId = Tables.getNamespaceId(master.getInstance(), tableId);
+        String namespaceId = getNamespaceIdFromTableId(tableOp, tableId);
 
         final boolean canMerge;
         try {
@@ -326,7 +326,7 @@ class FateServiceHandler implements FateService.Iface {
         Text endRow = ByteBufferUtil.toText(arguments.get(2));
 
         final String tableId = ClientServiceHandler.checkTableId(master.getInstance(), tableName, tableOp);
-        String namespaceId = Tables.getNamespaceId(master.getInstance(), tableId);
+        String namespaceId = getNamespaceIdFromTableId(tableOp, tableId);
 
         final boolean canDeleteRange;
         try {
@@ -350,7 +350,7 @@ class FateServiceHandler implements FateService.Iface {
         boolean setTime = Boolean.parseBoolean(ByteBufferUtil.toString(arguments.get(3)));
 
         final String tableId = ClientServiceHandler.checkTableId(master.getInstance(), tableName, tableOp);
-        String namespaceId = Tables.getNamespaceId(master.getInstance(), tableId);
+        String namespaceId = getNamespaceIdFromTableId(tableOp, tableId);
 
         final boolean canBulkImport;
         try {
@@ -373,7 +373,7 @@ class FateServiceHandler implements FateService.Iface {
         byte[] endRow = ByteBufferUtil.toBytes(arguments.get(2));
         List<IteratorSetting> iterators = IteratorUtil.decodeIteratorSettings(ByteBufferUtil.toBytes(arguments.get(3)));
         CompactionStrategyConfig compactionStrategy = CompactionStrategyConfigUtil.decode(ByteBufferUtil.toBytes(arguments.get(4)));
-        String namespaceId = Tables.getNamespaceId(master.getInstance(), tableId);
+        String namespaceId = getNamespaceIdFromTableId(tableOp, tableId);
 
         final boolean canCompact;
         try {
@@ -393,7 +393,7 @@ class FateServiceHandler implements FateService.Iface {
       case TABLE_CANCEL_COMPACT: {
         TableOperation tableOp = TableOperation.COMPACT_CANCEL;
         String tableId = validateTableIdArgument(arguments.get(0), tableOp, null);
-        String namespaceId = Tables.getNamespaceId(master.getInstance(), tableId);
+        String namespaceId = getNamespaceIdFromTableId(tableOp, tableId);
 
         final boolean canCancelCompact;
         try {
@@ -440,7 +440,7 @@ class FateServiceHandler implements FateService.Iface {
         String exportDir = ByteBufferUtil.toString(arguments.get(1));
 
         String tableId = ClientServiceHandler.checkTableId(master.getInstance(), tableName, tableOp);
-        String namespaceId = Tables.getNamespaceId(master.getInstance(), tableId);
+        String namespaceId = getNamespaceIdFromTableId(tableOp, tableId);
 
         final boolean canExport;
         try {
@@ -453,7 +453,7 @@ class FateServiceHandler implements FateService.Iface {
         if (!canExport)
           throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
 
-        master.fate.seedTransaction(opid, new TraceRepo<>(new ExportTable(tableName, tableId, exportDir)), autoCleanup);
+        master.fate.seedTransaction(opid, new TraceRepo<>(new ExportTable(namespaceId, tableName, tableId, exportDir)), autoCleanup);
         break;
       }
       default:
@@ -461,6 +461,16 @@ class FateServiceHandler implements FateService.Iface {
     }
   }
 
+  private String getNamespaceIdFromTableId(TableOperation tableOp, String tableId) throws ThriftTableOperationException {
+    String namespaceId;
+    try {
+      namespaceId = Tables.getNamespaceId(master.getInstance(), tableId);
+    } catch (TableNotFoundException e) {
+      throw new ThriftTableOperationException(tableId, null, tableOp, TableOperationExceptionType.NOTFOUND, e.getMessage());
+    }
+    return namespaceId;
+  }
+
   /**
    * Inspects the {@link ThriftSecurityException} and throws a {@link ThriftTableOperationException} if the {@link SecurityErrorCode} on the
    * {@link ThriftSecurityException} was {code}TABLE_DOESNT_EXIST{code}. If the {@link ThriftSecurityException} is thrown because a table doesn't exist anymore,

http://git-wip-us.apache.org/repos/asf/accumulo/blob/78176e59/server/master/src/main/java/org/apache/accumulo/master/MasterClientServiceHandler.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/MasterClientServiceHandler.java b/server/master/src/main/java/org/apache/accumulo/master/MasterClientServiceHandler.java
index 69e7cdd..4bec432 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/MasterClientServiceHandler.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/MasterClientServiceHandler.java
@@ -112,7 +112,7 @@ class MasterClientServiceHandler extends FateServiceHandler implements MasterCli
 
   @Override
   public long initiateFlush(TInfo tinfo, TCredentials c, String tableId) throws ThriftSecurityException, ThriftTableOperationException {
-    String namespaceId = Tables.getNamespaceId(instance, tableId);
+    String namespaceId = getNamespaceIdFromTableId(TableOperation.FLUSH, tableId);
     master.security.canFlush(c, tableId, namespaceId);
 
     String zTablePath = Constants.ZROOT + "/" + master.getInstance().getInstanceID() + Constants.ZTABLES + "/" + tableId + Constants.ZTABLE_FLUSH_ID;
@@ -140,7 +140,7 @@ class MasterClientServiceHandler extends FateServiceHandler implements MasterCli
   @Override
   public void waitForFlush(TInfo tinfo, TCredentials c, String tableId, ByteBuffer startRow, ByteBuffer endRow, long flushID, long maxLoops)
       throws ThriftSecurityException, ThriftTableOperationException {
-    String namespaceId = Tables.getNamespaceId(instance, tableId);
+    String namespaceId = getNamespaceIdFromTableId(TableOperation.FLUSH, tableId);
     master.security.canFlush(c, tableId, namespaceId);
 
     if (endRow != null && startRow != null && ByteBufferUtil.toText(startRow).compareTo(ByteBufferUtil.toText(endRow)) >= 0)
@@ -253,6 +253,16 @@ class MasterClientServiceHandler extends FateServiceHandler implements MasterCli
 
   }
 
+  private String getNamespaceIdFromTableId(TableOperation tableOp, String tableId) throws ThriftTableOperationException {
+    String namespaceId;
+    try {
+      namespaceId = Tables.getNamespaceId(instance, tableId);
+    } catch (TableNotFoundException e) {
+      throw new ThriftTableOperationException(tableId, null, tableOp, TableOperationExceptionType.NOTFOUND, e.getMessage());
+    }
+    return namespaceId;
+  }
+
   @Override
   public MasterMonitorInfo getMasterStats(TInfo info, TCredentials credentials) throws ThriftSecurityException {
     return master.getMasterMonitorInfo();
@@ -424,7 +434,7 @@ class MasterClientServiceHandler extends FateServiceHandler implements MasterCli
   private void alterTableProperty(TCredentials c, String tableName, String property, String value, TableOperation op) throws ThriftSecurityException,
       ThriftTableOperationException {
     final String tableId = ClientServiceHandler.checkTableId(master.getInstance(), tableName, op);
-    String namespaceId = Tables.getNamespaceId(master.getInstance(), tableId);
+    String namespaceId = getNamespaceIdFromTableId(op, tableId);
     if (!master.security.canAlterTable(c, tableId, namespaceId))
       throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/78176e59/server/master/src/main/java/org/apache/accumulo/master/tableOps/CloneTable.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/CloneTable.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/CloneTable.java
index 84529a6..f9e0e81 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/CloneTable.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/CloneTable.java
@@ -19,38 +19,23 @@ package org.apache.accumulo.master.tableOps;
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.accumulo.core.client.Instance;
-import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.client.impl.thrift.TableOperation;
-import org.apache.accumulo.core.client.impl.thrift.TableOperationExceptionType;
-import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException;
 import org.apache.accumulo.fate.Repo;
 import org.apache.accumulo.master.Master;
-import org.apache.accumulo.server.client.HdfsZooInstance;
 
 public class CloneTable extends MasterRepo {
 
   private static final long serialVersionUID = 1L;
   private CloneInfo cloneInfo;
 
-  public CloneTable(String user, String srcTableId, String tableName, Map<String,String> propertiesToSet, Set<String> propertiesToExclude)
-      throws ThriftTableOperationException {
+  public CloneTable(String user, String namespaceId, String srcTableId, String tableName, Map<String,String> propertiesToSet, Set<String> propertiesToExclude) {
     cloneInfo = new CloneInfo();
     cloneInfo.user = user;
     cloneInfo.srcTableId = srcTableId;
     cloneInfo.tableName = tableName;
     cloneInfo.propertiesToExclude = propertiesToExclude;
     cloneInfo.propertiesToSet = propertiesToSet;
-    Instance inst = HdfsZooInstance.getInstance();
-    try {
-      cloneInfo.srcNamespaceId = Tables.getNamespaceId(inst, cloneInfo.srcTableId);
-    } catch (IllegalArgumentException e) {
-      if (inst == null || cloneInfo.srcTableId == null) {
-        // just throw the exception if the illegal argument was thrown by the argument checker and not due to table non-existence
-        throw e;
-      }
-      throw new ThriftTableOperationException(cloneInfo.srcTableId, "", TableOperation.CLONE, TableOperationExceptionType.NOTFOUND, "Table does not exist");
-    }
+    cloneInfo.srcNamespaceId = namespaceId;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/accumulo/blob/78176e59/server/master/src/main/java/org/apache/accumulo/master/tableOps/ExportTable.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/ExportTable.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/ExportTable.java
index cd50a18..9dd4800 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/ExportTable.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/ExportTable.java
@@ -16,10 +16,9 @@
  */
 package org.apache.accumulo.master.tableOps;
 
-import org.apache.accumulo.core.client.impl.Tables;
+import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException;
 import org.apache.accumulo.fate.Repo;
 import org.apache.accumulo.master.Master;
-import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.hadoop.fs.Path;
 
 public class ExportTable extends MasterRepo {
@@ -27,12 +26,12 @@ public class ExportTable extends MasterRepo {
 
   private final ExportInfo tableInfo;
 
-  public ExportTable(String tableName, String tableId, String exportDir) {
+  public ExportTable(String namespaceId, String tableName, String tableId, String exportDir) throws ThriftTableOperationException {
     tableInfo = new ExportInfo();
     tableInfo.tableName = tableName;
     tableInfo.exportDir = exportDir;
     tableInfo.tableID = tableId;
-    tableInfo.namespaceID = Tables.getNamespaceId(HdfsZooInstance.getInstance(), tableId);
+    tableInfo.namespaceID = namespaceId;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/accumulo/blob/78176e59/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
index acbb513..6c585d0 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
@@ -59,6 +59,7 @@ import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Durability;
 import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.impl.CompressedIterators;
 import org.apache.accumulo.core.client.impl.DurabilityImpl;
 import org.apache.accumulo.core.client.impl.ScannerImpl;
@@ -448,7 +449,13 @@ public class TabletServer extends AccumuloServerContext implements Runnable {
         long readaheadThreshold) throws NotServingTabletException, ThriftSecurityException, org.apache.accumulo.core.tabletserver.thrift.TooManyFilesException {
 
       String tableId = new String(textent.getTable(), UTF_8);
-      if (!security.canScan(credentials, tableId, Tables.getNamespaceId(getInstance(), tableId), range, columns, ssiList, ssio, authorizations))
+      String namespaceId;
+      try {
+        namespaceId = Tables.getNamespaceId(getInstance(), tableId);
+      } catch (TableNotFoundException e1) {
+        throw new NotServingTabletException(textent);
+      }
+      if (!security.canScan(credentials, tableId, namespaceId, range, columns, ssiList, ssio, authorizations))
         throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
 
       if (!security.userHasAuthorizations(credentials, authorizations))
@@ -596,9 +603,16 @@ public class TabletServer extends AccumuloServerContext implements Runnable {
         throw new IllegalArgumentException("Cannot batch scan over multiple tables");
 
       // check if user has permission to the tables
-      for (String tableId : tables)
-        if (!security.canScan(credentials, tableId, Tables.getNamespaceId(getInstance(), tableId), tbatch, tcolumns, ssiList, ssio, authorizations))
+      for (String tableId : tables) {
+        String namespaceId;
+        try {
+          namespaceId = Tables.getNamespaceId(getInstance(), tableId);
+        } catch (TableNotFoundException e1) {
+          throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.TABLE_DOESNT_EXIST);
+        }
+        if (!security.canScan(credentials, tableId, namespaceId, tbatch, tcolumns, ssiList, ssio, authorizations))
           throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
+      }
 
       try {
         if (!security.userHasAuthorizations(credentials, authorizations))
@@ -715,11 +729,12 @@ public class TabletServer extends AccumuloServerContext implements Runnable {
         return;
       }
 
+      String tableId = "";
       try {
         // if user has no permission to write to this table, add it to
         // the failures list
         boolean sameTable = us.currentTablet != null && (us.currentTablet.getExtent().getTableId().equals(keyExtent.getTableId()));
-        String tableId = keyExtent.getTableId().toString();
+        tableId = keyExtent.getTableId().toString();
         if (sameTable || security.canWrite(us.getCredentials(), tableId, Tables.getNamespaceId(getInstance(), tableId))) {
           long t2 = System.currentTimeMillis();
           us.authTimes.addStat(t2 - t1);
@@ -743,6 +758,15 @@ public class TabletServer extends AccumuloServerContext implements Runnable {
             updateMetrics.add(TabletServerUpdateMetrics.PERMISSION_ERRORS, 0);
           return;
         }
+      } catch (TableNotFoundException tnfe) {
+        log.error("Table " + tableId + " not found ", tnfe);
+        long t2 = System.currentTimeMillis();
+        us.authTimes.addStat(t2 - t1);
+        us.currentTablet = null;
+        us.authFailures.put(keyExtent, SecurityErrorCode.TABLE_DOESNT_EXIST);
+        if (updateMetrics.isEnabled())
+          updateMetrics.add(TabletServerUpdateMetrics.UNKNOWN_TABLET_ERRORS, 0);
+        return;
       } catch (ThriftSecurityException e) {
         log.error("Denying permission to check user " + us.getUser() + " with user " + e.getUser(), e);
         long t2 = System.currentTimeMillis();
@@ -999,7 +1023,13 @@ public class TabletServer extends AccumuloServerContext implements Runnable {
         throws NotServingTabletException, ConstraintViolationException, ThriftSecurityException {
 
       final String tableId = new String(tkeyExtent.getTable(), UTF_8);
-      if (!security.canWrite(credentials, tableId, Tables.getNamespaceId(getInstance(), tableId)))
+      String namespaceId;
+      try {
+        namespaceId = Tables.getNamespaceId(getInstance(), tableId);
+      } catch (TableNotFoundException e1) {
+        throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.TABLE_DOESNT_EXIST);
+      }
+      if (!security.canWrite(credentials, tableId, namespaceId))
         throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
       final KeyExtent keyExtent = new KeyExtent(tkeyExtent);
       final Tablet tablet = onlineTablets.get(new KeyExtent(keyExtent));
@@ -1232,7 +1262,13 @@ public class TabletServer extends AccumuloServerContext implements Runnable {
         TDurability tdurabilty) throws ThriftSecurityException, TException {
 
       Authorizations userauths = null;
-      if (!security.canConditionallyUpdate(credentials, tableId, Tables.getNamespaceId(getInstance(), tableId), authorizations))
+      String namespaceId;
+      try {
+        namespaceId = Tables.getNamespaceId(getInstance(), tableId);
+      } catch (TableNotFoundException e) {
+        throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.TABLE_DOESNT_EXIST);
+      }
+      if (!security.canConditionallyUpdate(credentials, tableId, namespaceId, authorizations))
         throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
 
       userauths = security.getUserAuthorizations(credentials);
@@ -1320,9 +1356,9 @@ public class TabletServer extends AccumuloServerContext implements Runnable {
       String namespaceId;
       try {
         namespaceId = Tables.getNamespaceId(getInstance(), tableId);
-      } catch (IllegalArgumentException ex) {
-        // table does not exist, try to educate the client
-        throw new NotServingTabletException(tkeyExtent);
+      } catch (TableNotFoundException ex) {
+        // tableOperationsImpl catches ThriftSeccurityException and checks for missing table
+        throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.TABLE_DOESNT_EXIST);
       }
 
       if (!security.canSplitTablet(credentials, tableId, namespaceId))


[2/3] accumulo git commit: Merge branch 'ACCUMULO-4577_1.7' into 1.8

Posted by mm...@apache.org.
Merge branch 'ACCUMULO-4577_1.7' into 1.8

Conflicts:
	core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java
	server/master/src/main/java/org/apache/accumulo/master/tableOps/CloneTable.java
	server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java


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

Branch: refs/heads/1.8
Commit: a0b2dc864c053a5d03d13d42eb68e5b53a4379eb
Parents: 04847d9 78176e5
Author: Mike Miller <mm...@apache.org>
Authored: Wed Feb 8 14:09:11 2017 -0500
Committer: Mike Miller <mm...@apache.org>
Committed: Wed Feb 8 14:09:11 2017 -0500

----------------------------------------------------------------------
 .../accumulo/core/client/impl/Tables.java       |  2 +-
 .../server/client/ClientServiceHandler.java     | 22 +++++---
 .../server/conf/TableParentConfiguration.java   |  7 ++-
 .../accumulo/master/FateServiceHandler.java     | 36 ++++++++-----
 .../master/MasterClientServiceHandler.java      | 16 ++++--
 .../accumulo/master/tableOps/CloneTable.java    | 20 +-------
 .../accumulo/master/tableOps/ExportTable.java   |  7 ++-
 .../apache/accumulo/tserver/TabletServer.java   | 54 ++++++++++++++++----
 8 files changed, 109 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/a0b2dc86/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a0b2dc86/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a0b2dc86/server/master/src/main/java/org/apache/accumulo/master/FateServiceHandler.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a0b2dc86/server/master/src/main/java/org/apache/accumulo/master/MasterClientServiceHandler.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a0b2dc86/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
----------------------------------------------------------------------
diff --cc server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
index 7751681,6c585d0..a937235
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
@@@ -61,7 -59,7 +61,8 @@@ import org.apache.accumulo.core.client.
  import org.apache.accumulo.core.client.AccumuloSecurityException;
  import org.apache.accumulo.core.client.Durability;
  import org.apache.accumulo.core.client.Instance;
 +import org.apache.accumulo.core.client.SampleNotPresentException;
+ import org.apache.accumulo.core.client.TableNotFoundException;
  import org.apache.accumulo.core.client.impl.CompressedIterators;
  import org.apache.accumulo.core.client.impl.DurabilityImpl;
  import org.apache.accumulo.core.client.impl.ScannerImpl;
@@@ -465,11 -446,16 +466,17 @@@ public class TabletServer extends Accum
      @Override
      public InitialScan startScan(TInfo tinfo, TCredentials credentials, TKeyExtent textent, TRange range, List<TColumn> columns, int batchSize,
          List<IterInfo> ssiList, Map<String,Map<String,String>> ssio, List<ByteBuffer> authorizations, boolean waitForWrites, boolean isolated,
 -        long readaheadThreshold) throws NotServingTabletException, ThriftSecurityException, org.apache.accumulo.core.tabletserver.thrift.TooManyFilesException {
 +        long readaheadThreshold, TSamplerConfiguration tSamplerConfig, long batchTimeOut, String context) throws NotServingTabletException,
 +        ThriftSecurityException, org.apache.accumulo.core.tabletserver.thrift.TooManyFilesException, TSampleNotPresentException {
  
        String tableId = new String(textent.getTable(), UTF_8);
-       if (!security.canScan(credentials, tableId, Tables.getNamespaceId(getInstance(), tableId), range, columns, ssiList, ssio, authorizations))
+       String namespaceId;
+       try {
+         namespaceId = Tables.getNamespaceId(getInstance(), tableId);
+       } catch (TableNotFoundException e1) {
+         throw new NotServingTabletException(textent);
+       }
+       if (!security.canScan(credentials, tableId, namespaceId, range, columns, ssiList, ssio, authorizations))
          throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
  
        if (!security.userHasAuthorizations(credentials, authorizations))
@@@ -1270,10 -1259,16 +1300,16 @@@
  
      @Override
      public TConditionalSession startConditionalUpdate(TInfo tinfo, TCredentials credentials, List<ByteBuffer> authorizations, String tableId,
 -        TDurability tdurabilty) throws ThriftSecurityException, TException {
 +        TDurability tdurabilty, String classLoaderContext) throws ThriftSecurityException, TException {
  
        Authorizations userauths = null;
-       if (!security.canConditionallyUpdate(credentials, tableId, Tables.getNamespaceId(getInstance(), tableId), authorizations))
+       String namespaceId;
+       try {
+         namespaceId = Tables.getNamespaceId(getInstance(), tableId);
+       } catch (TableNotFoundException e) {
+         throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.TABLE_DOESNT_EXIST);
+       }
+       if (!security.canConditionallyUpdate(credentials, tableId, namespaceId, authorizations))
          throw new ThriftSecurityException(credentials.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
  
        userauths = security.getUserAuthorizations(credentials);


[3/3] accumulo git commit: ACCUMULO-4577: Fix merge error

Posted by mm...@apache.org.
ACCUMULO-4577: Fix merge error


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

Branch: refs/heads/1.8
Commit: f4120422bcaf8466c5c49054761c4bc10f8fba68
Parents: a0b2dc8
Author: Mike Miller <mm...@apache.org>
Authored: Wed Feb 8 14:36:26 2017 -0500
Committer: Mike Miller <mm...@apache.org>
Committed: Wed Feb 8 14:36:26 2017 -0500

----------------------------------------------------------------------
 .../src/main/java/org/apache/accumulo/core/client/impl/Tables.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/f4120422/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java b/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java
index cbf9671..fcf838f 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java
@@ -272,7 +272,7 @@ public class Tables {
    * @throws IllegalArgumentException
    *           if the table doesn't exist in ZooKeeper
    */
-  public static String getNamespaceId(Instance instance, String tableId) throws IllegalArgumentException {
+  public static String getNamespaceId(Instance instance, String tableId) throws TableNotFoundException {
     checkArgument(instance != null, "instance is null");
     checkArgument(tableId != null, "tableId is null");