You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by el...@apache.org on 2015/07/02 19:28:39 UTC

[1/2] accumulo git commit: ACCUMULO-3925 Wrap expected exceptions in a special class for better handling

Repository: accumulo
Updated Branches:
  refs/heads/master 560b3a5d1 -> eb1ba1860


ACCUMULO-3925 Wrap expected exceptions in a special class for better handling

Since the FATE runner is the one which logs errors when an operation
throws an exception, we need to pass extra state up to the runner to
discern between unexpected failure and expected failure. We do this
by wrapping the original exception.


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

Branch: refs/heads/master
Commit: bb031b34f880b50ab5c796b0b5ec3515434efeb1
Parents: 560b3a5
Author: Josh Elser <el...@apache.org>
Authored: Mon Jun 29 13:24:31 2015 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Thu Jul 2 12:38:07 2015 -0400

----------------------------------------------------------------------
 ...AcceptableThriftTableOperationException.java | 34 ++++++++++++++++++++
 .../accumulo/fate/AcceptableException.java      | 29 +++++++++++++++++
 .../java/org/apache/accumulo/fate/Fate.java     |  9 +++++-
 .../accumulo/master/FateServiceHandler.java     |  6 ++--
 .../accumulo/master/tableOps/BulkImport.java    | 20 ++++++------
 .../master/tableOps/ClonePermissions.java       |  4 +--
 .../accumulo/master/tableOps/CloneTable.java    |  7 ++--
 .../accumulo/master/tableOps/CompactRange.java  | 10 +++---
 .../master/tableOps/CompactionDriver.java       |  8 ++---
 .../tableOps/ImportPopulateZookeeper.java       |  6 ++--
 .../accumulo/master/tableOps/ImportTable.java   | 12 +++----
 .../accumulo/master/tableOps/LoadFiles.java     |  4 +--
 .../master/tableOps/MapImportFileNames.java     |  4 +--
 .../master/tableOps/MoveExportedFiles.java      |  6 ++--
 .../master/tableOps/PopulateMetadataTable.java  |  8 ++---
 .../master/tableOps/RenameNamespace.java        |  4 +--
 .../accumulo/master/tableOps/RenameTable.java   |  6 ++--
 .../accumulo/master/tableOps/TableRangeOp.java  |  6 ++--
 .../apache/accumulo/master/tableOps/Utils.java  | 19 +++++------
 .../master/tableOps/WriteExportFiles.java       |  8 ++---
 20 files changed, 142 insertions(+), 68 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/bb031b34/core/src/main/java/org/apache/accumulo/core/client/impl/AcceptableThriftTableOperationException.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/AcceptableThriftTableOperationException.java b/core/src/main/java/org/apache/accumulo/core/client/impl/AcceptableThriftTableOperationException.java
new file mode 100644
index 0000000..98c1bf5
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/AcceptableThriftTableOperationException.java
@@ -0,0 +1,34 @@
+/*
+ * 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.core.client.impl;
+
+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.AcceptableException;
+
+/**
+ * Concrete implementation of {@link AcceptableException} for table operations.
+ */
+public class AcceptableThriftTableOperationException extends ThriftTableOperationException implements AcceptableException {
+
+  private static final long serialVersionUID = 1L;
+
+  public AcceptableThriftTableOperationException(String tableId, String tableName, TableOperation op, TableOperationExceptionType type, String description) {
+    super(tableId, tableName, op, type, description);
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bb031b34/fate/src/main/java/org/apache/accumulo/fate/AcceptableException.java
----------------------------------------------------------------------
diff --git a/fate/src/main/java/org/apache/accumulo/fate/AcceptableException.java b/fate/src/main/java/org/apache/accumulo/fate/AcceptableException.java
new file mode 100644
index 0000000..39683c1
--- /dev/null
+++ b/fate/src/main/java/org/apache/accumulo/fate/AcceptableException.java
@@ -0,0 +1,29 @@
+/*
+ * 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.fate;
+
+/**
+ * An exception for FATE operations to use to denote when an Exception is acceptable and should not trigger warning messages. This exception is intended to wrap
+ * an existing exception from a FATE op implementation so that the FATE runner can know that the exception doesn't need to warn.
+ * <p>
+ * Often times, problems that map well into the FATE execution model have states in which it is impossible to know ahead of time if an exception will be thrown.
+ * For example, with concurrent create table operations, one of the operations will fail because the table already exists, but this is not an error condition
+ * for the system. It is normal and expected.
+ */
+public interface AcceptableException {
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bb031b34/fate/src/main/java/org/apache/accumulo/fate/Fate.java
----------------------------------------------------------------------
diff --git a/fate/src/main/java/org/apache/accumulo/fate/Fate.java b/fate/src/main/java/org/apache/accumulo/fate/Fate.java
index cf2ab73..2d0dc1f 100644
--- a/fate/src/main/java/org/apache/accumulo/fate/Fate.java
+++ b/fate/src/main/java/org/apache/accumulo/fate/Fate.java
@@ -105,7 +105,14 @@ public class Fate<T> {
 
     private void transitionToFailed(long tid, Repo<T> op, Exception e) {
       String tidStr = String.format("%016x", tid);
-      log.warn("Failed to execute Repo, tid=" + tidStr, e);
+      final String msg = "Failed to execute Repo, tid=" + tidStr;
+      // Certain FATE ops that throw exceptions don't need to be propagated up to the Monitor
+      // as a warning. They're a normal, handled failure condition.
+      if (e instanceof AcceptableException) {
+        log.debug(msg, e.getCause());
+      } else {
+        log.warn(msg, e);
+      }
       store.setProperty(tid, EXCEPTION_PROP, e);
       store.setStatus(tid, TStatus.FAILED_IN_PROGRESS);
       log.info("Updated status for Repo with tid=" + tidStr + " to FAILED_IN_PROGRESS");

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bb031b34/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 a1ce1d4..c97e11a 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
@@ -529,7 +529,8 @@ class FateServiceHandler implements FateService.Iface {
       return VALID_ID.and(userValidator).validate(tableId);
     } catch (IllegalArgumentException e) {
       String why = e.getMessage();
-      log.warn(why);
+      // Information provided by a client should generate a user-level exception, not a system-level warning.
+      log.debug(why);
       throw new ThriftTableOperationException(tableId, null, op, TableOperationExceptionType.INVALID_NAME, why);
     }
   }
@@ -552,7 +553,8 @@ class FateServiceHandler implements FateService.Iface {
       return validator.validate(arg);
     } catch (IllegalArgumentException e) {
       String why = e.getMessage();
-      log.warn(why);
+      // Information provided by a client should generate a user-level exception, not a system-level warning.
+      log.debug(why);
       throw new ThriftTableOperationException(null, String.valueOf(arg), op, TableOperationExceptionType.INVALID_NAME, why);
     }
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bb031b34/server/master/src/main/java/org/apache/accumulo/master/tableOps/BulkImport.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/BulkImport.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/BulkImport.java
index 031a80c..eb19706 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/BulkImport.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/BulkImport.java
@@ -25,10 +25,10 @@ import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.impl.AcceptableThriftTableOperationException;
 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.core.conf.Property;
 import org.apache.accumulo.core.file.FileOperations;
 import org.apache.accumulo.core.master.state.tables.TableState;
@@ -99,7 +99,7 @@ public class BulkImport extends MasterRepo {
         reserve2 = Utils.reserveHdfsDirectory(errorDir, tid);
       return reserve2;
     } else {
-      throw new ThriftTableOperationException(tableId, null, TableOperation.BULK_IMPORT, TableOperationExceptionType.OFFLINE, null);
+      throw new AcceptableThriftTableOperationException(tableId, null, TableOperation.BULK_IMPORT, TableOperationExceptionType.OFFLINE, null);
     }
   }
 
@@ -120,14 +120,14 @@ public class BulkImport extends MasterRepo {
       // ignored
     }
     if (errorStatus == null)
-      throw new ThriftTableOperationException(tableId, null, TableOperation.BULK_IMPORT, TableOperationExceptionType.BULK_BAD_ERROR_DIRECTORY, errorDir
-          + " does not exist");
+      throw new AcceptableThriftTableOperationException(tableId, null, TableOperation.BULK_IMPORT, TableOperationExceptionType.BULK_BAD_ERROR_DIRECTORY,
+          errorDir + " does not exist");
     if (!errorStatus.isDirectory())
-      throw new ThriftTableOperationException(tableId, null, TableOperation.BULK_IMPORT, TableOperationExceptionType.BULK_BAD_ERROR_DIRECTORY, errorDir
-          + " is not a directory");
+      throw new AcceptableThriftTableOperationException(tableId, null, TableOperation.BULK_IMPORT, TableOperationExceptionType.BULK_BAD_ERROR_DIRECTORY,
+          errorDir + " is not a directory");
     if (fs.listStatus(errorPath).length != 0)
-      throw new ThriftTableOperationException(tableId, null, TableOperation.BULK_IMPORT, TableOperationExceptionType.BULK_BAD_ERROR_DIRECTORY, errorDir
-          + " is not empty");
+      throw new AcceptableThriftTableOperationException(tableId, null, TableOperation.BULK_IMPORT, TableOperationExceptionType.BULK_BAD_ERROR_DIRECTORY,
+          errorDir + " is not empty");
 
     ZooArbitrator.start(Constants.BULK_ARBITRATOR_TYPE, tid);
 
@@ -138,8 +138,8 @@ public class BulkImport extends MasterRepo {
       return new LoadFiles(tableId, sourceDir, bulkDir, errorDir, setTime);
     } catch (IOException ex) {
       log.error("error preparing the bulk import directory", ex);
-      throw new ThriftTableOperationException(tableId, null, TableOperation.BULK_IMPORT, TableOperationExceptionType.BULK_BAD_INPUT_DIRECTORY, sourceDir + ": "
-          + ex);
+      throw new AcceptableThriftTableOperationException(tableId, null, TableOperation.BULK_IMPORT, TableOperationExceptionType.BULK_BAD_INPUT_DIRECTORY,
+          sourceDir + ": " + ex);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bb031b34/server/master/src/main/java/org/apache/accumulo/master/tableOps/ClonePermissions.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/ClonePermissions.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/ClonePermissions.java
index cbcc708..fa58550 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/ClonePermissions.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/ClonePermissions.java
@@ -17,10 +17,10 @@
 package org.apache.accumulo.master.tableOps;
 
 import org.apache.accumulo.core.client.NamespaceNotFoundException;
+import org.apache.accumulo.core.client.impl.AcceptableThriftTableOperationException;
 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.ThriftSecurityException;
-import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException;
 import org.apache.accumulo.core.security.TablePermission;
 import org.apache.accumulo.fate.Repo;
 import org.apache.accumulo.master.Master;
@@ -61,7 +61,7 @@ class ClonePermissions extends MasterRepo {
     try {
       return new CloneZookeeper(cloneInfo);
     } catch (NamespaceNotFoundException e) {
-      throw new ThriftTableOperationException(null, cloneInfo.tableName, TableOperation.CLONE, TableOperationExceptionType.NAMESPACE_NOTFOUND,
+      throw new AcceptableThriftTableOperationException(null, cloneInfo.tableName, TableOperation.CLONE, TableOperationExceptionType.NAMESPACE_NOTFOUND,
           "Namespace for target table not found");
     }
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bb031b34/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..bae4f26 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
@@ -20,10 +20,10 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.client.impl.AcceptableThriftTableOperationException;
 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;
@@ -34,7 +34,7 @@ public class CloneTable extends MasterRepo {
   private CloneInfo cloneInfo;
 
   public CloneTable(String user, String srcTableId, String tableName, Map<String,String> propertiesToSet, Set<String> propertiesToExclude)
-      throws ThriftTableOperationException {
+      throws AcceptableThriftTableOperationException {
     cloneInfo = new CloneInfo();
     cloneInfo.user = user;
     cloneInfo.srcTableId = srcTableId;
@@ -49,7 +49,8 @@ public class CloneTable extends MasterRepo {
         // 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");
+      throw new AcceptableThriftTableOperationException(cloneInfo.srcTableId, "", TableOperation.CLONE, TableOperationExceptionType.NOTFOUND,
+          "Table does not exist");
     }
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bb031b34/server/master/src/main/java/org/apache/accumulo/master/tableOps/CompactRange.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/CompactRange.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/CompactRange.java
index 2a508c5..d9db5b2 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/CompactRange.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/CompactRange.java
@@ -23,11 +23,11 @@ import java.util.List;
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.admin.CompactionStrategyConfig;
+import org.apache.accumulo.core.client.impl.AcceptableThriftTableOperationException;
 import org.apache.accumulo.core.client.impl.CompactionStrategyConfigUtil;
 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.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter.Mutator;
@@ -53,7 +53,7 @@ public class CompactRange extends MasterRepo {
   private byte[] config;
 
   public CompactRange(String tableId, byte[] startRow, byte[] endRow, List<IteratorSetting> iterators, CompactionStrategyConfig compactionStrategy)
-      throws ThriftTableOperationException {
+      throws AcceptableThriftTableOperationException {
 
     Preconditions.checkNotNull(tableId, "Invalid argument: null tableId");
     Preconditions.checkNotNull(iterators, "Invalid argument: null iterator list");
@@ -70,7 +70,7 @@ public class CompactRange extends MasterRepo {
     }
 
     if (this.startRow != null && this.endRow != null && new Text(startRow).compareTo(new Text(endRow)) >= 0)
-      throw new ThriftTableOperationException(tableId, null, TableOperation.COMPACT, TableOperationExceptionType.BAD_RANGE,
+      throw new AcceptableThriftTableOperationException(tableId, null, TableOperation.COMPACT, TableOperationExceptionType.BAD_RANGE,
           "start row must be less than end row");
   }
 
@@ -105,7 +105,7 @@ public class CompactRange extends MasterRepo {
             log.debug("txidString : " + txidString);
             log.debug("tokens[" + i + "] : " + tokens[i]);
 
-            throw new ThriftTableOperationException(tableId, null, TableOperation.COMPACT, TableOperationExceptionType.OTHER,
+            throw new AcceptableThriftTableOperationException(tableId, null, TableOperation.COMPACT, TableOperationExceptionType.OTHER,
                 "Another compaction with iterators and/or a compaction strategy is running");
           }
 
@@ -125,7 +125,7 @@ public class CompactRange extends MasterRepo {
 
       return new CompactionDriver(Long.parseLong(new String(cid, UTF_8).split(",")[0]), tableId, startRow, endRow);
     } catch (NoNodeException nne) {
-      throw new ThriftTableOperationException(tableId, null, TableOperation.COMPACT, TableOperationExceptionType.NOTFOUND, null);
+      throw new AcceptableThriftTableOperationException(tableId, null, TableOperation.COMPACT, TableOperationExceptionType.NOTFOUND, null);
     }
 
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bb031b34/server/master/src/main/java/org/apache/accumulo/master/tableOps/CompactionDriver.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/CompactionDriver.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/CompactionDriver.java
index 0db93c1..f4fe037 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/CompactionDriver.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/CompactionDriver.java
@@ -26,10 +26,10 @@ import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.IsolatedScanner;
 import org.apache.accumulo.core.client.RowIterator;
 import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.impl.AcceptableThriftTableOperationException;
 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.core.data.Key;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
@@ -77,7 +77,7 @@ class CompactionDriver extends MasterRepo {
 
     if (Long.parseLong(new String(zoo.getData(zCancelID, null))) >= compactId) {
       // compaction was canceled
-      throw new ThriftTableOperationException(tableId, null, TableOperation.COMPACT, TableOperationExceptionType.OTHER, "Compaction canceled");
+      throw new AcceptableThriftTableOperationException(tableId, null, TableOperation.COMPACT, TableOperationExceptionType.OTHER, "Compaction canceled");
     }
 
     MapCounter<TServerInstance> serversToFlush = new MapCounter<TServerInstance>();
@@ -140,10 +140,10 @@ class CompactionDriver extends MasterRepo {
     Instance instance = master.getInstance();
     Tables.clearCache(instance);
     if (tabletCount == 0 && !Tables.exists(instance, tableId))
-      throw new ThriftTableOperationException(tableId, null, TableOperation.COMPACT, TableOperationExceptionType.NOTFOUND, null);
+      throw new AcceptableThriftTableOperationException(tableId, null, TableOperation.COMPACT, TableOperationExceptionType.NOTFOUND, null);
 
     if (serversToFlush.size() == 0 && Tables.getTableState(instance, tableId) == TableState.OFFLINE)
-      throw new ThriftTableOperationException(tableId, null, TableOperation.COMPACT, TableOperationExceptionType.OFFLINE, null);
+      throw new AcceptableThriftTableOperationException(tableId, null, TableOperation.COMPACT, TableOperationExceptionType.OFFLINE, null);
 
     if (tabletsToWaitFor == 0)
       return 0;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bb031b34/server/master/src/main/java/org/apache/accumulo/master/tableOps/ImportPopulateZookeeper.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/ImportPopulateZookeeper.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/ImportPopulateZookeeper.java
index 71e9124..e76dd09 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/ImportPopulateZookeeper.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/ImportPopulateZookeeper.java
@@ -22,12 +22,12 @@ import java.util.Map.Entry;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.client.impl.AcceptableThriftTableOperationException;
 import org.apache.accumulo.core.client.impl.Namespaces;
 import org.apache.accumulo.core.client.impl.TableOperationsImpl;
 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.fate.zookeeper.ZooUtil.NodeExistsPolicy;
 import org.apache.accumulo.master.Master;
@@ -60,7 +60,7 @@ class ImportPopulateZookeeper extends MasterRepo {
       FileSystem ns = fs.getVolumeByPath(path).getFileSystem();
       return TableOperationsImpl.getExportedProps(ns, path);
     } catch (IOException ioe) {
-      throw new ThriftTableOperationException(tableInfo.tableId, tableInfo.tableName, TableOperation.IMPORT, TableOperationExceptionType.OTHER,
+      throw new AcceptableThriftTableOperationException(tableInfo.tableId, tableInfo.tableName, TableOperation.IMPORT, TableOperationExceptionType.OTHER,
           "Error reading table props from " + path + " " + ioe.getMessage());
     }
   }
@@ -87,7 +87,7 @@ class ImportPopulateZookeeper extends MasterRepo {
 
     for (Entry<String,String> entry : getExportedProps(env.getFileSystem()).entrySet())
       if (!TablePropUtil.setTableProperty(tableInfo.tableId, entry.getKey(), entry.getValue())) {
-        throw new ThriftTableOperationException(tableInfo.tableId, tableInfo.tableName, TableOperation.IMPORT, TableOperationExceptionType.OTHER,
+        throw new AcceptableThriftTableOperationException(tableInfo.tableId, tableInfo.tableName, TableOperation.IMPORT, TableOperationExceptionType.OTHER,
             "Invalid table property " + entry.getKey());
       }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bb031b34/server/master/src/main/java/org/apache/accumulo/master/tableOps/ImportTable.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/ImportTable.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/ImportTable.java
index dc33303..b9b5327 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/ImportTable.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/ImportTable.java
@@ -26,9 +26,9 @@ import java.util.zip.ZipInputStream;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.client.impl.AcceptableThriftTableOperationException;
 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.ServerConstants;
@@ -77,7 +77,7 @@ public class ImportTable extends MasterRepo {
     }
   }
 
-  public void checkVersions(Master env) throws ThriftTableOperationException {
+  public void checkVersions(Master env) throws AcceptableThriftTableOperationException {
     Path path = new Path(tableInfo.exportDir, Constants.EXPORT_FILE);
     Integer exportVersion = null;
     Integer dataVersion = null;
@@ -101,17 +101,17 @@ public class ImportTable extends MasterRepo {
       }
     } catch (IOException ioe) {
       log.warn("{}", ioe.getMessage(), ioe);
-      throw new ThriftTableOperationException(null, tableInfo.tableName, TableOperation.IMPORT, TableOperationExceptionType.OTHER,
+      throw new AcceptableThriftTableOperationException(null, tableInfo.tableName, TableOperation.IMPORT, TableOperationExceptionType.OTHER,
           "Failed to read export metadata " + ioe.getMessage());
     }
 
     if (exportVersion == null || exportVersion > ExportTable.VERSION)
-      throw new ThriftTableOperationException(null, tableInfo.tableName, TableOperation.IMPORT, TableOperationExceptionType.OTHER,
+      throw new AcceptableThriftTableOperationException(null, tableInfo.tableName, TableOperation.IMPORT, TableOperationExceptionType.OTHER,
           "Incompatible export version " + exportVersion);
 
     if (dataVersion == null || dataVersion > ServerConstants.DATA_VERSION)
-      throw new ThriftTableOperationException(null, tableInfo.tableName, TableOperation.IMPORT, TableOperationExceptionType.OTHER, "Incompatible data version "
-          + exportVersion);
+      throw new AcceptableThriftTableOperationException(null, tableInfo.tableName, TableOperation.IMPORT, TableOperationExceptionType.OTHER,
+          "Incompatible data version " + exportVersion);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bb031b34/server/master/src/main/java/org/apache/accumulo/master/tableOps/LoadFiles.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/LoadFiles.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/LoadFiles.java
index 4a56c6f..9e72eab 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/LoadFiles.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/LoadFiles.java
@@ -31,12 +31,12 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Future;
 import java.util.concurrent.ThreadPoolExecutor;
 
+import org.apache.accumulo.core.client.impl.AcceptableThriftTableOperationException;
 import org.apache.accumulo.core.client.impl.ServerClient;
 import org.apache.accumulo.core.client.impl.thrift.ClientService;
 import org.apache.accumulo.core.client.impl.thrift.ClientService.Client;
 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.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.trace.Tracer;
@@ -107,7 +107,7 @@ class LoadFiles extends MasterRepo {
       // Maybe this is a re-try... clear the flag and try again
       fs.delete(writable);
       if (!fs.createNewFile(writable))
-        throw new ThriftTableOperationException(tableId, null, TableOperation.BULK_IMPORT, TableOperationExceptionType.BULK_BAD_ERROR_DIRECTORY,
+        throw new AcceptableThriftTableOperationException(tableId, null, TableOperation.BULK_IMPORT, TableOperationExceptionType.BULK_BAD_ERROR_DIRECTORY,
             "Unable to write to " + this.errorDir);
     }
     fs.delete(writable);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bb031b34/server/master/src/main/java/org/apache/accumulo/master/tableOps/MapImportFileNames.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/MapImportFileNames.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/MapImportFileNames.java
index 4a43c68..06bd86a 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/MapImportFileNames.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/MapImportFileNames.java
@@ -23,9 +23,9 @@ import java.io.IOException;
 import java.io.OutputStreamWriter;
 
 import org.apache.accumulo.core.Constants;
+import org.apache.accumulo.core.client.impl.AcceptableThriftTableOperationException;
 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.core.file.FileOperations;
 import org.apache.accumulo.fate.Repo;
 import org.apache.accumulo.master.Master;
@@ -95,7 +95,7 @@ class MapImportFileNames extends MasterRepo {
       return new PopulateMetadataTable(tableInfo);
     } catch (IOException ioe) {
       log.warn("{}", ioe.getMessage(), ioe);
-      throw new ThriftTableOperationException(tableInfo.tableId, tableInfo.tableName, TableOperation.IMPORT, TableOperationExceptionType.OTHER,
+      throw new AcceptableThriftTableOperationException(tableInfo.tableId, tableInfo.tableName, TableOperation.IMPORT, TableOperationExceptionType.OTHER,
           "Error writing mapping file " + path + " " + ioe.getMessage());
     } finally {
       if (mappingsWriter != null)

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bb031b34/server/master/src/main/java/org/apache/accumulo/master/tableOps/MoveExportedFiles.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/MoveExportedFiles.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/MoveExportedFiles.java
index c1bcc49..d6eb7be 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/MoveExportedFiles.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/MoveExportedFiles.java
@@ -19,9 +19,9 @@ package org.apache.accumulo.master.tableOps;
 import java.io.IOException;
 import java.util.Map;
 
+import org.apache.accumulo.core.client.impl.AcceptableThriftTableOperationException;
 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.fs.VolumeManager;
@@ -50,7 +50,7 @@ class MoveExportedFiles extends MasterRepo {
 
       for (String oldFileName : fileNameMappings.keySet()) {
         if (!fs.exists(new Path(tableInfo.exportDir, oldFileName))) {
-          throw new ThriftTableOperationException(tableInfo.tableId, tableInfo.tableName, TableOperation.IMPORT, TableOperationExceptionType.OTHER,
+          throw new AcceptableThriftTableOperationException(tableInfo.tableId, tableInfo.tableName, TableOperation.IMPORT, TableOperationExceptionType.OTHER,
               "File referenced by exported table does not exists " + oldFileName);
         }
       }
@@ -67,7 +67,7 @@ class MoveExportedFiles extends MasterRepo {
       return new FinishImportTable(tableInfo);
     } catch (IOException ioe) {
       log.warn("{}", ioe.getMessage(), ioe);
-      throw new ThriftTableOperationException(tableInfo.tableId, tableInfo.tableName, TableOperation.IMPORT, TableOperationExceptionType.OTHER,
+      throw new AcceptableThriftTableOperationException(tableInfo.tableId, tableInfo.tableName, TableOperation.IMPORT, TableOperationExceptionType.OTHER,
           "Error renaming files " + ioe.getMessage());
     }
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bb031b34/server/master/src/main/java/org/apache/accumulo/master/tableOps/PopulateMetadataTable.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/PopulateMetadataTable.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/PopulateMetadataTable.java
index e35f01a..6e19b61 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/PopulateMetadataTable.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/PopulateMetadataTable.java
@@ -31,9 +31,9 @@ import java.util.zip.ZipInputStream;
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
+import org.apache.accumulo.core.client.impl.AcceptableThriftTableOperationException;
 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.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
@@ -134,8 +134,8 @@ class PopulateMetadataTable extends MasterRepo {
               String newName = fileNameMappings.get(oldName);
 
               if (newName == null) {
-                throw new ThriftTableOperationException(tableInfo.tableId, tableInfo.tableName, TableOperation.IMPORT, TableOperationExceptionType.OTHER,
-                    "File " + oldName + " does not exist in import dir");
+                throw new AcceptableThriftTableOperationException(tableInfo.tableId, tableInfo.tableName, TableOperation.IMPORT,
+                    TableOperationExceptionType.OTHER, "File " + oldName + " does not exist in import dir");
               }
 
               cq = new Text(bulkDir + "/" + newName);
@@ -183,7 +183,7 @@ class PopulateMetadataTable extends MasterRepo {
       return new MoveExportedFiles(tableInfo);
     } catch (IOException ioe) {
       log.warn("{}", ioe.getMessage(), ioe);
-      throw new ThriftTableOperationException(tableInfo.tableId, tableInfo.tableName, TableOperation.IMPORT, TableOperationExceptionType.OTHER,
+      throw new AcceptableThriftTableOperationException(tableInfo.tableId, tableInfo.tableName, TableOperation.IMPORT, TableOperationExceptionType.OTHER,
           "Error reading " + path + " " + ioe.getMessage());
     } finally {
       if (zis != null) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bb031b34/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameNamespace.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameNamespace.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameNamespace.java
index 1f09db0..5373b94 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameNamespace.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameNamespace.java
@@ -18,10 +18,10 @@ package org.apache.accumulo.master.tableOps;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.client.impl.AcceptableThriftTableOperationException;
 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.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.Repo;
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
@@ -68,7 +68,7 @@ public class RenameNamespace extends MasterRepo {
           if (currentName.equals(newName))
             return null; // assume in this case the operation is running again, so we are done
           if (!currentName.equals(oldName)) {
-            throw new ThriftTableOperationException(null, oldName, TableOperation.RENAME, TableOperationExceptionType.NAMESPACE_NOTFOUND,
+            throw new AcceptableThriftTableOperationException(null, oldName, TableOperation.RENAME, TableOperationExceptionType.NAMESPACE_NOTFOUND,
                 "Name changed while processing");
           }
           return newName.getBytes();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bb031b34/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTable.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTable.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTable.java
index 053749f..f85d411 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTable.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTable.java
@@ -21,11 +21,11 @@ import static java.nio.charset.StandardCharsets.UTF_8;
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.NamespaceNotFoundException;
+import org.apache.accumulo.core.client.impl.AcceptableThriftTableOperationException;
 import org.apache.accumulo.core.client.impl.Namespaces;
 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.core.util.Pair;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.Repo;
@@ -63,7 +63,7 @@ public class RenameTable extends MasterRepo {
 
     // ensure no attempt is made to rename across namespaces
     if (newTableName.contains(".") && !namespaceId.equals(Namespaces.getNamespaceId(instance, qualifiedNewTableName.getFirst())))
-      throw new ThriftTableOperationException(tableId, oldTableName, TableOperation.RENAME, TableOperationExceptionType.INVALID_NAME,
+      throw new AcceptableThriftTableOperationException(tableId, oldTableName, TableOperation.RENAME, TableOperationExceptionType.INVALID_NAME,
           "Namespace in new table name does not match the old table name");
 
     IZooReaderWriter zoo = ZooReaderWriter.getInstance();
@@ -84,7 +84,7 @@ public class RenameTable extends MasterRepo {
           if (currentName.equals(newName))
             return null; // assume in this case the operation is running again, so we are done
           if (!currentName.equals(oldName)) {
-            throw new ThriftTableOperationException(null, oldTableName, TableOperation.RENAME, TableOperationExceptionType.NOTFOUND,
+            throw new AcceptableThriftTableOperationException(null, oldTableName, TableOperation.RENAME, TableOperationExceptionType.NOTFOUND,
                 "Name changed while processing");
           }
           return newName.getBytes(UTF_8);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bb031b34/server/master/src/main/java/org/apache/accumulo/master/tableOps/TableRangeOp.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/TableRangeOp.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/TableRangeOp.java
index 879470b..ca8a171 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/TableRangeOp.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/TableRangeOp.java
@@ -16,10 +16,10 @@
  */
 package org.apache.accumulo.master.tableOps;
 
+import org.apache.accumulo.core.client.impl.AcceptableThriftTableOperationException;
 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.core.data.impl.KeyExtent;
 import org.apache.accumulo.core.metadata.RootTable;
 import org.apache.accumulo.core.util.TextUtil;
@@ -48,7 +48,7 @@ public class TableRangeOp extends MasterRepo {
     return Utils.reserveNamespace(namespaceId, tid, false, true, TableOperation.MERGE) + Utils.reserveTable(tableId, tid, true, true, TableOperation.MERGE);
   }
 
-  public TableRangeOp(MergeInfo.Operation op, String tableId, Text startRow, Text endRow) throws ThriftTableOperationException {
+  public TableRangeOp(MergeInfo.Operation op, String tableId, Text startRow, Text endRow) throws AcceptableThriftTableOperationException {
 
     this.tableId = tableId;
     this.startRow = TextUtil.getBytes(startRow);
@@ -69,7 +69,7 @@ public class TableRangeOp extends MasterRepo {
 
     if (start != null && end != null)
       if (start.compareTo(end) >= 0)
-        throw new ThriftTableOperationException(tableId, null, TableOperation.MERGE, TableOperationExceptionType.BAD_RANGE,
+        throw new AcceptableThriftTableOperationException(tableId, null, TableOperation.MERGE, TableOperationExceptionType.BAD_RANGE,
             "start row must be less than end row");
 
     env.mustBeOnline(tableId);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bb031b34/server/master/src/main/java/org/apache/accumulo/master/tableOps/Utils.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/Utils.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/Utils.java
index 0fb9138..2baf7ac 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/Utils.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/Utils.java
@@ -24,11 +24,11 @@ import java.util.concurrent.locks.ReentrantLock;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.client.impl.AcceptableThriftTableOperationException;
 import org.apache.accumulo.core.client.impl.Namespaces;
 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.core.util.Base64;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.DistributedReadWriteLock;
@@ -46,15 +46,16 @@ public class Utils {
   private static final byte[] ZERO_BYTE = new byte[] {'0'};
   private static final Logger log = LoggerFactory.getLogger(Utils.class);
 
-  static void checkTableDoesNotExist(Instance instance, String tableName, String tableId, TableOperation operation) throws ThriftTableOperationException {
+  static void checkTableDoesNotExist(Instance instance, String tableName, String tableId, TableOperation operation)
+      throws AcceptableThriftTableOperationException {
 
     String id = Tables.getNameToIdMap(instance).get(tableName);
 
     if (id != null && !id.equals(tableId))
-      throw new ThriftTableOperationException(null, tableName, operation, TableOperationExceptionType.EXISTS, null);
+      throw new AcceptableThriftTableOperationException(null, tableName, operation, TableOperationExceptionType.EXISTS, null);
   }
 
-  static String getNextTableId(String tableName, Instance instance) throws ThriftTableOperationException {
+  static String getNextTableId(String tableName, Instance instance) throws AcceptableThriftTableOperationException {
 
     String tableId = null;
     try {
@@ -71,7 +72,7 @@ public class Utils {
       return new String(nid, UTF_8);
     } catch (Exception e1) {
       log.error("Failed to assign tableId to " + tableName, e1);
-      throw new ThriftTableOperationException(tableId, tableName, TableOperation.CREATE, TableOperationExceptionType.OTHER, e1.getMessage());
+      throw new AcceptableThriftTableOperationException(tableId, tableName, TableOperation.CREATE, TableOperationExceptionType.OTHER, e1.getMessage());
     }
   }
 
@@ -84,7 +85,7 @@ public class Utils {
         Instance instance = HdfsZooInstance.getInstance();
         IZooReaderWriter zk = ZooReaderWriter.getInstance();
         if (!zk.exists(ZooUtil.getRoot(instance) + Constants.ZTABLES + "/" + tableId))
-          throw new ThriftTableOperationException(tableId, "", op, TableOperationExceptionType.NOTFOUND, "Table does not exist");
+          throw new AcceptableThriftTableOperationException(tableId, "", op, TableOperationExceptionType.NOTFOUND, "Table does not exist");
       }
       log.info("table " + tableId + " (" + Long.toHexString(tid) + ") locked for " + (writeLock ? "write" : "read") + " operation: " + op);
       return 0;
@@ -108,7 +109,7 @@ public class Utils {
         Instance instance = HdfsZooInstance.getInstance();
         IZooReaderWriter zk = ZooReaderWriter.getInstance();
         if (!zk.exists(ZooUtil.getRoot(instance) + Constants.ZNAMESPACES + "/" + namespaceId))
-          throw new ThriftTableOperationException(namespaceId, "", op, TableOperationExceptionType.NAMESPACE_NOTFOUND, "Namespace does not exist");
+          throw new AcceptableThriftTableOperationException(namespaceId, "", op, TableOperationExceptionType.NAMESPACE_NOTFOUND, "Namespace does not exist");
       }
       log.info("namespace " + namespaceId + " (" + Long.toHexString(id) + ") locked for " + (writeLock ? "write" : "read") + " operation: " + op);
       return 0;
@@ -154,11 +155,11 @@ public class Utils {
   }
 
   static void checkNamespaceDoesNotExist(Instance instance, String namespace, String namespaceId, TableOperation operation)
-      throws ThriftTableOperationException {
+      throws AcceptableThriftTableOperationException {
 
     String n = Namespaces.getNameToIdMap(instance).get(namespace);
 
     if (n != null && !n.equals(namespaceId))
-      throw new ThriftTableOperationException(null, namespace, operation, TableOperationExceptionType.NAMESPACE_EXISTS, null);
+      throw new AcceptableThriftTableOperationException(null, namespace, operation, TableOperationExceptionType.NAMESPACE_EXISTS, null);
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bb031b34/server/master/src/main/java/org/apache/accumulo/master/tableOps/WriteExportFiles.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/WriteExportFiles.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/WriteExportFiles.java
index a492957..78ed2c0 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/WriteExportFiles.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/WriteExportFiles.java
@@ -35,10 +35,10 @@ import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.impl.AcceptableThriftTableOperationException;
 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.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
 import org.apache.accumulo.core.conf.Property;
@@ -74,7 +74,7 @@ class WriteExportFiles extends MasterRepo {
     if (Tables.getTableState(conn.getInstance(), tableInfo.tableID) != TableState.OFFLINE) {
       Tables.clearCache(conn.getInstance());
       if (Tables.getTableState(conn.getInstance(), tableInfo.tableID) != TableState.OFFLINE) {
-        throw new ThriftTableOperationException(tableInfo.tableID, tableInfo.tableName, TableOperation.EXPORT, TableOperationExceptionType.OTHER,
+        throw new AcceptableThriftTableOperationException(tableInfo.tableID, tableInfo.tableName, TableOperation.EXPORT, TableOperationExceptionType.OTHER,
             "Table is not offline");
       }
     }
@@ -109,7 +109,7 @@ class WriteExportFiles extends MasterRepo {
     metaScanner.fetchColumnFamily(LogColumnFamily.NAME);
 
     if (metaScanner.iterator().hasNext()) {
-      throw new ThriftTableOperationException(tableInfo.tableID, tableInfo.tableName, TableOperation.EXPORT, TableOperationExceptionType.OTHER,
+      throw new AcceptableThriftTableOperationException(tableInfo.tableID, tableInfo.tableName, TableOperation.EXPORT, TableOperationExceptionType.OTHER,
           "Write ahead logs found for table");
     }
 
@@ -121,7 +121,7 @@ class WriteExportFiles extends MasterRepo {
     try {
       exportTable(master.getFileSystem(), master, tableInfo.tableName, tableInfo.tableID, tableInfo.exportDir);
     } catch (IOException ioe) {
-      throw new ThriftTableOperationException(tableInfo.tableID, tableInfo.tableName, TableOperation.EXPORT, TableOperationExceptionType.OTHER,
+      throw new AcceptableThriftTableOperationException(tableInfo.tableID, tableInfo.tableName, TableOperation.EXPORT, TableOperationExceptionType.OTHER,
           "Failed to create export files " + ioe.getMessage());
     }
     Utils.unreserveNamespace(tableInfo.namespaceID, tid, false);


[2/2] accumulo git commit: ACCUMULO-3927 Commit checkstyle changes

Posted by el...@apache.org.
ACCUMULO-3927 Commit checkstyle changes


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

Branch: refs/heads/master
Commit: eb1ba18607e1afd3d15a7ba86f968fe47e53d7b8
Parents: bb031b3
Author: Josh Elser <el...@apache.org>
Authored: Thu Jul 2 13:27:27 2015 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Thu Jul 2 13:27:27 2015 -0400

----------------------------------------------------------------------
 .../org/apache/accumulo/tserver/tablet/SplitInfo.java    |  6 +++---
 .../java/org/apache/accumulo/tserver/tablet/Tablet.java  | 11 ++++++-----
 .../main/java/org/apache/accumulo/test/ManySplitIT.java  |  5 ++---
 3 files changed, 11 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/eb1ba186/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/SplitInfo.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/SplitInfo.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/SplitInfo.java
index 64b6a11..3cae52c 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/SplitInfo.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/SplitInfo.java
@@ -41,10 +41,10 @@ final public class SplitInfo {
   private final long initFlushID;
   private final long initCompactID;
   private final TServerInstance lastLocation;
-  private final Map<Long, ? extends Collection<FileRef>> bulkImported;
+  private final Map<Long,? extends Collection<FileRef>> bulkImported;
 
   SplitInfo(String d, SortedMap<FileRef,DataFileValue> dfv, String time, long initFlushID, long initCompactID, TServerInstance lastLocation,
-      Map<Long, ? extends Collection<FileRef>> bulkImported) {
+      Map<Long,? extends Collection<FileRef>> bulkImported) {
     this.dir = d;
     this.datafiles = dfv;
     this.time = time;
@@ -78,7 +78,7 @@ final public class SplitInfo {
     return lastLocation;
   }
 
-  public Map<Long, ? extends Collection<FileRef>> getBulkImported() {
+  public Map<Long,? extends Collection<FileRef>> getBulkImported() {
     return bulkImported;
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/eb1ba186/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
index c0fb918..a40c95d 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
@@ -322,7 +322,7 @@ public class Tablet implements TabletCommitter {
   }
 
   private Tablet(TabletServer tabletServer, Text location, KeyExtent extent, TabletResourceManager trm, SortedMap<FileRef,DataFileValue> datafiles,
-      String time, long initFlushID, long initCompactID, TServerInstance lastLocation, Map<Long, ? extends Collection<FileRef>> bulkImported) throws IOException {
+      String time, long initFlushID, long initCompactID, TServerInstance lastLocation, Map<Long,? extends Collection<FileRef>> bulkImported) throws IOException {
     this(tabletServer, extent, location, trm, NO_LOG_ENTRIES, datafiles, time, lastLocation, new HashSet<FileRef>(), initFlushID, initCompactID, bulkImported);
   }
 
@@ -445,7 +445,7 @@ public class Tablet implements TabletCommitter {
     return null;
   }
 
-  private static Map<Long, List<FileRef>> lookupBulkImported(SortedMap<Key,Value> tabletsKeyValues, VolumeManager fs) {
+  private static Map<Long,List<FileRef>> lookupBulkImported(SortedMap<Key,Value> tabletsKeyValues, VolumeManager fs) {
     Map<Long,List<FileRef>> result = new HashMap<>();
     for (Entry<Key,Value> entry : tabletsKeyValues.entrySet()) {
       if (entry.getKey().getColumnFamily().compareTo(BulkFileColumnFamily.NAME) == 0) {
@@ -473,7 +473,8 @@ public class Tablet implements TabletCommitter {
    */
   private Tablet(final TabletServer tabletServer, final KeyExtent extent, final Text location, final TabletResourceManager trm,
       final List<LogEntry> rawLogEntries, final SortedMap<FileRef,DataFileValue> rawDatafiles, String time, final TServerInstance lastLocation,
-      final Set<FileRef> scanFiles, final long initFlushID, final long initCompactID, final Map<Long, ? extends Collection<FileRef>> bulkImported) throws IOException {
+      final Set<FileRef> scanFiles, final long initFlushID, final long initCompactID, final Map<Long,? extends Collection<FileRef>> bulkImported)
+      throws IOException {
 
     TableConfiguration tblConf = tabletServer.getTableConfiguration(extent);
     if (null == tblConf) {
@@ -2297,8 +2298,8 @@ public class Tablet implements TabletCommitter {
       String time = tabletTime.getMetadataValue();
 
       MetadataTableUtil.splitTablet(high, extent.getPrevEndRow(), splitRatio, getTabletServer(), getTabletServer().getLock());
-      MasterMetadataUtil.addNewTablet(getTabletServer(), low, lowDirectory, getTabletServer().getTabletSession(), lowDatafileSizes, getBulkIngestedFiles(), time,
-          lastFlushID, lastCompactID, getTabletServer().getLock());
+      MasterMetadataUtil.addNewTablet(getTabletServer(), low, lowDirectory, getTabletServer().getTabletSession(), lowDatafileSizes, getBulkIngestedFiles(),
+          time, lastFlushID, lastCompactID, getTabletServer().getLock());
       MetadataTableUtil.finishSplit(high, highDatafileSizes, highDatafilesToRemove, getTabletServer(), getTabletServer().getLock());
 
       log.log(TLevel.TABLET_HIST, extent + " split " + low + " " + high);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/eb1ba186/test/src/main/java/org/apache/accumulo/test/ManySplitIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/ManySplitIT.java b/test/src/main/java/org/apache/accumulo/test/ManySplitIT.java
index dc61f53..5676831 100644
--- a/test/src/main/java/org/apache/accumulo/test/ManySplitIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/ManySplitIT.java
@@ -37,7 +37,6 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Text;
 import org.junit.Test;
 
-
 public class ManySplitIT extends ConfigurableMacBase {
 
   final int SPLITS = 10_000;
@@ -53,7 +52,7 @@ public class ManySplitIT extends ConfigurableMacBase {
     tableOperations.create(tableName);
     SortedSet<Text> splits = new TreeSet<Text>();
     for (byte b : "123456789abcde".getBytes(UTF_8)) {
-      splits.add(new Text(new byte[]{'1', ';', b}));
+      splits.add(new Text(new byte[] {'1', ';', b}));
     }
     tableOperations.addSplits(MetadataTable.NAME, splits);
     splits.clear();
@@ -69,7 +68,7 @@ public class ManySplitIT extends ConfigurableMacBase {
         while (!stop.get()) {
           UtilWaitThread.sleep(1000);
           try {
-            log.info("splits: " + tableOperations.listSplits(tableName).size() );
+            log.info("splits: " + tableOperations.listSplits(tableName).size());
           } catch (TableNotFoundException | AccumuloException | AccumuloSecurityException e) {
             // TODO Auto-generated catch block
             e.printStackTrace();