You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by kt...@apache.org on 2013/10/08 21:02:42 UTC

[2/2] git commit: ACCUMULO-1667 add synchronous option to online and offline table

ACCUMULO-1667 add synchronous option to online and offline table


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

Branch: refs/heads/master
Commit: e70a40dbab016b22b69b148161b7ce1c6af878f1
Parents: 53fb6c7
Author: Keith Turner <kt...@apache.org>
Authored: Tue Oct 8 14:23:48 2013 -0400
Committer: Keith Turner <kt...@apache.org>
Committed: Tue Oct 8 14:34:09 2013 -0400

----------------------------------------------------------------------
 .../core/client/admin/TableOperations.java      |  32 ++++
 .../core/client/admin/TableOperationsImpl.java  | 145 ++++++++++++++++++-
 .../core/client/mock/MockTableOperations.java   |  10 ++
 .../apache/accumulo/core/util/MapCounter.java   |  86 +++++++++++
 .../util/shell/commands/OfflineCommand.java     |  26 +++-
 .../core/util/shell/commands/OnlineCommand.java |  26 +++-
 .../client/admin/TableOperationsHelperTest.java |   8 +
 .../apache/accumulo/server/master/Master.java   |  18 ++-
 .../server/master/tableOps/CompactRange.java    |   2 +-
 .../accumulo/server/tabletserver/Tablet.java    |   2 +-
 .../server/tabletserver/TabletServer.java       |   2 +-
 .../apache/accumulo/server/util/MapCounter.java |  86 -----------
 .../randomwalk/concurrent/OfflineTable.java     |   4 +-
 .../randomwalk/multitable/OfflineTable.java     |   4 +-
 .../accumulo/test/ConditionalWriterTest.java    |  15 +-
 .../org/apache/accumulo/test/ShellServerIT.java |   5 +-
 16 files changed, 347 insertions(+), 124 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/e70a40db/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java
index f8ec8c3..dc95250 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperations.java
@@ -503,6 +503,7 @@ public interface TableOperations {
       AccumuloSecurityException;
   
   /**
+   * Initiates taking a table offline, but does not wait for action to complete
    * 
    * @param tableName
    *          the table to take offline
@@ -517,6 +518,22 @@ public interface TableOperations {
   /**
    * 
    * @param tableName
+   *          the table to take offline
+   * @param wait
+   *          if true, then will not return until table is offline
+   * @throws AccumuloException
+   *           when there is a general accumulo error
+   * @throws AccumuloSecurityException
+   *           when the user does not have the proper permissions
+   * @throws TableNotFoundException
+   * @since 1.6.0
+   */
+  public void offline(String tableName, boolean wait) throws AccumuloSecurityException, AccumuloException, TableNotFoundException;
+  
+  /**
+   * Initiates bringing a table online, but does not wait for action to complete
+   * 
+   * @param tableName
    *          the table to take online
    * @throws AccumuloException
    *           when there is a general accumulo error
@@ -525,6 +542,21 @@ public interface TableOperations {
    * @throws TableNotFoundException
    */
   public void online(String tableName) throws AccumuloSecurityException, AccumuloException, TableNotFoundException;
+ 
+  /**
+   * 
+   * @param tableName
+   *          the table to take online
+   * @param wait
+   *          if true, then will not return until table is online
+   * @throws AccumuloException
+   *           when there is a general accumulo error
+   * @throws AccumuloSecurityException
+   *           when the user does not have the proper permissions
+   * @throws TableNotFoundException
+   * @since 1.6.0
+   */
+  public void online(String tableName, boolean wait) throws AccumuloSecurityException, AccumuloException, TableNotFoundException;
   
   /**
    * Clears the tablet locator cache for a specified table

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e70a40db/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 215d326..8f14fba 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
@@ -27,6 +27,7 @@ import java.util.Collections;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
@@ -47,7 +48,9 @@ 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;
+import org.apache.accumulo.core.client.IsolatedScanner;
 import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.client.RowIterator;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableDeletedException;
 import org.apache.accumulo.core.client.TableExistsException;
@@ -72,8 +75,10 @@ import org.apache.accumulo.core.conf.ConfigurationCopy;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.constraints.Constraint;
 import org.apache.accumulo.core.data.ByteSequence;
+import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.file.FileUtil;
 import org.apache.accumulo.core.iterators.IteratorUtil;
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
@@ -84,6 +89,7 @@ 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;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.Credentials;
 import org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException;
@@ -92,6 +98,7 @@ import org.apache.accumulo.core.util.ArgumentChecker;
 import org.apache.accumulo.core.util.ByteBufferUtil;
 import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.accumulo.core.util.LocalityGroupUtil;
+import org.apache.accumulo.core.util.MapCounter;
 import org.apache.accumulo.core.util.NamingThreadFactory;
 import org.apache.accumulo.core.util.OpTimer;
 import org.apache.accumulo.core.util.Pair;
@@ -1127,6 +1134,123 @@ public class TableOperationsImpl extends TableOperationsHelper {
     // disableGC);
   }
   
+  private void waitForTableStateTransition(String tableId, TableState expectedState) throws AccumuloException, TableNotFoundException,
+      AccumuloSecurityException {
+    
+    Text startRow = null;
+    Text lastRow = null;
+
+    while (true) {
+
+      if (Tables.getTableState(instance, tableId) != expectedState) {
+        Tables.clearCache(instance);
+        if (Tables.getTableState(instance, tableId) != expectedState) {
+          if (!Tables.exists(instance, tableId))
+            throw new TableDeletedException(tableId);
+          throw new AccumuloException("Unexpected table state " + tableId + " " + Tables.getTableState(instance, tableId) + " != " + expectedState);
+        }
+      }
+      
+      Range range = new KeyExtent(new Text(tableId), null, null).toMetadataRange();
+      if (startRow == null || lastRow == null) 
+        range = new KeyExtent(new Text(tableId), null, null).toMetadataRange();
+      else
+        range = new Range(startRow, lastRow);
+      
+      Scanner scanner = instance.getConnector(credentials.getPrincipal(), credentials.getToken()).createScanner(MetadataTable.NAME, Authorizations.EMPTY);
+      scanner = new IsolatedScanner(scanner);
+      TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN.fetch(scanner);
+      scanner.fetchColumnFamily(TabletsSection.CurrentLocationColumnFamily.NAME);
+      scanner.setRange(range);
+      
+      RowIterator rowIter = new RowIterator(scanner);
+      
+      KeyExtent lastExtent = null;
+      
+      int total = 0;
+      int waitFor = 0;
+      int holes = 0;
+      Text continueRow = null;
+      MapCounter<String> serverCounts = new MapCounter<String>();
+      
+      while (rowIter.hasNext()) {
+        Iterator<Entry<Key,Value>> row = rowIter.next();
+        
+        total++;
+
+        KeyExtent extent = null;
+        String future = null;
+        String current = null;
+        
+        while (row.hasNext()) {
+          Entry<Key,Value> entry = row.next();
+          Key key = entry.getKey();
+          
+          if (key.getColumnFamily().equals(TabletsSection.FutureLocationColumnFamily.NAME))
+            future = entry.getValue().toString();
+          
+          if (key.getColumnFamily().equals(TabletsSection.CurrentLocationColumnFamily.NAME))
+            current = entry.getValue().toString();
+          
+          if (TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN.hasColumns(key))
+            extent = new KeyExtent(key.getRow(), entry.getValue());
+        }
+        
+        if ((expectedState == TableState.ONLINE && current == null) || (expectedState == TableState.OFFLINE && (future != null || current != null))) {
+          if (continueRow == null)
+            continueRow = extent.getMetadataEntry();
+          waitFor++;
+          lastRow = extent.getMetadataEntry();
+          
+          if(current != null)
+            serverCounts.increment(current, 1);
+          if(future != null)
+            serverCounts.increment(future, 1);
+        }
+        
+        if (!extent.getTableId().toString().equals(tableId)) {
+          throw new AccumuloException("Saw unexpected table Id " + tableId + " " + extent);
+        }
+        
+        if (lastExtent != null && !extent.isPreviousExtent(lastExtent)) {
+          holes++;
+        }
+        
+        lastExtent = extent;
+      }
+      
+      if (continueRow != null) {
+        startRow = continueRow;
+      }
+      
+      if (holes > 0 || total == 0) {
+        startRow = null;
+        lastRow = null;
+      }
+      
+      if (waitFor > 0 || holes > 0 || total == 0) {
+        long waitTime;
+        long maxPerServer = 0;
+        if(serverCounts.size() > 0){
+          maxPerServer = Collections.max(serverCounts.values());
+          waitTime = maxPerServer * 10;
+        }else
+          waitTime = waitFor * 10;
+        waitTime = Math.max(100, waitTime);
+        waitTime = Math.min(5000, waitTime);
+        log.trace("Waiting for " + waitFor + "("+maxPerServer+") tablets, startRow = " + startRow + " lastRow = "+lastRow+", holes=" + holes+" sleeping:"+waitTime+"ms");
+        UtilWaitThread.sleep(waitTime);
+      } else {
+        break;
+      }
+
+    }
+  }
+  @Override
+  public void offline(String tableName) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
+    offline(tableName, false);
+  }
+  
   /**
    * 
    * @param tableName
@@ -1138,10 +1262,11 @@ public class TableOperationsImpl extends TableOperationsHelper {
    * @throws TableNotFoundException
    */
   @Override
-  public void offline(String tableName) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
+  public void offline(String tableName, boolean wait) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
     
     ArgumentChecker.notNull(tableName);
-    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes()));
+    String tableId = Tables.getTableId(instance, tableName);
+    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableId.getBytes()));
     Map<String,String> opts = new HashMap<String,String>();
     
     try {
@@ -1150,6 +1275,14 @@ public class TableOperationsImpl extends TableOperationsHelper {
       // should not happen
       throw new RuntimeException(e);
     }
+    
+    if(wait)
+      waitForTableStateTransition(tableId, TableState.OFFLINE);
+  }
+  
+  @Override
+  public void online(String tableName) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
+    online(tableName, false);
   }
   
   /**
@@ -1163,9 +1296,10 @@ public class TableOperationsImpl extends TableOperationsHelper {
    * @throws TableNotFoundException
    */
   @Override
-  public void online(String tableName) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
+  public void online(String tableName, boolean wait) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
     ArgumentChecker.notNull(tableName);
-    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes()));
+    String tableId = Tables.getTableId(instance, tableName);
+    List<ByteBuffer> args = Arrays.asList(ByteBuffer.wrap(tableId.getBytes()));
     Map<String,String> opts = new HashMap<String,String>();
     
     try {
@@ -1174,6 +1308,9 @@ public class TableOperationsImpl extends TableOperationsHelper {
       // should not happen
       throw new RuntimeException(e);
     }
+    
+    if(wait)
+      waitForTableStateTransition(tableId, TableState.ONLINE);
   }
   
   /**

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e70a40db/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 91021e0..a3c2043 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
@@ -281,12 +281,22 @@ public class MockTableOperations extends TableOperationsHelper {
   
   @Override
   public void offline(String tableName) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
+    offline(tableName, false);
+  }
+  
+  @Override
+  public void offline(String tableName, boolean wait) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
     if (!exists(tableName))
       throw new TableNotFoundException(tableName, tableName, "");
   }
   
   @Override
   public void online(String tableName) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
+    online(tableName, false);
+  }
+  
+  @Override
+  public void online(String tableName, boolean wait) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
     if (!exists(tableName))
       throw new TableNotFoundException(tableName, tableName, "");
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e70a40db/core/src/main/java/org/apache/accumulo/core/util/MapCounter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/MapCounter.java b/core/src/main/java/org/apache/accumulo/core/util/MapCounter.java
new file mode 100644
index 0000000..30f9f1d
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/util/MapCounter.java
@@ -0,0 +1,86 @@
+/*
+ * 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.util;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Set;
+
+public class MapCounter<KT> {
+  
+  static class MutableLong {
+    long l = 0l;
+  }
+  
+  private HashMap<KT,MutableLong> map;
+  
+  public MapCounter() {
+    map = new HashMap<KT,MutableLong>();
+  }
+  
+  public long increment(KT key, long l) {
+    MutableLong ml = map.get(key);
+    if (ml == null) {
+      ml = new MutableLong();
+      map.put(key, ml);
+    }
+    
+    ml.l += l;
+    
+    if (ml.l == 0) {
+      map.remove(key);
+    }
+    
+    return ml.l;
+  }
+  
+  public long decrement(KT key, long l) {
+    return increment(key, -1 * l);
+  }
+  
+  public boolean contains(KT key) {
+    return map.containsKey(key);
+  }
+  
+  public long get(KT key) {
+    MutableLong ml = map.get(key);
+    if (ml == null) {
+      return 0;
+    }
+    
+    return ml.l;
+  }
+  
+  public Set<KT> keySet() {
+    return map.keySet();
+  }
+  
+  public Collection<Long> values() {
+    Collection<MutableLong> vals = map.values();
+    ArrayList<Long> ret = new ArrayList<Long>(vals.size());
+    for (MutableLong ml : vals) {
+      ret.add(ml.l);
+    }
+    
+    return ret;
+  }
+  
+  public int size() {
+    return map.size();
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e70a40db/core/src/main/java/org/apache/accumulo/core/util/shell/commands/OfflineCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/OfflineCommand.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/OfflineCommand.java
index 70de3d4..fde4514 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/OfflineCommand.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/OfflineCommand.java
@@ -21,8 +21,15 @@ import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.util.shell.Shell;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.Options;
 
 public class OfflineCommand extends TableOperation {
+  
+  private boolean wait;
+  private Option waitOpt;
+  
   @Override
   public String description() {
     return "starts the process of taking table offline";
@@ -32,8 +39,23 @@ public class OfflineCommand extends TableOperation {
     if (tableName.equals(MetadataTable.NAME)) {
       Shell.log.info("  You cannot take the " + MetadataTable.NAME + " offline.");
     } else {
-      Shell.log.info("Attempting to begin taking " + tableName + " offline");
-      shellState.getConnector().tableOperations().offline(tableName);
+      shellState.getConnector().tableOperations().offline(tableName, wait);
+      Shell.log.info("Offline of table " + tableName + (wait ? " completed." : " initiated..."));
     }
   }
+  
+  
+  @Override
+  public int execute(final String fullCommand, final CommandLine cl, final Shell shellState) throws Exception {
+    wait = cl.hasOption(waitOpt.getLongOpt());
+    return super.execute(fullCommand, cl, shellState);
+  }
+  
+  @Override
+  public Options getOptions() {
+    final Options opts = super.getOptions();
+    waitOpt = new Option("w", "wait", false, "wait for offline to finish");
+    opts.addOption(waitOpt); 
+    return opts;
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e70a40db/core/src/main/java/org/apache/accumulo/core/util/shell/commands/OnlineCommand.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/OnlineCommand.java b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/OnlineCommand.java
index 5ffbe3a..bbf2b72 100644
--- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/OnlineCommand.java
+++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/OnlineCommand.java
@@ -21,19 +21,41 @@ import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.util.shell.Shell;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.Options;
 
 public class OnlineCommand extends TableOperation {
+  
+  private boolean wait;
+  private Option waitOpt;
+  
   @Override
   public String description() {
     return "starts the process of putting a table online";
   }
   
+  @Override
   protected void doTableOp(final Shell shellState, final String tableName) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
     if (tableName.equals(MetadataTable.NAME)) {
       Shell.log.info("  The " + MetadataTable.NAME + " is always online.");
     } else {
-      Shell.log.info("Attempting to begin bringing " + tableName + " online");
-      shellState.getConnector().tableOperations().online(tableName);
+      shellState.getConnector().tableOperations().online(tableName, wait);
+      Shell.log.info("Online of table " + tableName + (wait ? " completed." : " initiated..."));
     }
   }
+  
+  @Override
+  public int execute(final String fullCommand, final CommandLine cl, final Shell shellState) throws Exception {
+    wait = cl.hasOption(waitOpt.getLongOpt());
+    return super.execute(fullCommand, cl, shellState);
+  }
+  
+  @Override
+  public Options getOptions() {
+    final Options opts = super.getOptions();
+    waitOpt = new Option("w", "wait", false, "wait for online to finish");
+    opts.addOption(waitOpt); 
+    return opts;
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e70a40db/core/src/test/java/org/apache/accumulo/core/client/admin/TableOperationsHelperTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/admin/TableOperationsHelperTest.java b/core/src/test/java/org/apache/accumulo/core/client/admin/TableOperationsHelperTest.java
index 913c2a8..32136a8 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/admin/TableOperationsHelperTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/admin/TableOperationsHelperTest.java
@@ -180,6 +180,14 @@ public class TableOperationsHelperTest {
     public void online(String tableName) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {}
     
     @Override
+    public void offline(String tableName, boolean wait) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {
+      
+    }
+    
+    @Override
+    public void online(String tableName, boolean wait) throws AccumuloSecurityException, AccumuloException, TableNotFoundException {}
+    
+    @Override
     public void clearLocatorCache(String tableName) throws TableNotFoundException {}
     
     @Override

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e70a40db/server/src/main/java/org/apache/accumulo/server/master/Master.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/master/Master.java b/server/src/main/java/org/apache/accumulo/server/master/Master.java
index 62ea7d7..9089975 100644
--- a/server/src/main/java/org/apache/accumulo/server/master/Master.java
+++ b/server/src/main/java/org/apache/accumulo/server/master/Master.java
@@ -383,6 +383,14 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
     return result;
   }
   
+  private void checkNotMetadataID(String tableId, TableOperation operation) throws ThriftTableOperationException {
+    if (MetadataTable.ID.equals(tableId) || RootTable.ID.equals(tableId)) {
+      String why = "Table names cannot be == " + RootTable.NAME + " or " + MetadataTable.NAME;
+      log.warn(why);
+      throw new ThriftTableOperationException(tableId, null, operation, TableOperationExceptionType.OTHER, why);
+    }
+  }
+  
   private void checkNotMetadataTable(String tableName, TableOperation operation) throws ThriftTableOperationException {
     if (MetadataTable.NAME.equals(tableName) || RootTable.NAME.equals(tableName)) {
       String why = "Table names cannot be == " + RootTable.NAME + " or " + MetadataTable.NAME;
@@ -876,9 +884,8 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
           break;
         }
         case ONLINE: {
-          String tableName = ByteBufferUtil.toString(arguments.get(0));
-          final String tableId = checkTableId(tableName, TableOperation.ONLINE);
-          checkNotMetadataTable(tableName, TableOperation.ONLINE);
+          final String tableId = ByteBufferUtil.toString(arguments.get(0));
+          checkNotMetadataID(tableId, TableOperation.ONLINE);
           
           if (!security.canOnlineOfflineTable(c, tableId, op))
             throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);
@@ -887,9 +894,8 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
           break;
         }
         case OFFLINE: {
-          String tableName = ByteBufferUtil.toString(arguments.get(0));
-          final String tableId = checkTableId(tableName, TableOperation.OFFLINE);
-          checkNotMetadataTable(tableName, TableOperation.OFFLINE);
+          final String tableId = ByteBufferUtil.toString(arguments.get(0));
+          checkNotMetadataID(tableId, TableOperation.OFFLINE);
           
           if (!security.canOnlineOfflineTable(c, tableId, op))
             throw new ThriftSecurityException(c.getPrincipal(), SecurityErrorCode.PERMISSION_DENIED);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e70a40db/server/src/main/java/org/apache/accumulo/server/master/tableOps/CompactRange.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/master/tableOps/CompactRange.java b/server/src/main/java/org/apache/accumulo/server/master/tableOps/CompactRange.java
index 41b052b..e162cc7 100644
--- a/server/src/main/java/org/apache/accumulo/server/master/tableOps/CompactRange.java
+++ b/server/src/main/java/org/apache/accumulo/server/master/tableOps/CompactRange.java
@@ -45,6 +45,7 @@ import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.RootTable;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
 import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.core.util.MapCounter;
 import org.apache.accumulo.fate.Repo;
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 import org.apache.accumulo.fate.zookeeper.IZooReaderWriter.Mutator;
@@ -52,7 +53,6 @@ import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.accumulo.server.master.LiveTServerSet.TServerConnection;
 import org.apache.accumulo.server.master.Master;
 import org.apache.accumulo.server.master.state.TServerInstance;
-import org.apache.accumulo.server.util.MapCounter;
 import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 import org.apache.commons.codec.binary.Hex;
 import org.apache.hadoop.io.Text;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e70a40db/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java b/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java
index bd15958..cbafeb0 100644
--- a/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java
+++ b/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java
@@ -90,6 +90,7 @@ import org.apache.accumulo.core.security.ColumnVisibility;
 import org.apache.accumulo.core.security.Credentials;
 import org.apache.accumulo.core.util.CachedConfiguration;
 import org.apache.accumulo.core.util.LocalityGroupUtil;
+import org.apache.accumulo.core.util.MapCounter;
 import org.apache.accumulo.core.util.LocalityGroupUtil.LocalityGroupConfigurationError;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.core.util.UtilWaitThread;
@@ -119,7 +120,6 @@ import org.apache.accumulo.server.tabletserver.log.MutationReceiver;
 import org.apache.accumulo.server.tabletserver.mastermessage.TabletStatusMessage;
 import org.apache.accumulo.server.tabletserver.metrics.TabletServerMinCMetrics;
 import org.apache.accumulo.server.util.FileUtil;
-import org.apache.accumulo.server.util.MapCounter;
 import org.apache.accumulo.server.util.MetadataTableUtil;
 import org.apache.accumulo.server.util.MetadataTableUtil.LogEntry;
 import org.apache.accumulo.server.util.TabletOperations;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e70a40db/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java b/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java
index d3c554b..abb8750 100644
--- a/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java
+++ b/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java
@@ -129,6 +129,7 @@ import org.apache.accumulo.core.util.ByteBufferUtil;
 import org.apache.accumulo.core.util.ColumnFQ;
 import org.apache.accumulo.core.util.Daemon;
 import org.apache.accumulo.core.util.LoggingRunnable;
+import org.apache.accumulo.core.util.MapCounter;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.core.util.ServerServices;
 import org.apache.accumulo.core.util.ServerServices.Service;
@@ -193,7 +194,6 @@ import org.apache.accumulo.server.tabletserver.metrics.TabletServerScanMetrics;
 import org.apache.accumulo.server.tabletserver.metrics.TabletServerUpdateMetrics;
 import org.apache.accumulo.server.util.FileSystemMonitor;
 import org.apache.accumulo.server.util.Halt;
-import org.apache.accumulo.server.util.MapCounter;
 import org.apache.accumulo.server.util.MetadataTableUtil;
 import org.apache.accumulo.server.util.MetadataTableUtil.LogEntry;
 import org.apache.accumulo.server.util.TServerUtils;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e70a40db/server/src/main/java/org/apache/accumulo/server/util/MapCounter.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/util/MapCounter.java b/server/src/main/java/org/apache/accumulo/server/util/MapCounter.java
deleted file mode 100644
index 2731540..0000000
--- a/server/src/main/java/org/apache/accumulo/server/util/MapCounter.java
+++ /dev/null
@@ -1,86 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.server.util;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Set;
-
-public class MapCounter<KT> {
-  
-  static class MutableLong {
-    long l = 0l;
-  }
-  
-  private HashMap<KT,MutableLong> map;
-  
-  public MapCounter() {
-    map = new HashMap<KT,MutableLong>();
-  }
-  
-  public long increment(KT key, long l) {
-    MutableLong ml = map.get(key);
-    if (ml == null) {
-      ml = new MutableLong();
-      map.put(key, ml);
-    }
-    
-    ml.l += l;
-    
-    if (ml.l == 0) {
-      map.remove(key);
-    }
-    
-    return ml.l;
-  }
-  
-  public long decrement(KT key, long l) {
-    return increment(key, -1 * l);
-  }
-  
-  public boolean contains(KT key) {
-    return map.containsKey(key);
-  }
-  
-  public long get(KT key) {
-    MutableLong ml = map.get(key);
-    if (ml == null) {
-      return 0;
-    }
-    
-    return ml.l;
-  }
-  
-  public Set<KT> keySet() {
-    return map.keySet();
-  }
-  
-  public Collection<Long> values() {
-    Collection<MutableLong> vals = map.values();
-    ArrayList<Long> ret = new ArrayList<Long>(vals.size());
-    for (MutableLong ml : vals) {
-      ret.add(ml.l);
-    }
-    
-    return ret;
-  }
-  
-  public int size() {
-    return map.size();
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e70a40db/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/OfflineTable.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/OfflineTable.java b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/OfflineTable.java
index a6a9e5c..c4d9bab 100644
--- a/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/OfflineTable.java
+++ b/test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/OfflineTable.java
@@ -40,10 +40,10 @@ public class OfflineTable extends Test {
     String tableName = tableNames.get(rand.nextInt(tableNames.size()));
     
     try {
-      conn.tableOperations().offline(tableName);
+      conn.tableOperations().offline(tableName, rand.nextBoolean());
       log.debug("Offlined " + tableName);
       UtilWaitThread.sleep(rand.nextInt(200));
-      conn.tableOperations().online(tableName);
+      conn.tableOperations().online(tableName, rand.nextBoolean());
       log.debug("Onlined " + tableName);
     } catch (TableNotFoundException tne) {
       log.debug("offline or online failed " + tableName + ", doesnt exist");

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e70a40db/test/src/main/java/org/apache/accumulo/test/randomwalk/multitable/OfflineTable.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/randomwalk/multitable/OfflineTable.java b/test/src/main/java/org/apache/accumulo/test/randomwalk/multitable/OfflineTable.java
index bcec93a..3e16f4d 100644
--- a/test/src/main/java/org/apache/accumulo/test/randomwalk/multitable/OfflineTable.java
+++ b/test/src/main/java/org/apache/accumulo/test/randomwalk/multitable/OfflineTable.java
@@ -38,9 +38,9 @@ public class OfflineTable extends Test {
     Random rand = new Random();
     String tableName = tables.get(rand.nextInt(tables.size()));
     
-    state.getConnector().tableOperations().offline(tableName);
+    state.getConnector().tableOperations().offline(tableName, rand.nextBoolean());
     log.debug("Table " + tableName + " offline ");
-    state.getConnector().tableOperations().online(tableName);
+    state.getConnector().tableOperations().online(tableName, rand.nextBoolean());
     log.debug("Table " + tableName + " online ");
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e70a40db/test/src/test/java/org/apache/accumulo/test/ConditionalWriterTest.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/ConditionalWriterTest.java b/test/src/test/java/org/apache/accumulo/test/ConditionalWriterTest.java
index 633ea76..70f115b 100644
--- a/test/src/test/java/org/apache/accumulo/test/ConditionalWriterTest.java
+++ b/test/src/test/java/org/apache/accumulo/test/ConditionalWriterTest.java
@@ -51,8 +51,6 @@ import org.apache.accumulo.core.client.TableDeletedException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.TableOfflineException;
 import org.apache.accumulo.core.client.ZooKeeperInstance;
-import org.apache.accumulo.core.client.impl.Tables;
-import org.apache.accumulo.core.client.impl.TabletLocator;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.data.ArrayByteSequence;
 import org.apache.accumulo.core.data.ByteSequence;
@@ -67,7 +65,6 @@ import org.apache.accumulo.core.iterators.user.SummingCombiner;
 import org.apache.accumulo.core.iterators.user.VersioningIterator;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.ColumnVisibility;
-import org.apache.accumulo.core.security.Credentials;
 import org.apache.accumulo.core.security.TablePermission;
 import org.apache.accumulo.core.util.FastFormat;
 import org.apache.accumulo.core.util.UtilWaitThread;
@@ -1176,9 +1173,7 @@ public class ConditionalWriterTest {
     
     ConditionalWriter cw = conn.createConditionalWriter(table, new ConditionalWriterConfig());
     
-    conn.tableOperations().offline(table);
-    
-    waitForSingleTabletTableToGoOffline(table, zki);
+    conn.tableOperations().offline(table, true);
     
     ConditionalMutation cm1 = new ConditionalMutation("r1", new Condition("tx", "seq"));
     cm1.put("tx", "seq", "1");
@@ -1202,14 +1197,6 @@ public class ConditionalWriterTest {
     zki.close();
   }
   
-  void waitForSingleTabletTableToGoOffline(String table, ZooKeeperInstance zki) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-    TabletLocator locator = TabletLocator.getLocator(zki, new Text(Tables.getNameToIdMap(zki).get(table)));
-    while (locator.locateTablet(new Credentials("root", new PasswordToken(secret)), new Text("a"), false, false) != null) {
-      UtilWaitThread.sleep(50);
-      locator.invalidateCache();
-    }
-  }
-  
   @Test
   public void testError() throws Exception {
     String table = "foo10";

http://git-wip-us.apache.org/repos/asf/accumulo/blob/e70a40db/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java b/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
index 4fbd293..d526cd4 100644
--- a/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
@@ -779,9 +779,8 @@ public class ShellServerIT {
     
     exec("constraint -a FooConstraint", true);
     
-    exec("offline ptc");
-    UtilWaitThread.sleep(500);
-    exec("online ptc");
+    exec("offline -w ptc");
+    exec("online -w ptc");
     
     exec("table ptc", true);
     exec("insert foo f q v", false);