Return-Path: X-Original-To: apmail-accumulo-commits-archive@www.apache.org Delivered-To: apmail-accumulo-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 762741888C for ; Thu, 2 Jul 2015 17:28:44 +0000 (UTC) Received: (qmail 68209 invoked by uid 500); 2 Jul 2015 17:28:39 -0000 Delivered-To: apmail-accumulo-commits-archive@accumulo.apache.org Received: (qmail 68172 invoked by uid 500); 2 Jul 2015 17:28:39 -0000 Mailing-List: contact commits-help@accumulo.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@accumulo.apache.org Delivered-To: mailing list commits@accumulo.apache.org Received: (qmail 68135 invoked by uid 99); 2 Jul 2015 17:28:39 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 02 Jul 2015 17:28:39 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 7DB4BE365B; Thu, 2 Jul 2015 17:28:39 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: elserj@apache.org To: commits@accumulo.apache.org Date: Thu, 02 Jul 2015 17:28:39 -0000 Message-Id: <000f3e4a211b4571a8d89fb358ba969b@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [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 Authored: Mon Jun 29 13:24:31 2015 -0400 Committer: Josh Elser 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. + *

+ * 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 { private void transitionToFailed(long tid, Repo 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 propertiesToSet, Set 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 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 serversToFlush = new MapCounter(); @@ -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 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);