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 4E61B10B0B for ; Sat, 28 Dec 2013 01:07:35 +0000 (UTC) Received: (qmail 99490 invoked by uid 500); 28 Dec 2013 01:07:35 -0000 Delivered-To: apmail-accumulo-commits-archive@accumulo.apache.org Received: (qmail 99330 invoked by uid 500); 28 Dec 2013 01:07:35 -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 99220 invoked by uid 99); 28 Dec 2013 01:07:34 -0000 Received: from tyr.zones.apache.org (HELO tyr.zones.apache.org) (140.211.11.114) by apache.org (qpsmtpd/0.29) with ESMTP; Sat, 28 Dec 2013 01:07:34 +0000 Received: by tyr.zones.apache.org (Postfix, from userid 65534) id 8880B8804AB; Sat, 28 Dec 2013 01:07:34 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: ctubbsii@apache.org To: commits@accumulo.apache.org Date: Sat, 28 Dec 2013 01:07:40 -0000 Message-Id: <22faee8c0ff8441696473763814b19ac@git.apache.org> In-Reply-To: <4e6dc8b91560437babb274384cd09b07@git.apache.org> References: <4e6dc8b91560437babb274384cd09b07@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [7/8] git commit: ACCUMULO-1965 Fix exception handling for namespaces ACCUMULO-1965 Fix exception handling for namespaces Make the exception handling propagate correctly to the client. Vastly expanded on existing namespace tests to include checks for throwing the correct exceptions. Consolidated a lot of the fate operations code and refactored the master (slightly; moved inner classes to separate files) to be easier to modify the relevant RPC handling code. Fixed many API bugs related to throwing correct exceptions, found from the new tests added to NamespacesIT. Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/f35e3f47 Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/f35e3f47 Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/f35e3f47 Branch: refs/heads/master Commit: f35e3f47265868c560b3d49edebf1a8c24196512 Parents: c41900e Author: Christopher Tubbs Authored: Tue Dec 24 00:06:12 2013 -0500 Committer: Christopher Tubbs Committed: Fri Dec 27 19:55:34 2013 -0500 ---------------------------------------------------------------------- .../org/apache/accumulo/core/Constants.java | 4 - .../core/client/TableNotFoundException.java | 21 +- .../core/client/admin/NamespaceOperations.java | 2 +- .../client/admin/NamespaceOperationsImpl.java | 193 +- .../client/admin/TableOperationsHelper.java | 35 +- .../core/client/admin/TableOperationsImpl.java | 256 +- .../core/client/impl/ConnectorImpl.java | 55 +- .../accumulo/core/client/impl/MasterClient.java | 76 +- .../accumulo/core/client/impl/Namespaces.java | 33 +- .../accumulo/core/client/impl/Tables.java | 32 +- .../thrift/TableOperationExceptionType.java | 11 +- .../accumulo/core/client/mock/MockAccumulo.java | 6 +- .../core/client/mock/MockTableOperations.java | 12 +- .../core/master/thrift/FateOperation.java | 103 + .../core/master/thrift/FateService.java | 5017 +++++++++ .../core/master/thrift/MasterClientService.java | 9946 +----------------- .../core/master/thrift/TableOperation.java | 94 - .../accumulo/core/metadata/MetadataTable.java | 4 +- .../accumulo/core/metadata/RootTable.java | 4 +- .../util/shell/commands/NamespacesCommand.java | 4 +- core/src/main/thrift/client.thrift | 203 +- core/src/main/thrift/master.thrift | 222 +- .../core/client/mock/MockNamespacesTest.java | 5 +- .../client/mock/MockTableOperationsTest.java | 49 +- .../server/client/ClientServiceHandler.java | 37 +- .../server/conf/NamespaceConfiguration.java | 5 +- .../apache/accumulo/server/init/Initialize.java | 9 +- .../accumulo/server/master/LiveTServerSet.java | 114 +- .../security/AuditedSecurityOperation.java | 8 +- .../server/security/SecurityOperation.java | 6 +- .../server/security/handler/ZKPermHandler.java | 6 +- .../accumulo/master/FateServiceHandler.java | 412 + .../java/org/apache/accumulo/master/Master.java | 852 +- .../master/MasterClientServiceHandler.java | 463 + .../accumulo/master/tableOps/CloneTable.java | 10 +- .../master/tableOps/RenameNamespace.java | 3 +- .../accumulo/master/tableOps/RenameTable.java | 3 +- .../apache/accumulo/master/tableOps/Utils.java | 4 +- .../test/randomwalk/concurrent/Config.java | 4 +- .../org/apache/accumulo/test/NamespacesIT.java | 1202 ++- .../org/apache/accumulo/test/ShellServerIT.java | 6 +- 41 files changed, 7683 insertions(+), 11848 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/accumulo/blob/f35e3f47/core/src/main/java/org/apache/accumulo/core/Constants.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/Constants.java b/core/src/main/java/org/apache/accumulo/core/Constants.java index 75452e3..c151abb 100644 --- a/core/src/main/java/org/apache/accumulo/core/Constants.java +++ b/core/src/main/java/org/apache/accumulo/core/Constants.java @@ -113,8 +113,4 @@ public class Constants { // Variables that will be substituted with environment vars in PropertyType.PATH values public static final String[] PATH_PROPERTY_ENV_VARS = new String[] {"ACCUMULO_HOME", "ACCUMULO_CONF_DIR"}; - public static final String DEFAULT_NAMESPACE = ""; - public static final String ACCUMULO_NAMESPACE = "accumulo"; - public static final String DEFAULT_NAMESPACE_ID = "+default"; - public static final String ACCUMULO_NAMESPACE_ID = "+accumulo"; } http://git-wip-us.apache.org/repos/asf/accumulo/blob/f35e3f47/core/src/main/java/org/apache/accumulo/core/client/TableNotFoundException.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/client/TableNotFoundException.java b/core/src/main/java/org/apache/accumulo/core/client/TableNotFoundException.java index 88ab449..6d27336 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/TableNotFoundException.java +++ b/core/src/main/java/org/apache/accumulo/core/client/TableNotFoundException.java @@ -16,6 +16,7 @@ */ package org.apache.accumulo.core.client; +import org.apache.accumulo.core.client.impl.Tables; import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException; /** @@ -27,9 +28,9 @@ public class TableNotFoundException extends Exception { * */ private static final long serialVersionUID = 1L; - + private String tableName; - + /** * @param tableId * the internal id of the table that was sought @@ -43,7 +44,7 @@ public class TableNotFoundException extends Exception { + " does not exist" + (description != null && !description.isEmpty() ? " (" + description + ")" : "")); this.tableName = tableName; } - + /** * @param tableId * the internal id of the table that was sought @@ -58,7 +59,7 @@ public class TableNotFoundException extends Exception { this(tableId, tableName, description); super.initCause(cause); } - + /** * @param e * constructs an exception from a thrift exception @@ -66,7 +67,17 @@ public class TableNotFoundException extends Exception { public TableNotFoundException(ThriftTableOperationException e) { this(e.getTableId(), e.getTableName(), e.getDescription(), e); } - + + /** + * @param tableName + * the original specified table + * @param e + * indicates that a table wasn't found because the namespace specified in the table name wasn't found + */ + public TableNotFoundException(String tableName, NamespaceNotFoundException e) { + this(null, tableName, "Namespace " + Tables.qualify(tableName).getFirst() + " does not exist.", e); + } + /** * @return the name of the table sought */ http://git-wip-us.apache.org/repos/asf/accumulo/blob/f35e3f47/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperations.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperations.java b/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperations.java index 535287d..b81310e 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperations.java +++ b/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperations.java @@ -277,7 +277,7 @@ public interface NamespaceOperations { /** * Check whether a given iterator configuration conflicts with existing configuration; in particular, determine if the name or priority are already in use for - * the specified scopes. If not, an IllegalArgumentException is thrown, wrapped in an AccumuloException. + * the specified scopes. If so, an IllegalArgumentException is thrown, wrapped in an AccumuloException. * * @param namespace * the name of the namespace http://git-wip-us.apache.org/repos/asf/accumulo/blob/f35e3f47/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperationsImpl.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperationsImpl.java index e5e18dc..569a3b6 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperationsImpl.java +++ b/core/src/main/java/org/apache/accumulo/core/client/admin/NamespaceOperationsImpl.java @@ -27,7 +27,6 @@ import java.util.Map.Entry; import java.util.SortedSet; import java.util.TreeSet; -import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.client.AccumuloException; import org.apache.accumulo.core.client.AccumuloSecurityException; import org.apache.accumulo.core.client.Instance; @@ -35,13 +34,13 @@ import org.apache.accumulo.core.client.IteratorSetting; import org.apache.accumulo.core.client.NamespaceExistsException; import org.apache.accumulo.core.client.NamespaceNotEmptyException; import org.apache.accumulo.core.client.NamespaceNotFoundException; -import org.apache.accumulo.core.client.TableOfflineException; +import org.apache.accumulo.core.client.TableExistsException; +import org.apache.accumulo.core.client.TableNotFoundException; import org.apache.accumulo.core.client.impl.ClientExec; import org.apache.accumulo.core.client.impl.ClientExecReturn; import org.apache.accumulo.core.client.impl.MasterClient; import org.apache.accumulo.core.client.impl.Namespaces; import org.apache.accumulo.core.client.impl.ServerClient; -import org.apache.accumulo.core.client.impl.Tables; import org.apache.accumulo.core.client.impl.thrift.ClientService; import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode; import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException; @@ -49,29 +48,27 @@ import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException import org.apache.accumulo.core.constraints.Constraint; import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope; import org.apache.accumulo.core.iterators.SortedKeyValueIterator; +import org.apache.accumulo.core.master.thrift.FateOperation; import org.apache.accumulo.core.master.thrift.MasterClientService; -import org.apache.accumulo.core.master.thrift.TableOperation; import org.apache.accumulo.core.security.Credentials; import org.apache.accumulo.core.util.ArgumentChecker; -import org.apache.accumulo.core.util.ByteBufferUtil; import org.apache.accumulo.core.util.OpTimer; -import org.apache.accumulo.core.util.UtilWaitThread; import org.apache.accumulo.trace.instrument.Tracer; import org.apache.log4j.Level; import org.apache.log4j.Logger; -import org.apache.thrift.TException; -import org.apache.thrift.transport.TTransportException; public class NamespaceOperationsImpl extends NamespaceOperationsHelper { private Instance instance; private Credentials credentials; + private TableOperationsImpl tableOps; private static final Logger log = Logger.getLogger(TableOperations.class); - public NamespaceOperationsImpl(Instance instance, Credentials credentials) { + public NamespaceOperationsImpl(Instance instance, Credentials credentials, TableOperationsImpl tableOps) { ArgumentChecker.notNull(instance, credentials); this.instance = instance; this.credentials = credentials; + this.tableOps = tableOps; } @Override @@ -97,120 +94,10 @@ public class NamespaceOperationsImpl extends NamespaceOperationsHelper { ArgumentChecker.notNull(namespace); try { - doNamespaceOperation(TableOperation.CREATE, Arrays.asList(ByteBuffer.wrap(namespace.getBytes())), Collections. emptyMap()); + doNamespaceFateOperation(FateOperation.NAMESPACE_CREATE, Arrays.asList(ByteBuffer.wrap(namespace.getBytes())), Collections. emptyMap()); } catch (NamespaceNotFoundException e) { - throw new AssertionError("Shouldn't happen: " + e.getMessage()); - } - } - - private long beginNamespaceOperation() throws ThriftSecurityException, TException { - while (true) { - MasterClientService.Iface client = null; - try { - client = MasterClient.getConnectionWithRetry(instance); - return client.beginNamespaceOperation(Tracer.traceInfo(), credentials.toThrift(instance)); - } catch (TTransportException tte) { - log.debug("Failed to call beginNamespaceOperation(), retrying ... ", tte); - UtilWaitThread.sleep(100); - } finally { - MasterClient.close(client); - } - } - } - - private void executeNamespaceOperation(long opid, TableOperation op, List args, Map opts, boolean autoCleanUp) - throws ThriftSecurityException, TException, ThriftTableOperationException { - while (true) { - MasterClientService.Iface client = null; - try { - client = MasterClient.getConnectionWithRetry(instance); - client.executeNamespaceOperation(Tracer.traceInfo(), credentials.toThrift(instance), opid, op, args, opts, autoCleanUp); - break; - } catch (TTransportException tte) { - log.debug("Failed to call executeTableOperation(), retrying ... ", tte); - UtilWaitThread.sleep(100); - } finally { - MasterClient.close(client); - } - } - } - - private String waitForNamespaceOperation(long opid) throws ThriftSecurityException, TException, ThriftTableOperationException { - while (true) { - MasterClientService.Iface client = null; - try { - client = MasterClient.getConnectionWithRetry(instance); - return client.waitForNamespaceOperation(Tracer.traceInfo(), credentials.toThrift(instance), opid); - } catch (TTransportException tte) { - log.debug("Failed to call waitForTableOperation(), retrying ... ", tte); - UtilWaitThread.sleep(100); - } finally { - MasterClient.close(client); - } - } - } - - private void finishNamespaceOperation(long opid) throws ThriftSecurityException, TException { - while (true) { - MasterClientService.Iface client = null; - try { - client = MasterClient.getConnectionWithRetry(instance); - client.finishNamespaceOperation(Tracer.traceInfo(), credentials.toThrift(instance), opid); - break; - } catch (TTransportException tte) { - log.debug("Failed to call finishTableOperation(), retrying ... ", tte); - UtilWaitThread.sleep(100); - } finally { - MasterClient.close(client); - } - } - } - - private String doNamespaceOperation(TableOperation op, List args, Map opts) throws AccumuloSecurityException, - NamespaceExistsException, NamespaceNotFoundException, AccumuloException { - return doNamespaceOperation(op, args, opts, true); - } - - private String doNamespaceOperation(TableOperation op, List args, Map opts, boolean wait) throws AccumuloSecurityException, - NamespaceExistsException, NamespaceNotFoundException, AccumuloException { - Long opid = null; - - try { - opid = beginNamespaceOperation(); - executeNamespaceOperation(opid, op, args, opts, !wait); - if (!wait) { - opid = null; - return null; - } - String ret = waitForNamespaceOperation(opid); - Tables.clearCache(instance); - return ret; - } catch (ThriftSecurityException e) { - String tableName = ByteBufferUtil.toString(args.get(0)); - String tableInfo = Tables.getPrintableTableInfoFromName(instance, tableName); - throw new AccumuloSecurityException(e.user, e.code, tableInfo, e); - } catch (ThriftTableOperationException e) { - switch (e.getType()) { - case EXISTS: - throw new NamespaceExistsException(e); - case NOTFOUND: - throw new NamespaceNotFoundException(e); - case OFFLINE: - throw new TableOfflineException(instance, null); - case OTHER: - default: - throw new AccumuloException(e.description, e); - } - } catch (Exception e) { - throw new AccumuloException(e.getMessage(), e); - } finally { - // always finish table op, even when exception - if (opid != null) - try { - finishNamespaceOperation(opid); - } catch (Exception e) { - log.warn(e.getMessage(), e); - } + // should not happen + throw new AssertionError(e); } } @@ -219,7 +106,7 @@ public class NamespaceOperationsImpl extends NamespaceOperationsHelper { ArgumentChecker.notNull(namespace); String namespaceId = Namespaces.getNamespaceId(instance, namespace); - if (namespaceId.equals(Constants.ACCUMULO_NAMESPACE_ID) || namespaceId.equals(Constants.DEFAULT_NAMESPACE_ID)) { + if (namespaceId.equals(Namespaces.ACCUMULO_NAMESPACE_ID) || namespaceId.equals(Namespaces.DEFAULT_NAMESPACE_ID)) { log.debug(credentials.getPrincipal() + " attempted to delete the " + namespaceId + " namespace"); throw new AccumuloSecurityException(credentials.getPrincipal(), SecurityErrorCode.UNSUPPORTED_OPERATION); } @@ -232,9 +119,10 @@ public class NamespaceOperationsImpl extends NamespaceOperationsHelper { Map opts = new HashMap(); try { - doNamespaceOperation(TableOperation.DELETE, args, opts); + doNamespaceFateOperation(FateOperation.NAMESPACE_DELETE, args, opts); } catch (NamespaceExistsException e) { - throw new AssertionError("Shouldn't happen: " + e.getMessage()); + // should not happen + throw new AssertionError(e); } } @@ -245,14 +133,15 @@ public class NamespaceOperationsImpl extends NamespaceOperationsHelper { List args = Arrays.asList(ByteBuffer.wrap(oldNamespaceName.getBytes()), ByteBuffer.wrap(newNamespaceName.getBytes())); Map opts = new HashMap(); - doNamespaceOperation(TableOperation.RENAME, args, opts); + doNamespaceFateOperation(FateOperation.NAMESPACE_RENAME, args, opts); } @Override - public void setProperty(final String namespace, final String property, final String value) throws AccumuloException, AccumuloSecurityException { + public void setProperty(final String namespace, final String property, final String value) throws AccumuloException, AccumuloSecurityException, + NamespaceNotFoundException { ArgumentChecker.notNull(namespace, property, value); - MasterClient.execute(instance, new ClientExec() { + MasterClient.executeNamespace(instance, new ClientExec() { @Override public void execute(MasterClientService.Client client) throws Exception { client.setNamespaceProperty(Tracer.traceInfo(), credentials.toThrift(instance), namespace, property, value); @@ -261,10 +150,10 @@ public class NamespaceOperationsImpl extends NamespaceOperationsHelper { } @Override - public void removeProperty(final String namespace, final String property) throws AccumuloException, AccumuloSecurityException { + public void removeProperty(final String namespace, final String property) throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException { ArgumentChecker.notNull(namespace, property); - MasterClient.execute(instance, new ClientExec() { + MasterClient.executeNamespace(instance, new ClientExec() { @Override public void execute(MasterClientService.Client client) throws Exception { client.removeNamespaceProperty(Tracer.traceInfo(), credentials.toThrift(instance), namespace, property); @@ -284,7 +173,7 @@ public class NamespaceOperationsImpl extends NamespaceOperationsHelper { }).entrySet(); } catch (ThriftTableOperationException e) { switch (e.getType()) { - case NOTFOUND: + case NAMESPACE_NOTFOUND: throw new NamespaceNotFoundException(e); case OTHER: default: @@ -304,19 +193,6 @@ public class NamespaceOperationsImpl extends NamespaceOperationsHelper { } @Override - public void attachIterator(String namespace, IteratorSetting setting, EnumSet scopes) throws AccumuloSecurityException, AccumuloException, - NamespaceNotFoundException { - testClassLoad(namespace, setting.getIteratorClass(), SortedKeyValueIterator.class.getName()); - super.attachIterator(namespace, setting, scopes); - } - - @Override - public int addConstraint(String namespace, String constraintClassName) throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException { - testClassLoad(namespace, constraintClassName, Constraint.class.getName()); - return super.addConstraint(namespace, constraintClassName); - } - - @Override public boolean testClassLoad(final String namespace, final String className, final String asTypeName) throws NamespaceNotFoundException, AccumuloException, AccumuloSecurityException { ArgumentChecker.notNull(namespace, className, asTypeName); @@ -330,9 +206,8 @@ public class NamespaceOperationsImpl extends NamespaceOperationsHelper { }); } catch (ThriftTableOperationException e) { switch (e.getType()) { - case NOTFOUND: + case NAMESPACE_NOTFOUND: throw new NamespaceNotFoundException(e); - case OTHER: default: throw new AccumuloException(e.description, e); } @@ -344,4 +219,30 @@ public class NamespaceOperationsImpl extends NamespaceOperationsHelper { throw new AccumuloException(e); } } + + @Override + public void attachIterator(String namespace, IteratorSetting setting, EnumSet scopes) throws AccumuloSecurityException, AccumuloException, + NamespaceNotFoundException { + testClassLoad(namespace, setting.getIteratorClass(), SortedKeyValueIterator.class.getName()); + super.attachIterator(namespace, setting, scopes); + } + + @Override + public int addConstraint(String namespace, String constraintClassName) throws AccumuloException, AccumuloSecurityException, NamespaceNotFoundException { + testClassLoad(namespace, constraintClassName, Constraint.class.getName()); + return super.addConstraint(namespace, constraintClassName); + } + + private String doNamespaceFateOperation(FateOperation op, List args, Map opts) throws AccumuloSecurityException, + AccumuloException, NamespaceExistsException, NamespaceNotFoundException { + try { + return tableOps.doFateOperation(op, args, opts); + } catch (TableExistsException e) { + // should not happen + throw new AssertionError(e); + } catch (TableNotFoundException e) { + // should not happen + throw new AssertionError(e); + } + } } http://git-wip-us.apache.org/repos/asf/accumulo/blob/f35e3f47/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsHelper.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsHelper.java b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsHelper.java index 9e86466..843f572 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsHelper.java +++ b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsHelper.java @@ -29,17 +29,19 @@ import org.apache.accumulo.core.client.IteratorSetting; import org.apache.accumulo.core.client.TableNotFoundException; import org.apache.accumulo.core.conf.Property; import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope; +import org.apache.accumulo.core.util.ArgumentChecker; public abstract class TableOperationsHelper implements TableOperations { - + @Override public void attachIterator(String tableName, IteratorSetting setting) throws AccumuloSecurityException, AccumuloException, TableNotFoundException { attachIterator(tableName, setting, EnumSet.allOf(IteratorScope.class)); } - + @Override public void attachIterator(String tableName, IteratorSetting setting, EnumSet scopes) throws AccumuloSecurityException, AccumuloException, TableNotFoundException { + ArgumentChecker.notNull(tableName, setting, scopes); checkIteratorConflicts(tableName, setting, scopes); for (IteratorScope scope : scopes) { String root = String.format("%s%s.%s", Property.TABLE_ITERATOR_PREFIX, scope.name().toLowerCase(), setting.getName()); @@ -49,12 +51,10 @@ public abstract class TableOperationsHelper implements TableOperations { this.setProperty(tableName, root, setting.getPriority() + "," + setting.getIteratorClass()); } } - + @Override public void removeIterator(String tableName, String name, EnumSet scopes) throws AccumuloSecurityException, AccumuloException, TableNotFoundException { - if (!exists(tableName)) - throw new TableNotFoundException(null, tableName, null); Map copy = new TreeMap(); for (Entry property : this.getProperties(tableName)) { copy.put(property.getKey(), property.getValue()); @@ -67,16 +67,15 @@ public abstract class TableOperationsHelper implements TableOperations { } } } - + @Override public IteratorSetting getIteratorSetting(String tableName, String name, IteratorScope scope) throws AccumuloSecurityException, AccumuloException, TableNotFoundException { - if (!exists(tableName)) - throw new TableNotFoundException(null, tableName, null); + ArgumentChecker.notNull(tableName, name, scope); int priority = -1; String classname = null; Map settings = new HashMap(); - + String root = String.format("%s%s.%s", Property.TABLE_ITERATOR_PREFIX, scope.name().toLowerCase(), name); String opt = root + ".opt."; for (Entry property : this.getProperties(tableName)) { @@ -96,11 +95,9 @@ public abstract class TableOperationsHelper implements TableOperations { } return new IteratorSetting(priority, name, classname, settings); } - + @Override public Map> listIterators(String tableName) throws AccumuloSecurityException, AccumuloException, TableNotFoundException { - if (!exists(tableName)) - throw new TableNotFoundException(null, tableName, null); Map> result = new TreeMap>(); for (Entry property : this.getProperties(tableName)) { String name = property.getKey(); @@ -116,11 +113,10 @@ public abstract class TableOperationsHelper implements TableOperations { } return result; } - + @Override public void checkIteratorConflicts(String tableName, IteratorSetting setting, EnumSet scopes) throws AccumuloException, TableNotFoundException { - if (!exists(tableName)) - throw new TableNotFoundException(null, tableName, null); + ArgumentChecker.notNull(tableName, setting, scopes); for (IteratorScope scope : scopes) { String scopeStr = String.format("%s%s", Property.TABLE_ITERATOR_PREFIX, scope.name().toLowerCase()); String nameStr = String.format("%s.%s", scopeStr, setting.getName()); @@ -129,7 +125,8 @@ public abstract class TableOperationsHelper implements TableOperations { for (Entry property : this.getProperties(tableName)) { if (property.getKey().startsWith(scopeStr)) { if (property.getKey().equals(nameStr)) - throw new AccumuloException(new IllegalArgumentException("iterator name conflict for " + setting.getName() + ": " + property.getKey() + "=" + property.getValue())); + throw new AccumuloException(new IllegalArgumentException("iterator name conflict for " + setting.getName() + ": " + property.getKey() + "=" + + property.getValue())); if (property.getKey().startsWith(optStr)) optionConflicts.put(property.getKey(), property.getValue()); if (property.getKey().contains(".opt.")) @@ -149,7 +146,7 @@ public abstract class TableOperationsHelper implements TableOperations { throw new AccumuloException(new IllegalArgumentException("iterator options conflict for " + setting.getName() + ": " + optionConflicts)); } } - + @Override public int addConstraint(String tableName, String constraintClassName) throws AccumuloException, AccumuloSecurityException, TableNotFoundException { TreeSet constraintNumbers = new TreeSet(); @@ -175,12 +172,12 @@ public abstract class TableOperationsHelper implements TableOperations { this.setProperty(tableName, Property.TABLE_CONSTRAINT_PREFIX.toString() + i, constraintClassName); return i; } - + @Override public void removeConstraint(String tableName, int number) throws AccumuloException, AccumuloSecurityException { this.removeProperty(tableName, Property.TABLE_CONSTRAINT_PREFIX.toString() + number); } - + @Override public Map listConstraints(String tableName) throws AccumuloException, TableNotFoundException { Map constraints = new TreeMap(); http://git-wip-us.apache.org/repos/asf/accumulo/blob/f35e3f47/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java index 76cda7d..9956538 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java +++ b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java @@ -50,6 +50,7 @@ import org.apache.accumulo.core.client.AccumuloSecurityException; import org.apache.accumulo.core.client.Instance; import org.apache.accumulo.core.client.IsolatedScanner; import org.apache.accumulo.core.client.IteratorSetting; +import org.apache.accumulo.core.client.NamespaceExistsException; import org.apache.accumulo.core.client.NamespaceNotFoundException; import org.apache.accumulo.core.client.RowIterator; import org.apache.accumulo.core.client.Scanner; @@ -61,7 +62,6 @@ import org.apache.accumulo.core.client.impl.AccumuloServerException; import org.apache.accumulo.core.client.impl.ClientExec; import org.apache.accumulo.core.client.impl.ClientExecReturn; import org.apache.accumulo.core.client.impl.MasterClient; -import org.apache.accumulo.core.client.impl.Namespaces; import org.apache.accumulo.core.client.impl.ServerClient; import org.apache.accumulo.core.client.impl.ServerConfigurationUtil; import org.apache.accumulo.core.client.impl.Tables; @@ -72,7 +72,6 @@ import org.apache.accumulo.core.client.impl.thrift.ClientService.Client; import org.apache.accumulo.core.client.impl.thrift.TDiskUsage; import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException; import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException; -import org.apache.accumulo.core.client.security.SecurityErrorCode; import org.apache.accumulo.core.conf.AccumuloConfiguration; import org.apache.accumulo.core.conf.ConfigurationCopy; import org.apache.accumulo.core.conf.Property; @@ -87,8 +86,8 @@ import org.apache.accumulo.core.iterators.IteratorUtil; import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope; import org.apache.accumulo.core.iterators.SortedKeyValueIterator; import org.apache.accumulo.core.master.state.tables.TableState; +import org.apache.accumulo.core.master.thrift.FateOperation; import org.apache.accumulo.core.master.thrift.MasterClientService; -import org.apache.accumulo.core.master.thrift.TableOperation; import org.apache.accumulo.core.metadata.MetadataServicer; import org.apache.accumulo.core.metadata.MetadataTable; import org.apache.accumulo.core.metadata.RootTable; @@ -120,10 +119,6 @@ import org.apache.thrift.TApplicationException; import org.apache.thrift.TException; import org.apache.thrift.transport.TTransportException; -/** - * Provides a class for administering tables - * - */ public class TableOperationsImpl extends TableOperationsHelper { private Instance instance; private Credentials credentials; @@ -225,21 +220,21 @@ public class TableOperationsImpl extends TableOperationsHelper { opts = Collections.emptyMap(); try { - doTableOperation(TableOperation.CREATE, args, opts); - } catch (TableNotFoundException e1) { + doTableFateOperation(tableName, AccumuloException.class, FateOperation.TABLE_CREATE, args, opts); + } catch (TableNotFoundException e) { // should not happen - throw new RuntimeException(e1); + throw new AssertionError(e); } } - private long beginTableOperation() throws ThriftSecurityException, TException { + private long beginFateOperation() throws ThriftSecurityException, TException { while (true) { MasterClientService.Iface client = null; try { client = MasterClient.getConnectionWithRetry(instance); - return client.beginTableOperation(Tracer.traceInfo(), credentials.toThrift(instance)); + return client.beginFateOperation(Tracer.traceInfo(), credentials.toThrift(instance)); } catch (TTransportException tte) { - log.debug("Failed to call beginTableOperation(), retrying ... ", tte); + log.debug("Failed to call beginFateOperation(), retrying ... ", tte); UtilWaitThread.sleep(100); } finally { MasterClient.close(client); @@ -247,16 +242,17 @@ public class TableOperationsImpl extends TableOperationsHelper { } } - private void executeTableOperation(long opid, TableOperation op, List args, Map opts, boolean autoCleanUp) + // This method is for retrying in the case of network failures; anything else it passes to the caller to deal with + private void executeFateOperation(long opid, FateOperation op, List args, Map opts, boolean autoCleanUp) throws ThriftSecurityException, TException, ThriftTableOperationException { while (true) { MasterClientService.Iface client = null; try { client = MasterClient.getConnectionWithRetry(instance); - client.executeTableOperation(Tracer.traceInfo(), credentials.toThrift(instance), opid, op, args, opts, autoCleanUp); + client.executeFateOperation(Tracer.traceInfo(), credentials.toThrift(instance), opid, op, args, opts, autoCleanUp); break; } catch (TTransportException tte) { - log.debug("Failed to call executeTableOperation(), retrying ... ", tte); + log.debug("Failed to call executeFateOperation(), retrying ... ", tte); UtilWaitThread.sleep(100); } finally { MasterClient.close(client); @@ -264,14 +260,14 @@ public class TableOperationsImpl extends TableOperationsHelper { } } - private String waitForTableOperation(long opid) throws ThriftSecurityException, TException, ThriftTableOperationException { + private String waitForFateOperation(long opid) throws ThriftSecurityException, TException, ThriftTableOperationException { while (true) { MasterClientService.Iface client = null; try { client = MasterClient.getConnectionWithRetry(instance); - return client.waitForTableOperation(Tracer.traceInfo(), credentials.toThrift(instance), opid); + return client.waitForFateOperation(Tracer.traceInfo(), credentials.toThrift(instance), opid); } catch (TTransportException tte) { - log.debug("Failed to call waitForTableOperation(), retrying ... ", tte); + log.debug("Failed to call waitForFateOperation(), retrying ... ", tte); UtilWaitThread.sleep(100); } finally { MasterClient.close(client); @@ -279,15 +275,15 @@ public class TableOperationsImpl extends TableOperationsHelper { } } - private void finishTableOperation(long opid) throws ThriftSecurityException, TException { + private void finishFateOperation(long opid) throws ThriftSecurityException, TException { while (true) { MasterClientService.Iface client = null; try { client = MasterClient.getConnectionWithRetry(instance); - client.finishTableOperation(Tracer.traceInfo(), credentials.toThrift(instance), opid); + client.finishFateOperation(Tracer.traceInfo(), credentials.toThrift(instance), opid); break; } catch (TTransportException tte) { - log.debug("Failed to call finishTableOperation(), retrying ... ", tte); + log.debug("Failed to call finishFateOperation(), retrying ... ", tte); UtilWaitThread.sleep(100); } finally { MasterClient.close(client); @@ -295,38 +291,48 @@ public class TableOperationsImpl extends TableOperationsHelper { } } - private String doTableOperation(TableOperation op, List args, Map opts) throws AccumuloSecurityException, TableExistsException, - TableNotFoundException, AccumuloException { - return doTableOperation(op, args, opts, true); + String doFateOperation(FateOperation op, List args, Map opts) throws AccumuloSecurityException, TableExistsException, + TableNotFoundException, AccumuloException, NamespaceExistsException, NamespaceNotFoundException { + return doFateOperation(op, args, opts, true); } - private String doTableOperation(TableOperation op, List args, Map opts, boolean wait) throws AccumuloSecurityException, - TableExistsException, TableNotFoundException, AccumuloException { + String doFateOperation(FateOperation op, List args, Map opts, boolean wait) throws AccumuloSecurityException, + TableExistsException, TableNotFoundException, AccumuloException, NamespaceExistsException, NamespaceNotFoundException { Long opid = null; try { - opid = beginTableOperation(); - executeTableOperation(opid, op, args, opts, !wait); + opid = beginFateOperation(); + executeFateOperation(opid, op, args, opts, !wait); if (!wait) { opid = null; return null; } - String ret = waitForTableOperation(opid); + String ret = waitForFateOperation(opid); Tables.clearCache(instance); return ret; } catch (ThriftSecurityException e) { String tableName = ByteBufferUtil.toString(args.get(0)); - String tableInfo = Tables.getPrintableTableInfoFromName(instance, tableName); - throw new AccumuloSecurityException(e.user, e.code, tableInfo, e); + switch (e.getCode()) { + case TABLE_DOESNT_EXIST: + throw new TableNotFoundException(null, tableName, "Target table does not exist"); + case NAMESPACE_DOESNT_EXIST: + throw new NamespaceNotFoundException(null, tableName, "Target namespace does not exist"); + default: + String tableInfo = Tables.getPrintableTableInfoFromName(instance, tableName); + throw new AccumuloSecurityException(e.user, e.code, tableInfo, e); + } } catch (ThriftTableOperationException e) { switch (e.getType()) { case EXISTS: throw new TableExistsException(e); case NOTFOUND: throw new TableNotFoundException(e); + case NAMESPACE_EXISTS: + throw new NamespaceExistsException(e); + case NAMESPACE_NOTFOUND: + throw new NamespaceNotFoundException(e); case OFFLINE: throw new TableOfflineException(instance, null); - case OTHER: default: throw new AccumuloException(e.description, e); } @@ -336,7 +342,7 @@ public class TableOperationsImpl extends TableOperationsHelper { // always finish table op, even when exception if (opid != null) try { - finishTableOperation(opid); + finishFateOperation(opid); } catch (Exception e) { log.warn(e.getMessage(), e); } @@ -525,10 +531,10 @@ public class TableOperationsImpl extends TableOperationsHelper { : TextUtil.getByteBuffer(end)); Map opts = new HashMap(); try { - doTableOperation(TableOperation.MERGE, args, opts); + doTableFateOperation(tableName, TableNotFoundException.class, FateOperation.TABLE_MERGE, args, opts); } catch (TableExistsException e) { // should not happen - throw new RuntimeException(e); + throw new AssertionError(e); } } @@ -541,10 +547,10 @@ public class TableOperationsImpl extends TableOperationsHelper { : TextUtil.getByteBuffer(end)); Map opts = new HashMap(); try { - doTableOperation(TableOperation.DELETE_RANGE, args, opts); + doTableFateOperation(tableName, TableNotFoundException.class, FateOperation.TABLE_DELETE_RANGE, args, opts); } catch (TableExistsException e) { // should not happen - throw new RuntimeException(e); + throw new AssertionError(e); } } @@ -666,10 +672,10 @@ public class TableOperationsImpl extends TableOperationsHelper { Map opts = new HashMap(); try { - doTableOperation(TableOperation.DELETE, args, opts); + doTableFateOperation(tableName, TableNotFoundException.class, FateOperation.TABLE_DELETE, args, opts); } catch (TableExistsException e) { // should not happen - throw new RuntimeException(e); + throw new AssertionError(e); } } @@ -680,12 +686,6 @@ public class TableOperationsImpl extends TableOperationsHelper { ArgumentChecker.notNull(srcTableName, newTableName); - String namespace = Tables.qualify(newTableName).getFirst(); - if (!namespaceExists(namespace)) { - String info = "Namespace not found while cloning table"; - throw new IllegalArgumentException(new NamespaceNotFoundException(null, namespace, info)); - } - String srcTableId = Tables.getTableId(instance, srcTableName); if (flush) @@ -697,14 +697,6 @@ public class TableOperationsImpl extends TableOperationsHelper { if (propertiesToSet == null) propertiesToSet = Collections.emptyMap(); - HashSet excludeProps = getUniqueNamespaceProperties(namespace, srcTableName, propertiesToSet); - for (String p : propertiesToExclude) { - excludeProps.add(p); - } - - if (!Collections.disjoint(excludeProps, propertiesToSet.keySet())) - throw new IllegalArgumentException("propertiesToSet and propertiesToExclude not disjoint"); - List args = Arrays.asList(ByteBuffer.wrap(srcTableId.getBytes()), ByteBuffer.wrap(newTableName.getBytes())); Map opts = new HashMap(); for (Entry entry : propertiesToSet.entrySet()) { @@ -717,35 +709,7 @@ public class TableOperationsImpl extends TableOperationsHelper { opts.put(CLONE_EXCLUDE_PREFIX + prop, ""); } - doTableOperation(TableOperation.CLONE, args, opts); - } - - // get the properties that are only in the namespace so that we can exclude them when copying table properties. - // also, don't exclude properties that are going to be explicitly set. - private HashSet getUniqueNamespaceProperties(String namespace, String table, Map propsToSet) throws TableNotFoundException, - AccumuloException { - HashSet props = new HashSet(); - try { - Iterable> n = new NamespaceOperationsImpl(instance, credentials).getProperties(namespace); - Iterable> t = getProperties(table); - Map tmap = new HashMap(); - for (Entry e : t) { - tmap.put(e.getKey(), e.getValue()); - } - for (Entry e : n) { - String val = tmap.get(e.getKey()); - if (e.getValue().equals(val)) { - props.add(e.getKey()); - } - } - } catch (NamespaceNotFoundException e) { - throw new IllegalStateException(new NamespaceNotFoundException(null, namespace, null)); - } - - for (Entry e : propsToSet.entrySet()) { - props.remove(e.getKey()); - } - return props; + doTableFateOperation(newTableName, AccumuloException.class, FateOperation.TABLE_CLONE, args, opts); } /** @@ -770,7 +734,7 @@ public class TableOperationsImpl extends TableOperationsHelper { List args = Arrays.asList(ByteBuffer.wrap(oldTableName.getBytes(Constants.UTF8)), ByteBuffer.wrap(newTableName.getBytes(Constants.UTF8))); Map opts = new HashMap(); - doTableOperation(TableOperation.RENAME, args, opts); + doTableFateOperation(oldTableName, TableNotFoundException.class, FateOperation.TABLE_RENAME, args, opts); } /** @@ -827,10 +791,15 @@ public class TableOperationsImpl extends TableOperationsHelper { Map opts = new HashMap(); try { - doTableOperation(TableOperation.COMPACT, args, opts, wait); + doFateOperation(FateOperation.TABLE_COMPACT, args, opts, wait); } catch (TableExistsException e) { // should not happen - throw new RuntimeException(e); + throw new AssertionError(e); + } catch (NamespaceExistsException e) { + // should not happen + throw new AssertionError(e); + } catch (NamespaceNotFoundException e) { + throw new TableNotFoundException(null, tableName, "Namespace not found", e); } } @@ -842,10 +811,10 @@ public class TableOperationsImpl extends TableOperationsHelper { Map opts = new HashMap(); try { - doTableOperation(TableOperation.COMPACT_CANCEL, args, opts, true); + doTableFateOperation(tableName, TableNotFoundException.class, FateOperation.TABLE_CANCEL_COMPACT, args, opts); } catch (TableExistsException e) { // should not happen - throw new RuntimeException(e); + throw new AssertionError(e); } } @@ -887,16 +856,17 @@ public class TableOperationsImpl extends TableOperationsHelper { } } } catch (ThriftSecurityException e) { - if (e.getCode().equals(SecurityErrorCode.TABLE_DOESNT_EXIST)) { - throw new TableNotFoundException(tableId, null, e.getMessage(), e); + switch (e.getCode()) { + case TABLE_DOESNT_EXIST: + throw new TableNotFoundException(tableId, null, e.getMessage(), e); + default: + log.debug("flush security exception on table id " + tableId); + throw new AccumuloSecurityException(e.user, e.code, e); } - log.debug("flush security exception on table id " + tableId); - throw new AccumuloSecurityException(e.user, e.code, e); } catch (ThriftTableOperationException e) { switch (e.getType()) { case NOTFOUND: throw new TableNotFoundException(e); - case OTHER: default: throw new AccumuloException(e.description, e); } @@ -922,12 +892,16 @@ public class TableOperationsImpl extends TableOperationsHelper { @Override public void setProperty(final String tableName, final String property, final String value) throws AccumuloException, AccumuloSecurityException { ArgumentChecker.notNull(tableName, property, value); - MasterClient.execute(instance, new ClientExec() { - @Override - public void execute(MasterClientService.Client client) throws Exception { - client.setTableProperty(Tracer.traceInfo(), credentials.toThrift(instance), tableName, property, value); - } - }); + try { + MasterClient.executeTable(instance, new ClientExec() { + @Override + public void execute(MasterClientService.Client client) throws Exception { + client.setTableProperty(Tracer.traceInfo(), credentials.toThrift(instance), tableName, property, value); + } + }); + } catch (TableNotFoundException e) { + throw new AccumuloException(e); + } } /** @@ -945,12 +919,16 @@ public class TableOperationsImpl extends TableOperationsHelper { @Override public void removeProperty(final String tableName, final String property) throws AccumuloException, AccumuloSecurityException { ArgumentChecker.notNull(tableName, property); - MasterClient.execute(instance, new ClientExec() { - @Override - public void execute(MasterClientService.Client client) throws Exception { - client.removeTableProperty(Tracer.traceInfo(), credentials.toThrift(instance), tableName, property); - } - }); + try { + MasterClient.executeTable(instance, new ClientExec() { + @Override + public void execute(MasterClientService.Client client) throws Exception { + client.removeTableProperty(Tracer.traceInfo(), credentials.toThrift(instance), tableName, property); + } + }); + } catch (TableNotFoundException e) { + throw new AccumuloException(e); + } } /** @@ -976,7 +954,8 @@ public class TableOperationsImpl extends TableOperationsHelper { switch (e.getType()) { case NOTFOUND: throw new TableNotFoundException(e); - case OTHER: + case NAMESPACE_NOTFOUND: + throw new TableNotFoundException(tableName, new NamespaceNotFoundException(e)); default: throw new AccumuloException(e.description, e); } @@ -1021,7 +1000,13 @@ public class TableOperationsImpl extends TableOperationsHelper { setProperty(tableName, Property.TABLE_LOCALITY_GROUP_PREFIX + entry.getKey(), value); } - setProperty(tableName, Property.TABLE_LOCALITY_GROUPS.getKey(), StringUtil.join(groups.keySet(), ",")); + try { + setProperty(tableName, Property.TABLE_LOCALITY_GROUPS.getKey(), StringUtil.join(groups.keySet(), ",")); + } catch (AccumuloException e) { + if (e.getCause() instanceof TableNotFoundException) + throw (TableNotFoundException) e.getCause(); + throw e; + } // remove anything extraneous String prefix = Property.TABLE_LOCALITY_GROUP_PREFIX.getKey(); @@ -1181,6 +1166,8 @@ public class TableOperationsImpl extends TableOperationsHelper { public void importDirectory(String tableName, String dir, String failureDir, boolean setTime) throws IOException, AccumuloSecurityException, TableNotFoundException, AccumuloException { ArgumentChecker.notNull(tableName, dir, failureDir); + // check for table existance + Tables.getTableId(instance, tableName); Path dirPath = checkPath(dir, "Bulk", ""); Path failPath = checkPath(failureDir, "Bulk", "failure"); @@ -1190,13 +1177,11 @@ public class TableOperationsImpl extends TableOperationsHelper { Map opts = new HashMap(); try { - doTableOperation(TableOperation.BULK_IMPORT, args, opts); + doTableFateOperation(tableName, TableNotFoundException.class, FateOperation.TABLE_BULK_IMPORT, args, opts); } catch (TableExistsException e) { // should not happen - throw new RuntimeException(e); + throw new AssertionError(e); } - // return new BulkImportHelper(instance, credentials, tableName).importDirectory(new Path(dir), new Path(failureDir), numThreads, numAssignThreads, - // disableGC); } private void waitForTableStateTransition(String tableId, TableState expectedState) throws AccumuloException, TableNotFoundException, @@ -1340,10 +1325,10 @@ public class TableOperationsImpl extends TableOperationsHelper { Map opts = new HashMap(); try { - doTableOperation(TableOperation.OFFLINE, args, opts); + doTableFateOperation(tableName, TableNotFoundException.class, FateOperation.TABLE_OFFLINE, args, opts); } catch (TableExistsException e) { // should not happen - throw new RuntimeException(e); + throw new AssertionError(e); } if (wait) @@ -1373,10 +1358,10 @@ public class TableOperationsImpl extends TableOperationsHelper { Map opts = new HashMap(); try { - doTableOperation(TableOperation.ONLINE, args, opts); + doTableFateOperation(tableName, TableNotFoundException.class, FateOperation.TABLE_ONLINE, args, opts); } catch (TableExistsException e) { // should not happen - throw new RuntimeException(e); + throw new AssertionError(e); } if (wait) @@ -1431,7 +1416,8 @@ public class TableOperationsImpl extends TableOperationsHelper { switch (e.getType()) { case NOTFOUND: throw new TableNotFoundException(e); - case OTHER: + case NAMESPACE_NOTFOUND: + throw new TableNotFoundException(e.getTableName(), new NamespaceNotFoundException(e)); default: throw new AccumuloException(e.description, e); } @@ -1513,10 +1499,10 @@ public class TableOperationsImpl extends TableOperationsHelper { Map opts = Collections.emptyMap(); try { - doTableOperation(TableOperation.IMPORT, args, opts); - } catch (TableNotFoundException e1) { + doTableFateOperation(tableName, AccumuloException.class, FateOperation.TABLE_IMPORT, args, opts); + } catch (TableNotFoundException e) { // should not happen - throw new RuntimeException(e1); + throw new AssertionError(e); } } @@ -1530,10 +1516,10 @@ public class TableOperationsImpl extends TableOperationsHelper { Map opts = Collections.emptyMap(); try { - doTableOperation(TableOperation.EXPORT, args, opts); - } catch (TableExistsException e1) { + doTableFateOperation(tableName, TableNotFoundException.class, FateOperation.TABLE_EXPORT, args, opts); + } catch (TableExistsException e) { // should not happen - throw new RuntimeException(e1); + throw new AssertionError(e); } } @@ -1553,7 +1539,8 @@ public class TableOperationsImpl extends TableOperationsHelper { switch (e.getType()) { case NOTFOUND: throw new TableNotFoundException(e); - case OTHER: + case NAMESPACE_NOTFOUND: + throw new TableNotFoundException(tableName, new NamespaceNotFoundException(e)); default: throw new AccumuloException(e.description, e); } @@ -1579,7 +1566,26 @@ public class TableOperationsImpl extends TableOperationsHelper { return super.addConstraint(tableName, constraintClassName); } - private boolean namespaceExists(String namespace) { - return Namespaces.getNameToIdMap(instance).containsKey(namespace); + private void doTableFateOperation(String tableName, Class namespaceNotFoundExceptionClass, FateOperation op, List args, + Map opts) throws AccumuloSecurityException, AccumuloException, TableExistsException, TableNotFoundException { + try { + doFateOperation(op, args, opts); + } catch (NamespaceExistsException e) { + // should not happen + throw new AssertionError(e); + } catch (NamespaceNotFoundException e) { + if (namespaceNotFoundExceptionClass == null) { + // should not happen + throw new AssertionError(e); + } else if (AccumuloException.class.isAssignableFrom(namespaceNotFoundExceptionClass)) { + throw new AccumuloException("Cannot create table in non-existent namespace", e); + } else if (TableNotFoundException.class.isAssignableFrom(namespaceNotFoundExceptionClass)) { + throw new TableNotFoundException(null, tableName, "Namespace not found", e); + } else { + // should not happen + throw new AssertionError(e); + } + } } + } http://git-wip-us.apache.org/repos/asf/accumulo/blob/f35e3f47/core/src/main/java/org/apache/accumulo/core/client/impl/ConnectorImpl.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ConnectorImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ConnectorImpl.java index fcecb3a..7d9d3ab 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/impl/ConnectorImpl.java +++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ConnectorImpl.java @@ -34,10 +34,10 @@ import org.apache.accumulo.core.client.TableNotFoundException; import org.apache.accumulo.core.client.TableOfflineException; import org.apache.accumulo.core.client.admin.InstanceOperations; import org.apache.accumulo.core.client.admin.InstanceOperationsImpl; -import org.apache.accumulo.core.client.admin.SecurityOperations; -import org.apache.accumulo.core.client.admin.SecurityOperationsImpl; import org.apache.accumulo.core.client.admin.NamespaceOperations; import org.apache.accumulo.core.client.admin.NamespaceOperationsImpl; +import org.apache.accumulo.core.client.admin.SecurityOperations; +import org.apache.accumulo.core.client.admin.SecurityOperationsImpl; import org.apache.accumulo.core.client.admin.TableOperations; import org.apache.accumulo.core.client.admin.TableOperationsImpl; import org.apache.accumulo.core.client.impl.thrift.ClientService; @@ -55,16 +55,16 @@ public class ConnectorImpl extends Connector { private TableOperations tableops = null; private NamespaceOperations namespaceops = null; private InstanceOperations instanceops = null; - + public ConnectorImpl(final Instance instance, Credentials cred) throws AccumuloException, AccumuloSecurityException { ArgumentChecker.notNull(instance, cred); if (cred.getToken().isDestroyed()) throw new AccumuloSecurityException(cred.getPrincipal(), SecurityErrorCode.TOKEN_EXPIRED); - + this.instance = instance; - + this.credentials = cred; - + // Skip fail fast for system services; string literal for class name, to avoid if (!"org.apache.accumulo.server.security.SystemCredentials$SystemToken".equals(cred.getToken().getClass().getName())) { ServerClient.execute(instance, new ClientExec() { @@ -75,26 +75,29 @@ public class ConnectorImpl extends Connector { } }); } + + this.tableops = new TableOperationsImpl(instance, credentials); + this.namespaceops = new NamespaceOperationsImpl(instance, credentials, (TableOperationsImpl) tableops); } - + private String getTableId(String tableName) throws TableNotFoundException { String tableId = Tables.getTableId(instance, tableName); if (Tables.getTableState(instance, tableId) == TableState.OFFLINE) throw new TableOfflineException(instance, tableId); return tableId; } - + @Override public Instance getInstance() { return instance; } - + @Override public BatchScanner createBatchScanner(String tableName, Authorizations authorizations, int numQueryThreads) throws TableNotFoundException { ArgumentChecker.notNull(tableName, authorizations); return new TabletServerBatchReader(instance, credentials, getTableId(tableName), authorizations, numQueryThreads); } - + @Deprecated @Override public BatchDeleter createBatchDeleter(String tableName, Authorizations authorizations, int numQueryThreads, long maxMemory, long maxLatency, @@ -103,14 +106,14 @@ public class ConnectorImpl extends Connector { return new TabletServerBatchDeleter(instance, credentials, getTableId(tableName), authorizations, numQueryThreads, new BatchWriterConfig() .setMaxMemory(maxMemory).setMaxLatency(maxLatency, TimeUnit.MILLISECONDS).setMaxWriteThreads(maxWriteThreads)); } - + @Override public BatchDeleter createBatchDeleter(String tableName, Authorizations authorizations, int numQueryThreads, BatchWriterConfig config) throws TableNotFoundException { ArgumentChecker.notNull(tableName, authorizations); return new TabletServerBatchDeleter(instance, credentials, getTableId(tableName), authorizations, numQueryThreads, config); } - + @Deprecated @Override public BatchWriter createBatchWriter(String tableName, long maxMemory, long maxLatency, int maxWriteThreads) throws TableNotFoundException { @@ -118,68 +121,64 @@ public class ConnectorImpl extends Connector { return new BatchWriterImpl(instance, credentials, getTableId(tableName), new BatchWriterConfig().setMaxMemory(maxMemory) .setMaxLatency(maxLatency, TimeUnit.MILLISECONDS).setMaxWriteThreads(maxWriteThreads)); } - + @Override public BatchWriter createBatchWriter(String tableName, BatchWriterConfig config) throws TableNotFoundException { ArgumentChecker.notNull(tableName); return new BatchWriterImpl(instance, credentials, getTableId(tableName), config); } - + @Deprecated @Override public MultiTableBatchWriter createMultiTableBatchWriter(long maxMemory, long maxLatency, int maxWriteThreads) { return new MultiTableBatchWriterImpl(instance, credentials, new BatchWriterConfig().setMaxMemory(maxMemory) .setMaxLatency(maxLatency, TimeUnit.MILLISECONDS).setMaxWriteThreads(maxWriteThreads)); } - + @Override public MultiTableBatchWriter createMultiTableBatchWriter(BatchWriterConfig config) { return new MultiTableBatchWriterImpl(instance, credentials, config); } - + @Override public ConditionalWriter createConditionalWriter(String tableName, ConditionalWriterConfig config) throws TableNotFoundException { return new ConditionalWriterImpl(instance, credentials, getTableId(tableName), config); } - + @Override public Scanner createScanner(String tableName, Authorizations authorizations) throws TableNotFoundException { ArgumentChecker.notNull(tableName, authorizations); return new ScannerImpl(instance, credentials, getTableId(tableName), authorizations); } - + @Override public String whoami() { return credentials.getPrincipal(); } - + @Override public synchronized TableOperations tableOperations() { - if (tableops == null) - tableops = new TableOperationsImpl(instance, credentials); return tableops; } - + @Override public synchronized NamespaceOperations namespaceOperations() { - if (namespaceops == null) - namespaceops = new NamespaceOperationsImpl(instance, credentials); return namespaceops; } - + @Override public synchronized SecurityOperations securityOperations() { if (secops == null) secops = new SecurityOperationsImpl(instance, credentials); - + return secops; } - + @Override public synchronized InstanceOperations instanceOperations() { if (instanceops == null) instanceops = new InstanceOperationsImpl(instance, credentials); - + return instanceops; } } http://git-wip-us.apache.org/repos/asf/accumulo/blob/f35e3f47/core/src/main/java/org/apache/accumulo/core/client/impl/MasterClient.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/MasterClient.java b/core/src/main/java/org/apache/accumulo/core/client/impl/MasterClient.java index 6bef3a7..7356663 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/impl/MasterClient.java +++ b/core/src/main/java/org/apache/accumulo/core/client/impl/MasterClient.java @@ -22,7 +22,10 @@ import java.util.List; import org.apache.accumulo.core.client.AccumuloException; import org.apache.accumulo.core.client.AccumuloSecurityException; import org.apache.accumulo.core.client.Instance; +import org.apache.accumulo.core.client.NamespaceNotFoundException; +import org.apache.accumulo.core.client.TableNotFoundException; import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException; +import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException; import org.apache.accumulo.core.master.thrift.MasterClientService; import org.apache.accumulo.core.util.ArgumentChecker; import org.apache.accumulo.core.util.ThriftUtil; @@ -33,35 +36,36 @@ import org.apache.thrift.transport.TTransportException; public class MasterClient { private static final Logger log = Logger.getLogger(MasterClient.class); - + public static MasterClientService.Client getConnectionWithRetry(Instance instance) { ArgumentChecker.notNull(instance); - + while (true) { - + MasterClientService.Client result = getConnection(instance); if (result != null) return result; UtilWaitThread.sleep(250); } - + } - + public static MasterClientService.Client getConnection(Instance instance) { List locations = instance.getMasterLocations(); - + if (locations.size() == 0) { log.debug("No masters..."); return null; } - + String master = locations.get(0); if (master.endsWith(":0")) return null; - + try { // Master requests can take a long time: don't ever time out - MasterClientService.Client client = ThriftUtil.getClientNoTimeout(new MasterClientService.Client.Factory(), master, ServerConfigurationUtil.getConfiguration(instance)); + MasterClientService.Client client = ThriftUtil.getClientNoTimeout(new MasterClientService.Client.Factory(), master, + ServerConfigurationUtil.getConfiguration(instance)); return client; } catch (TTransportException tte) { if (tte.getCause().getClass().equals(UnknownHostException.class)) { @@ -72,7 +76,7 @@ public class MasterClient { return null; } } - + public static void close(MasterClientService.Iface iface) { TServiceClient client = (TServiceClient) iface; if (client != null && client.getInputProtocol() != null && client.getInputProtocol().getTransport() != null) { @@ -81,8 +85,9 @@ public class MasterClient { log.debug("Attempt to close null connection to the master", new Exception()); } } - - public static T execute(Instance instance, ClientExecReturn exec) throws AccumuloException, AccumuloSecurityException { + + public static T execute(Instance instance, ClientExecReturn exec) throws AccumuloException, AccumuloSecurityException, + TableNotFoundException { MasterClientService.Client client = null; while (true) { try { @@ -95,6 +100,15 @@ public class MasterClient { throw new AccumuloSecurityException(e.user, e.code, e); } catch (AccumuloException e) { throw e; + } catch (ThriftTableOperationException e) { + switch (e.getType()) { + case NAMESPACE_NOTFOUND: + throw new TableNotFoundException(e.getTableName(), new NamespaceNotFoundException(e)); + case NOTFOUND: + throw new TableNotFoundException(e); + default: + throw new AccumuloException(e); + } } catch (Exception e) { throw new AccumuloException(e); } finally { @@ -103,8 +117,9 @@ public class MasterClient { } } } - - public static void execute(Instance instance, ClientExec exec) throws AccumuloException, AccumuloSecurityException { + + public static void executeGeneric(Instance instance, ClientExec exec) throws AccumuloException, AccumuloSecurityException, + TableNotFoundException { MasterClientService.Client client = null; while (true) { try { @@ -118,6 +133,15 @@ public class MasterClient { throw new AccumuloSecurityException(e.user, e.code, e); } catch (AccumuloException e) { throw e; + } catch (ThriftTableOperationException e) { + switch (e.getType()) { + case NAMESPACE_NOTFOUND: + throw new TableNotFoundException(e.getTableName(), new NamespaceNotFoundException(e)); + case NOTFOUND: + throw new TableNotFoundException(e); + default: + throw new AccumuloException(e); + } } catch (Exception e) { throw new AccumuloException(e); } finally { @@ -126,5 +150,27 @@ public class MasterClient { } } } - + + public static void executeTable(Instance instance, ClientExec exec) throws AccumuloException, AccumuloSecurityException, + TableNotFoundException { + executeGeneric(instance, exec); + } + + public static void executeNamespace(Instance instance, ClientExec exec) throws AccumuloException, AccumuloSecurityException, + NamespaceNotFoundException { + try { + executeGeneric(instance, exec); + } catch (TableNotFoundException e) { + if (e.getCause() instanceof NamespaceNotFoundException) + throw (NamespaceNotFoundException) e.getCause(); + } + } + + public static void execute(Instance instance, ClientExec exec) throws AccumuloException, AccumuloSecurityException { + try { + executeGeneric(instance, exec); + } catch (TableNotFoundException e) { + throw new AssertionError(e); + } + } } http://git-wip-us.apache.org/repos/asf/accumulo/blob/f35e3f47/core/src/main/java/org/apache/accumulo/core/client/impl/Namespaces.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/Namespaces.java b/core/src/main/java/org/apache/accumulo/core/client/impl/Namespaces.java index 64a6441..1ca5dee 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/impl/Namespaces.java +++ b/core/src/main/java/org/apache/accumulo/core/client/impl/Namespaces.java @@ -19,6 +19,7 @@ package org.apache.accumulo.core.client.impl; import java.security.SecurityPermission; import java.util.LinkedList; import java.util.List; +import java.util.Map.Entry; import java.util.SortedMap; import java.util.TreeMap; @@ -31,6 +32,11 @@ import org.apache.accumulo.fate.zookeeper.ZooCache; public class Namespaces { private static SecurityPermission TABLES_PERMISSION = new SecurityPermission("tablesPermission"); + public static final String DEFAULT_NAMESPACE_ID = "+default"; + public static final String DEFAULT_NAMESPACE = ""; + public static final String ACCUMULO_NAMESPACE_ID = "+accumulo"; + public static final String ACCUMULO_NAMESPACE = "accumulo"; + private static ZooCache getZooCache(Instance instance) { SecurityManager sm = System.getSecurityManager(); if (sm != null) { @@ -87,27 +93,20 @@ public class Namespaces { } public static List getTableIds(Instance instance, String namespaceId) throws NamespaceNotFoundException { - List l = new LinkedList(); - for (String id : Tables.getIdToNameMap(instance).keySet()) { - if (Tables.getNamespace(instance, id).equals(namespaceId)) { - l.add(id); - } - } - return l; + String namespace = getNamespaceName(instance, namespaceId); + List names = new LinkedList(); + for (Entry nameToId : Tables.getNameToIdMap(instance).entrySet()) + if (namespace.equals(Tables.qualify(nameToId.getKey()).getFirst())) + names.add(nameToId.getValue()); + return names; } public static List getTableNames(Instance instance, String namespaceId) throws NamespaceNotFoundException { - ZooCache zc = getZooCache(instance); - List ids = getTableIds(instance, namespaceId); + String namespace = getNamespaceName(instance, namespaceId); List names = new LinkedList(); - String namespace = getNamespaceName(instance, namespaceId) + "."; - if (namespaceId.equals(Constants.DEFAULT_NAMESPACE_ID) || namespaceId.equals(Constants.ACCUMULO_NAMESPACE_ID)) { - // default and system namespaces aren't displayed for backwards compatibility - namespace = ""; - } - for (String id : ids) { - names.add(namespace + new String(zc.get(ZooUtil.getRoot(instance) + Constants.ZTABLES + "/" + id + Constants.ZTABLE_NAME), Constants.UTF8)); - } + for (String name : Tables.getNameToIdMap(instance).keySet()) + if (namespace.equals(Tables.qualify(name).getFirst())) + names.add(name); return names; } } http://git-wip-us.apache.org/repos/asf/accumulo/blob/f35e3f47/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java b/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java index f3f46d5..279453b 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java +++ b/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java @@ -59,13 +59,13 @@ public class Tables { for (String tableId : tableIds) { byte[] tableName = zc.get(ZooUtil.getRoot(instance) + Constants.ZTABLES + "/" + tableId + Constants.ZTABLE_NAME); byte[] nId = zc.get(ZooUtil.getRoot(instance) + Constants.ZTABLES + "/" + tableId + Constants.ZTABLE_NAMESPACE); - String namespaceName = Constants.DEFAULT_NAMESPACE; + String namespaceName = Namespaces.DEFAULT_NAMESPACE; // create fully qualified table name if (nId == null) { namespaceName = null; } else if (nId != null) { String namespaceId = new String(nId, Constants.UTF8); - if (!namespaceId.equals(Constants.DEFAULT_NAMESPACE_ID)) { + if (!namespaceId.equals(Namespaces.DEFAULT_NAMESPACE_ID)) { try { namespaceName = namespaceIdToNameMap.get(namespaceId); if (namespaceName == null) { @@ -91,9 +91,27 @@ public class Tables { } public static String getTableId(Instance instance, String tableName) throws TableNotFoundException { + try { + return _getTableId(instance, tableName); + } catch (NamespaceNotFoundException e) { + throw new TableNotFoundException(tableName, e); + } + } + + public static String _getTableId(Instance instance, String tableName) throws NamespaceNotFoundException, TableNotFoundException { String tableId = getNameToIdMap(instance).get(tableName); - if (tableId == null) - throw new TableNotFoundException(tableId, tableName, null); + if (tableId == null) { + // maybe the table exist, but the cache was not updated yet... so try to clear the cache and check again + clearCache(instance); + tableId = getNameToIdMap(instance).get(tableName); + if (tableId == null) { + String namespace = qualify(tableName).getFirst(); + if (Namespaces.getNameToIdMap(instance).containsKey(namespace)) + throw new TableNotFoundException(null, tableName, null); + else + throw new NamespaceNotFoundException(null, namespace, null); + } + } return tableId; } @@ -169,19 +187,19 @@ public class Tables { } public static String qualified(String tableName) { - return qualified(tableName, Constants.DEFAULT_NAMESPACE); + return qualified(tableName, Namespaces.DEFAULT_NAMESPACE); } public static String qualified(String tableName, String defaultNamespace) { Pair qualifiedTableName = Tables.qualify(tableName, defaultNamespace); - if (Constants.DEFAULT_NAMESPACE.equals(qualifiedTableName.getFirst())) + if (Namespaces.DEFAULT_NAMESPACE.equals(qualifiedTableName.getFirst())) return qualifiedTableName.getSecond(); else return qualifiedTableName.toString("", ".", ""); } public static Pair qualify(String tableName) { - return qualify(tableName, Constants.DEFAULT_NAMESPACE); + return qualify(tableName, Namespaces.DEFAULT_NAMESPACE); } public static Pair qualify(String tableName, String defaultNamespace) { http://git-wip-us.apache.org/repos/asf/accumulo/blob/f35e3f47/core/src/main/java/org/apache/accumulo/core/client/impl/thrift/TableOperationExceptionType.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/thrift/TableOperationExceptionType.java b/core/src/main/java/org/apache/accumulo/core/client/impl/thrift/TableOperationExceptionType.java index 313ca08..6222f8e 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/impl/thrift/TableOperationExceptionType.java +++ b/core/src/main/java/org/apache/accumulo/core/client/impl/thrift/TableOperationExceptionType.java @@ -34,7 +34,10 @@ import org.apache.thrift.TEnum; BULK_BAD_INPUT_DIRECTORY(3), BULK_BAD_ERROR_DIRECTORY(4), BAD_RANGE(5), - OTHER(6); + OTHER(6), + NAMESPACE_EXISTS(7), + NAMESPACE_NOTFOUND(8), + INVALID_NAME(9); private final int value; @@ -69,6 +72,12 @@ import org.apache.thrift.TEnum; return BAD_RANGE; case 6: return OTHER; + case 7: + return NAMESPACE_EXISTS; + case 8: + return NAMESPACE_NOTFOUND; + case 9: + return INVALID_NAME; default: return null; } http://git-wip-us.apache.org/repos/asf/accumulo/blob/f35e3f47/core/src/main/java/org/apache/accumulo/core/client/mock/MockAccumulo.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockAccumulo.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockAccumulo.java index 1d1853e..2c26ecc 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockAccumulo.java +++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockAccumulo.java @@ -22,9 +22,9 @@ import java.util.HashMap; import java.util.Map; import java.util.SortedSet; -import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.client.BatchScanner; import org.apache.accumulo.core.client.admin.TimeType; +import org.apache.accumulo.core.client.impl.Namespaces; import org.apache.accumulo.core.client.impl.Tables; import org.apache.accumulo.core.client.security.tokens.PasswordToken; import org.apache.accumulo.core.data.Mutation; @@ -52,8 +52,8 @@ public class MockAccumulo { MockUser root = new MockUser("root", new PasswordToken(new byte[0]), Authorizations.EMPTY); root.permissions.add(SystemPermission.SYSTEM); users.put(root.name, root); - namespaces.put(Constants.DEFAULT_NAMESPACE, new MockNamespace()); - namespaces.put(Constants.ACCUMULO_NAMESPACE, new MockNamespace()); + namespaces.put(Namespaces.DEFAULT_NAMESPACE, new MockNamespace()); + namespaces.put(Namespaces.ACCUMULO_NAMESPACE, new MockNamespace()); createTable("root", RootTable.NAME, true, TimeType.LOGICAL); createTable("root", MetadataTable.NAME, true, TimeType.LOGICAL); } http://git-wip-us.apache.org/repos/asf/accumulo/blob/f35e3f47/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java index 6bf9904..8c73c0f 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java +++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java @@ -34,6 +34,7 @@ import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.client.AccumuloException; import org.apache.accumulo.core.client.AccumuloSecurityException; import org.apache.accumulo.core.client.IteratorSetting; +import org.apache.accumulo.core.client.NamespaceNotFoundException; import org.apache.accumulo.core.client.TableExistsException; import org.apache.accumulo.core.client.TableNotFoundException; import org.apache.accumulo.core.client.admin.DiskUsage; @@ -183,17 +184,14 @@ public class MockTableOperations extends TableOperationsHelper { @Override public Iterable> getProperties(String tableName) throws TableNotFoundException { String namespace = Tables.qualify(tableName).getFirst(); - - if (!namespaceExists(namespace)) { - throw new IllegalArgumentException("Namespace (" + namespace + ") does not exist"); + if (!exists(tableName)) { + if (!namespaceExists(namespace)) + throw new TableNotFoundException(tableName, new NamespaceNotFoundException(null, namespace, null)); + throw new TableNotFoundException(null, tableName, null); } Set> props = new HashSet>(acu.namespaces.get(namespace).settings.entrySet()); - if (!exists(tableName)) { - throw new TableNotFoundException(tableName, tableName, ""); - } - Set> tableProps = acu.tables.get(tableName).settings.entrySet(); for (Entry e : tableProps) { if (props.contains(e)) { http://git-wip-us.apache.org/repos/asf/accumulo/blob/f35e3f47/core/src/main/java/org/apache/accumulo/core/master/thrift/FateOperation.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/master/thrift/FateOperation.java b/core/src/main/java/org/apache/accumulo/core/master/thrift/FateOperation.java new file mode 100644 index 0000000..60cb63e --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/core/master/thrift/FateOperation.java @@ -0,0 +1,103 @@ +/* + * 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. + */ +/** + * Autogenerated by Thrift Compiler (0.9.0) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.accumulo.core.master.thrift; + + +import java.util.Map; +import java.util.HashMap; +import org.apache.thrift.TEnum; + +@SuppressWarnings("all") public enum FateOperation implements org.apache.thrift.TEnum { + TABLE_CREATE(0), + TABLE_CLONE(1), + TABLE_DELETE(2), + TABLE_RENAME(3), + TABLE_ONLINE(4), + TABLE_OFFLINE(5), + TABLE_MERGE(6), + TABLE_DELETE_RANGE(7), + TABLE_BULK_IMPORT(8), + TABLE_COMPACT(9), + TABLE_IMPORT(10), + TABLE_EXPORT(11), + TABLE_CANCEL_COMPACT(12), + NAMESPACE_CREATE(13), + NAMESPACE_DELETE(14), + NAMESPACE_RENAME(15); + + private final int value; + + private FateOperation(int value) { + this.value = value; + } + + /** + * Get the integer value of this enum value, as defined in the Thrift IDL. + */ + public int getValue() { + return value; + } + + /** + * Find a the enum type by its integer value, as defined in the Thrift IDL. + * @return null if the value is not found. + */ + public static FateOperation findByValue(int value) { + switch (value) { + case 0: + return TABLE_CREATE; + case 1: + return TABLE_CLONE; + case 2: + return TABLE_DELETE; + case 3: + return TABLE_RENAME; + case 4: + return TABLE_ONLINE; + case 5: + return TABLE_OFFLINE; + case 6: + return TABLE_MERGE; + case 7: + return TABLE_DELETE_RANGE; + case 8: + return TABLE_BULK_IMPORT; + case 9: + return TABLE_COMPACT; + case 10: + return TABLE_IMPORT; + case 11: + return TABLE_EXPORT; + case 12: + return TABLE_CANCEL_COMPACT; + case 13: + return NAMESPACE_CREATE; + case 14: + return NAMESPACE_DELETE; + case 15: + return NAMESPACE_RENAME; + default: + return null; + } + } +}