You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by ec...@apache.org on 2013/11/21 18:54:48 UTC

[1/8] git commit: ACCUMULO-1866 deprecated Instance.getConfiguration()

Updated Branches:
  refs/heads/1.6.0-SNAPSHOT efb424680 -> fe46a60c4
  refs/heads/master 1fba05a8e -> 9b4115582


ACCUMULO-1866 deprecated Instance.getConfiguration()


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

Branch: refs/heads/master
Commit: efb424680fd7edb1a475fbb2015af2bc8d0583a4
Parents: 42853c1
Author: Keith Turner <kt...@apache.org>
Authored: Thu Nov 21 11:09:24 2013 -0500
Committer: Keith Turner <kt...@apache.org>
Committed: Thu Nov 21 11:09:24 2013 -0500

----------------------------------------------------------------------
 .../apache/accumulo/core/client/Instance.java   |  1 +
 .../client/admin/InstanceOperationsImpl.java    |  7 +++--
 .../core/client/admin/TableOperationsImpl.java  |  5 ++--
 .../core/client/impl/ConditionalWriterImpl.java |  6 ++--
 .../accumulo/core/client/impl/MasterClient.java |  2 +-
 .../core/client/impl/OfflineScanner.java        |  4 +--
 .../core/client/impl/ScannerIterator.java       |  2 +-
 .../accumulo/core/client/impl/ServerClient.java |  4 +--
 .../client/impl/ServerConfigurationFactory.java | 30 ++++++++++++++++++++
 .../impl/TabletServerBatchReaderIterator.java   |  2 +-
 .../client/impl/TabletServerBatchWriter.java    |  6 ++--
 .../accumulo/core/client/impl/Writer.java       |  2 +-
 .../core/metadata/MetadataLocationObtainer.java |  7 +++--
 .../accumulo/server/client/BulkImporter.java    |  9 +++---
 .../gc/GarbageCollectWriteAheadLogs.java        |  3 +-
 .../accumulo/gc/SimpleGarbageCollector.java     | 13 +++++----
 .../java/org/apache/accumulo/master/Master.java |  3 +-
 17 files changed, 72 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/efb42468/core/src/main/java/org/apache/accumulo/core/client/Instance.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/Instance.java b/core/src/main/java/org/apache/accumulo/core/client/Instance.java
index f8a7682..ab8ca19 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/Instance.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/Instance.java
@@ -132,6 +132,7 @@ public interface Instance {
    * Returns the AccumuloConfiguration to use when interacting with this instance.
    * 
    * @return the AccumuloConfiguration that specifies properties related to interacting with this instance
+   * @deprecated since 1.6.0
    */
   public abstract AccumuloConfiguration getConfiguration();
   

http://git-wip-us.apache.org/repos/asf/accumulo/blob/efb42468/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperationsImpl.java
index a131b66..333201e 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperationsImpl.java
@@ -30,6 +30,7 @@ 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.ServerClient;
+import org.apache.accumulo.core.client.impl.ServerConfigurationFactory;
 import org.apache.accumulo.core.client.impl.thrift.ClientService;
 import org.apache.accumulo.core.client.impl.thrift.ConfigurationType;
 import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
@@ -131,7 +132,7 @@ public class InstanceOperationsImpl implements InstanceOperations {
   public List<ActiveScan> getActiveScans(String tserver) throws AccumuloException, AccumuloSecurityException {
     Client client = null;
     try {
-      client = ThriftUtil.getTServerClient(tserver, instance.getConfiguration());
+      client = ThriftUtil.getTServerClient(tserver, ServerConfigurationFactory.getConfiguration(instance));
       
       List<ActiveScan> as = new ArrayList<ActiveScan>();
       for (org.apache.accumulo.core.tabletserver.thrift.ActiveScan activeScan : client.getActiveScans(Tracer.traceInfo(), credentials.toThrift(instance))) {
@@ -168,7 +169,7 @@ public class InstanceOperationsImpl implements InstanceOperations {
   public List<ActiveCompaction> getActiveCompactions(String tserver) throws AccumuloException, AccumuloSecurityException {
     Client client = null;
     try {
-      client = ThriftUtil.getTServerClient(tserver, instance.getConfiguration());
+      client = ThriftUtil.getTServerClient(tserver, ServerConfigurationFactory.getConfiguration(instance));
       
       List<ActiveCompaction> as = new ArrayList<ActiveCompaction>();
       for (org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction activeCompaction : client.getActiveCompactions(Tracer.traceInfo(),
@@ -192,7 +193,7 @@ public class InstanceOperationsImpl implements InstanceOperations {
   public void ping(String tserver) throws AccumuloException {
     TTransport transport = null;
     try {
-      transport = ThriftUtil.createTransport(AddressUtil.parseAddress(tserver), instance.getConfiguration());
+      transport = ThriftUtil.createTransport(AddressUtil.parseAddress(tserver), ServerConfigurationFactory.getConfiguration(instance));
       TabletClientService.Client client = ThriftUtil.createClient(new TabletClientService.Client.Factory(), transport);
       client.getTabletServerStatus(Tracer.traceInfo(), credentials.toThrift(instance));
     } catch (TTransportException e) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/efb42468/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 85d9fa2..a85772d 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
@@ -61,6 +61,7 @@ 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.ServerClient;
+import org.apache.accumulo.core.client.impl.ServerConfigurationFactory;
 import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.client.impl.TabletLocator;
 import org.apache.accumulo.core.client.impl.TabletLocator.TabletLocation;
@@ -469,7 +470,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
         }
         
         try {
-          TabletClientService.Client client = ThriftUtil.getTServerClient(tl.tablet_location, instance.getConfiguration());
+          TabletClientService.Client client = ThriftUtil.getTServerClient(tl.tablet_location, ServerConfigurationFactory.getConfiguration(instance));
           try {
             OpTimer opTimer = null;
             if (log.isTraceEnabled())
@@ -1112,7 +1113,7 @@ public class TableOperationsImpl extends TableOperationsHelper {
       ret = new Path(dir);
       fs = ret.getFileSystem(CachedConfiguration.getInstance());
     } else {
-      fs = FileUtil.getFileSystem(CachedConfiguration.getInstance(), instance.getConfiguration());
+      fs = FileUtil.getFileSystem(CachedConfiguration.getInstance(), ServerConfigurationFactory.getConfiguration(instance));
       ret = fs.makeQualified(new Path(dir));
     }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/efb42468/core/src/main/java/org/apache/accumulo/core/client/impl/ConditionalWriterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ConditionalWriterImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ConditionalWriterImpl.java
index 6ba27b9..1d35af4 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ConditionalWriterImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ConditionalWriterImpl.java
@@ -536,10 +536,10 @@ class ConditionalWriterImpl implements ConditionalWriter {
   
   private TabletClientService.Iface getClient(String location) throws TTransportException {
     TabletClientService.Iface client;
-    if (timeout < instance.getConfiguration().getTimeInMillis(Property.GENERAL_RPC_TIMEOUT))
-      client = ThriftUtil.getTServerClient(location, instance.getConfiguration(), timeout);
+    if (timeout < ServerConfigurationFactory.getConfiguration(instance).getTimeInMillis(Property.GENERAL_RPC_TIMEOUT))
+      client = ThriftUtil.getTServerClient(location, ServerConfigurationFactory.getConfiguration(instance), timeout);
     else
-      client = ThriftUtil.getTServerClient(location, instance.getConfiguration());
+      client = ThriftUtil.getTServerClient(location, ServerConfigurationFactory.getConfiguration(instance));
     return client;
   }
   

http://git-wip-us.apache.org/repos/asf/accumulo/blob/efb42468/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 dd28fca..4cf6e06 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
@@ -61,7 +61,7 @@ public class MasterClient {
     
     try {
       // Master requests can take a long time: don't ever time out
-      MasterClientService.Client client = ThriftUtil.getClientNoTimeout(new MasterClientService.Client.Factory(), master, instance.getConfiguration());
+      MasterClientService.Client client = ThriftUtil.getClientNoTimeout(new MasterClientService.Client.Factory(), master, ServerConfigurationFactory.getConfiguration(instance));
       return client;
     } catch (TTransportException tte) {
       if (tte.getCause().getClass().equals(UnknownHostException.class)) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/efb42468/core/src/main/java/org/apache/accumulo/core/client/impl/OfflineScanner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/OfflineScanner.java b/core/src/main/java/org/apache/accumulo/core/client/impl/OfflineScanner.java
index 0231fad..5e92d8b 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/OfflineScanner.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/OfflineScanner.java
@@ -227,7 +227,7 @@ class OfflineIterator implements Iterator<Entry<Key,Value>> {
     if (currentExtent != null && !extent.isPreviousExtent(currentExtent))
       throw new AccumuloException(" " + currentExtent + " is not previous extent " + extent);
     
-    String tablesDir = instance.getConfiguration().get(Property.INSTANCE_DFS_DIR) + "/tables";
+    String tablesDir = ServerConfigurationFactory.getConfiguration(instance).get(Property.INSTANCE_DFS_DIR) + "/tables";
 
     List<String> absFiles = new ArrayList<String>();
     for (String relPath : relFiles) {
@@ -296,7 +296,7 @@ class OfflineIterator implements Iterator<Entry<Key,Value>> {
     
     Configuration conf = CachedConfiguration.getInstance();
     
-    FileSystem defaultFs = FileUtil.getFileSystem(conf, instance.getConfiguration());
+    FileSystem defaultFs = FileUtil.getFileSystem(conf, ServerConfigurationFactory.getConfiguration(instance));
     
     for (SortedKeyValueIterator<Key,Value> reader : readers) {
       ((FileSKVIterator) reader).close();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/efb42468/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerIterator.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerIterator.java
index b202d3a..677a751 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerIterator.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerIterator.java
@@ -81,7 +81,7 @@ public class ScannerIterator implements Iterator<Entry<Key,Value>> {
       
       try {
         while (true) {
-          List<KeyValue> currentBatch = ThriftScanner.scan(instance, credentials, scanState, timeOut, instance.getConfiguration());
+          List<KeyValue> currentBatch = ThriftScanner.scan(instance, credentials, scanState, timeOut, ServerConfigurationFactory.getConfiguration(instance));
           
           if (currentBatch == null) {
             synchQ.add(EMPTY_LIST);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/efb42468/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java
index 90db5ee..89956db 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java
@@ -125,7 +125,7 @@ public class ServerClient {
   }
   
   public static Pair<String,ClientService.Client> getConnection(Instance instance, boolean preferCachedConnections) throws TTransportException {
-    AccumuloConfiguration conf = instance.getConfiguration();
+    AccumuloConfiguration conf = ServerConfigurationFactory.getConfiguration(instance);
     return getConnection(instance, preferCachedConnections, conf.getTimeInMillis(Property.GENERAL_RPC_TIMEOUT));
   }
   
@@ -142,7 +142,7 @@ public class ServerClient {
       if (data != null && !new String(data).equals("master"))
         servers.add(new ThriftTransportKey(
           new ServerServices(new String(data)).getAddressString(Service.TSERV_CLIENT),
-          rpcTimeout, SslConnectionParams.forClient(instance.getConfiguration())));
+          rpcTimeout, SslConnectionParams.forClient(ServerConfigurationFactory.getConfiguration(instance))));
     }
     
     boolean opened = false;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/efb42468/core/src/main/java/org/apache/accumulo/core/client/impl/ServerConfigurationFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ServerConfigurationFactory.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ServerConfigurationFactory.java
new file mode 100644
index 0000000..2c3427a
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ServerConfigurationFactory.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.core.client.impl;
+
+import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+
+/**
+ * All client side code that needs a server side configuration object should obtain it from here.
+ */
+public class ServerConfigurationFactory {
+  @SuppressWarnings("deprecation")
+  public static AccumuloConfiguration getConfiguration(Instance instance) {
+    return instance.getConfiguration();
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/efb42468/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java
index 7718207..9961f8f 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java
@@ -362,7 +362,7 @@ public class TabletServerBatchReaderIterator implements Iterator<Entry<Key,Value
           timeoutTrackers.put(tsLocation, timeoutTracker);
         }
         doLookup(instance, credentials, tsLocation, tabletsRanges, tsFailures, unscanned, receiver, columns, options, authorizations,
-            instance.getConfiguration(), timeoutTracker);
+            ServerConfigurationFactory.getConfiguration(instance), timeoutTracker);
         if (tsFailures.size() > 0) {
           locator.invalidateCache(tsFailures.keySet());
           synchronized (failures) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/efb42468/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java
index e2c2802..b79ae39 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java
@@ -855,10 +855,10 @@ public class TabletServerBatchWriter {
       try {
         TabletClientService.Iface client;
         
-        if (timeoutTracker.getTimeOut() < instance.getConfiguration().getTimeInMillis(Property.GENERAL_RPC_TIMEOUT))
-          client = ThriftUtil.getTServerClient(location, instance.getConfiguration(), timeoutTracker.getTimeOut());
+        if (timeoutTracker.getTimeOut() < ServerConfigurationFactory.getConfiguration(instance).getTimeInMillis(Property.GENERAL_RPC_TIMEOUT))
+          client = ThriftUtil.getTServerClient(location, ServerConfigurationFactory.getConfiguration(instance), timeoutTracker.getTimeOut());
         else
-          client = ThriftUtil.getTServerClient(location, instance.getConfiguration());
+          client = ThriftUtil.getTServerClient(location, ServerConfigurationFactory.getConfiguration(instance));
         
         try {
           MutationSet allFailures = new MutationSet();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/efb42468/core/src/main/java/org/apache/accumulo/core/client/impl/Writer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/Writer.java b/core/src/main/java/org/apache/accumulo/core/client/impl/Writer.java
index 11c67ca..e253024 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/Writer.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/Writer.java
@@ -93,7 +93,7 @@ public class Writer {
       }
       
       try {
-        updateServer(instance, m, tabLoc.tablet_extent, tabLoc.tablet_location, credentials, instance.getConfiguration());
+        updateServer(instance, m, tabLoc.tablet_extent, tabLoc.tablet_location, credentials, ServerConfigurationFactory.getConfiguration(instance));
         return;
       } catch (NotServingTabletException e) {
         log.trace("Not serving tablet, server = " + tabLoc.tablet_location);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/efb42468/core/src/main/java/org/apache/accumulo/core/metadata/MetadataLocationObtainer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/MetadataLocationObtainer.java b/core/src/main/java/org/apache/accumulo/core/metadata/MetadataLocationObtainer.java
index 0fb1a8e..cb86b77 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/MetadataLocationObtainer.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/MetadataLocationObtainer.java
@@ -34,6 +34,7 @@ import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.impl.AccumuloServerException;
 import org.apache.accumulo.core.client.impl.ScannerOptions;
+import org.apache.accumulo.core.client.impl.ServerConfigurationFactory;
 import org.apache.accumulo.core.client.impl.TabletLocator;
 import org.apache.accumulo.core.client.impl.TabletLocator.TabletLocation;
 import org.apache.accumulo.core.client.impl.TabletLocator.TabletLocations;
@@ -97,7 +98,7 @@ public class MetadataLocationObtainer implements TabletLocationObtainer {
       Map<String,Map<String,String>> serverSideIteratorOptions = Collections.emptyMap();
       
       boolean more = ThriftScanner.getBatchFromServer(instance, credentials, range, src.tablet_extent, src.tablet_location, encodedResults, locCols,
-          serverSideIteratorList, serverSideIteratorOptions, Constants.SCAN_BATCH_SIZE, Authorizations.EMPTY, false, instance.getConfiguration());
+          serverSideIteratorList, serverSideIteratorOptions, Constants.SCAN_BATCH_SIZE, Authorizations.EMPTY, false, ServerConfigurationFactory.getConfiguration(instance));
       
       decodeRows(encodedResults, results);
       
@@ -105,7 +106,7 @@ public class MetadataLocationObtainer implements TabletLocationObtainer {
         range = new Range(results.lastKey().followingKey(PartialKey.ROW_COLFAM_COLQUAL_COLVIS_TIME), true, new Key(stopRow).followingKey(PartialKey.ROW), false);
         encodedResults.clear();
         more = ThriftScanner.getBatchFromServer(instance, credentials, range, src.tablet_extent, src.tablet_location, encodedResults, locCols,
-            serverSideIteratorList, serverSideIteratorOptions, Constants.SCAN_BATCH_SIZE, Authorizations.EMPTY, false, instance.getConfiguration());
+            serverSideIteratorList, serverSideIteratorOptions, Constants.SCAN_BATCH_SIZE, Authorizations.EMPTY, false, ServerConfigurationFactory.getConfiguration(instance));
         
         decodeRows(encodedResults, results);
       }
@@ -178,7 +179,7 @@ public class MetadataLocationObtainer implements TabletLocationObtainer {
     Map<KeyExtent,List<Range>> failures = new HashMap<KeyExtent,List<Range>>();
     try {
       TabletServerBatchReaderIterator.doLookup(instance, credentials, tserver, tabletsRanges, failures, unscanned, rr, columns, opts, Authorizations.EMPTY,
-          instance.getConfiguration());
+          ServerConfigurationFactory.getConfiguration(instance));
       if (failures.size() > 0) {
         // invalidate extents in parents cache
         if (log.isTraceEnabled())

http://git-wip-us.apache.org/repos/asf/accumulo/blob/efb42468/server/base/src/main/java/org/apache/accumulo/server/client/BulkImporter.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/client/BulkImporter.java b/server/base/src/main/java/org/apache/accumulo/server/client/BulkImporter.java
index ecbe0c4..ff822ef 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/client/BulkImporter.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/client/BulkImporter.java
@@ -36,6 +36,7 @@ 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.impl.ServerClient;
+import org.apache.accumulo.core.client.impl.ServerConfigurationFactory;
 import org.apache.accumulo.core.client.impl.TabletLocator;
 import org.apache.accumulo.core.client.impl.TabletLocator.TabletLocation;
 import org.apache.accumulo.core.client.impl.Translator;
@@ -144,7 +145,7 @@ public class BulkImporter {
           public void run() {
             List<TabletLocation> tabletsToAssignMapFileTo = Collections.emptyList();
             try {
-              tabletsToAssignMapFileTo = findOverlappingTablets(instance.getConfiguration(), fs, locator, mapFile, credentials);
+              tabletsToAssignMapFileTo = findOverlappingTablets(ServerConfigurationFactory.getConfiguration(instance), fs, locator, mapFile, credentials);
             } catch (Exception ex) {
               log.warn("Unable to find tablets that overlap file " + mapFile.toString());
             }
@@ -207,7 +208,7 @@ public class BulkImporter {
             
             try {
               timer.start(Timers.QUERY_METADATA);
-              tabletsToAssignMapFileTo.addAll(findOverlappingTablets(instance.getConfiguration(), fs, locator, entry.getKey(), ke, credentials));
+              tabletsToAssignMapFileTo.addAll(findOverlappingTablets(ServerConfigurationFactory.getConfiguration(instance), fs, locator, entry.getKey(), ke, credentials));
               timer.stop(Timers.QUERY_METADATA);
               keListIter.remove();
             } catch (Exception ex) {
@@ -584,8 +585,8 @@ public class BulkImporter {
   private List<KeyExtent> assignMapFiles(Credentials credentials, String location, Map<KeyExtent,List<PathSize>> assignmentsPerTablet)
       throws AccumuloException, AccumuloSecurityException {
     try {
-      long timeInMillis = instance.getConfiguration().getTimeInMillis(Property.TSERV_BULK_TIMEOUT);
-      TabletClientService.Iface client = ThriftUtil.getTServerClient(location, instance.getConfiguration(), timeInMillis);
+      long timeInMillis = ServerConfigurationFactory.getConfiguration(instance).getTimeInMillis(Property.TSERV_BULK_TIMEOUT);
+      TabletClientService.Iface client = ThriftUtil.getTServerClient(location, ServerConfigurationFactory.getConfiguration(instance), timeInMillis);
       try {
         HashMap<KeyExtent,Map<String,org.apache.accumulo.core.data.thrift.MapFileInfo>> files = new HashMap<KeyExtent,Map<String,org.apache.accumulo.core.data.thrift.MapFileInfo>>();
         for (Entry<KeyExtent,List<PathSize>> entry : assignmentsPerTablet.entrySet()) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/efb42468/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogs.java
----------------------------------------------------------------------
diff --git a/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogs.java b/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogs.java
index 51ed477..6534bdf 100644
--- a/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogs.java
+++ b/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogs.java
@@ -30,6 +30,7 @@ import java.util.UUID;
 
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.client.impl.ServerConfigurationFactory;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.gc.thrift.GCStatus;
 import org.apache.accumulo.core.gc.thrift.GcCycleStats;
@@ -133,7 +134,7 @@ public class GarbageCollectWriteAheadLogs {
   }
   
   private int removeFiles(Map<String,Path> nameToFileMap, Map<String,ArrayList<Path>> serverToFileMap, Map<String, Path> sortedWALogs, final GCStatus status) {
-    AccumuloConfiguration conf = instance.getConfiguration();
+    AccumuloConfiguration conf = ServerConfigurationFactory.getConfiguration(instance);
     for (Entry<String,ArrayList<Path>> entry : serverToFileMap.entrySet()) {
       if (entry.getKey().isEmpty()) {
         // old-style log entry, just remove it

http://git-wip-us.apache.org/repos/asf/accumulo/blob/efb42468/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
----------------------------------------------------------------------
diff --git a/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java b/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
index fdd08f4..95a7262 100644
--- a/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
+++ b/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
@@ -41,6 +41,7 @@ import org.apache.accumulo.core.client.IsolatedScanner;
 import org.apache.accumulo.core.client.MutationsRejectedException;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.impl.ServerConfigurationFactory;
 import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Key;
@@ -153,9 +154,9 @@ public class SimpleGarbageCollector implements Iface {
     this.credentials = credentials;
     this.instance = instance;
     
-    gcStartDelay = instance.getConfiguration().getTimeInMillis(Property.GC_CYCLE_START);
-    long gcDelay = instance.getConfiguration().getTimeInMillis(Property.GC_CYCLE_DELAY);
-    numDeleteThreads = instance.getConfiguration().getCount(Property.GC_DELETE_THREADS);
+    gcStartDelay = ServerConfigurationFactory.getConfiguration(instance).getTimeInMillis(Property.GC_CYCLE_START);
+    long gcDelay = ServerConfigurationFactory.getConfiguration(instance).getTimeInMillis(Property.GC_CYCLE_DELAY);
+    numDeleteThreads = ServerConfigurationFactory.getConfiguration(instance).getCount(Property.GC_DELETE_THREADS);
     log.info("start delay: " + gcStartDelay + " milliseconds");
     log.info("time delay: " + gcDelay + " milliseconds");
     log.info("safemode: " + opts.safeMode);
@@ -481,7 +482,7 @@ public class SimpleGarbageCollector implements Iface {
       
       Trace.offNoFlush();
       try {
-        long gcDelay = instance.getConfiguration().getTimeInMillis(Property.GC_CYCLE_DELAY);
+        long gcDelay = ServerConfigurationFactory.getConfiguration(instance).getTimeInMillis(Property.GC_CYCLE_DELAY);
         log.debug("Sleeping for " + gcDelay + " milliseconds");
         Thread.sleep(gcDelay);
       } catch (InterruptedException e) {
@@ -534,8 +535,8 @@ public class SimpleGarbageCollector implements Iface {
   
   private HostAndPort startStatsService() throws UnknownHostException {
     Processor<Iface> processor = new Processor<Iface>(TraceWrap.service(this));
-    int port = instance.getConfiguration().getPort(Property.GC_PORT);
-    long maxMessageSize = instance.getConfiguration().getMemoryInBytes(Property.GENERAL_MAX_MESSAGE_SIZE);
+    int port = ServerConfigurationFactory.getConfiguration(instance).getPort(Property.GC_PORT);
+    long maxMessageSize = ServerConfigurationFactory.getConfiguration(instance).getMemoryInBytes(Property.GENERAL_MAX_MESSAGE_SIZE);
     HostAndPort result = HostAndPort.fromParts(opts.getAddress(), port);
     try {
       port = TServerUtils.startTServer(result, processor, this.getClass().getSimpleName(), "GC Monitor Service", 2, 1000, maxMessageSize).address.getPort();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/efb42468/server/master/src/main/java/org/apache/accumulo/master/Master.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/Master.java b/server/master/src/main/java/org/apache/accumulo/master/Master.java
index efe0002..c6f5ebb 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/Master.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/Master.java
@@ -44,6 +44,7 @@ import org.apache.accumulo.core.client.RowIterator;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.admin.TableOperationsImpl;
+import org.apache.accumulo.core.client.impl.ServerConfigurationFactory;
 import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.client.impl.ThriftTransportPool;
 import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode;
@@ -786,7 +787,7 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
 
     private void updatePlugins(String property) {
       if (property.equals(Property.MASTER_TABLET_BALANCER.getKey())) {
-        TabletBalancer balancer = instance.getConfiguration().instantiateClassProperty(Property.MASTER_TABLET_BALANCER, TabletBalancer.class,
+        TabletBalancer balancer = ServerConfigurationFactory.getConfiguration(instance).instantiateClassProperty(Property.MASTER_TABLET_BALANCER, TabletBalancer.class,
             new DefaultLoadBalancer());
         balancer.init(serverConfig);
         tabletBalancer = balancer;


[3/8] git commit: ACCUMULO-1914 make WALog cleanup after recovery an atomic mutation

Posted by ec...@apache.org.
ACCUMULO-1914 make WALog cleanup after recovery an atomic mutation


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

Branch: refs/heads/1.6.0-SNAPSHOT
Commit: 8bd6e0ebacb8fafeef8627807f610f10b036be6e
Parents: 32b6b65
Author: Eric Newton <er...@gmail.com>
Authored: Thu Nov 21 12:19:01 2013 -0500
Committer: Eric Newton <er...@gmail.com>
Committed: Thu Nov 21 12:19:01 2013 -0500

----------------------------------------------------------------------
 .../org/apache/accumulo/server/util/MetadataTable.java  | 12 +++++++-----
 1 file changed, 7 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/8bd6e0eb/src/server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java
----------------------------------------------------------------------
diff --git a/src/server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java b/src/server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java
index 9c4eebf..a9b72e0 100644
--- a/src/server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java
+++ b/src/server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java
@@ -991,8 +991,8 @@ public class MetadataTable extends org.apache.accumulo.core.util.MetadataTable {
   }
   
   public static void removeUnusedWALEntries(KeyExtent extent, List<LogEntry> logEntries, ZooLock zooLock) {
-    for (LogEntry entry : logEntries) {
-      if (entry.extent.equals(Constants.ROOT_TABLET_EXTENT)) {
+      if (extent.equals(Constants.ROOT_TABLET_EXTENT)) {
+        for (LogEntry entry : logEntries) {
         String root = getZookeeperLogLocation();
         while (true) {
           try {
@@ -1005,11 +1005,13 @@ public class MetadataTable extends org.apache.accumulo.core.util.MetadataTable {
           }
           UtilWaitThread.sleep(1000);
         }
+        }
       } else {
-        Mutation m = new Mutation(entry.extent.getMetadataEntry());
-        m.putDelete(Constants.METADATA_LOG_COLUMN_FAMILY, new Text(entry.server + "/" + entry.filename));
+        Mutation m = new Mutation(extent.getMetadataEntry());
+        for (LogEntry entry : logEntries) {
+          m.putDelete(Constants.METADATA_LOG_COLUMN_FAMILY, new Text(entry.server + "/" + entry.filename));
+        }
         update(SecurityConstants.getSystemCredentials(), zooLock, m);
-      }
     }
   }
   


[7/8] git commit: ACCUMULO-1914 make WALog cleanup after recovery an atomic mutation

Posted by ec...@apache.org.
ACCUMULO-1914 make WALog cleanup after recovery an atomic mutation


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

Branch: refs/heads/1.6.0-SNAPSHOT
Commit: fe46a60c40677df0909e400eda29c79e04f0ead9
Parents: efb4246 041270b
Author: Eric Newton <er...@gmail.com>
Authored: Thu Nov 21 12:45:13 2013 -0500
Committer: Eric Newton <er...@gmail.com>
Committed: Thu Nov 21 12:45:13 2013 -0500

----------------------------------------------------------------------
 .../accumulo/server/util/MetadataTableUtil.java | 99 +++-----------------
 1 file changed, 12 insertions(+), 87 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/fe46a60c/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java
----------------------------------------------------------------------
diff --cc server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java
index 76ea3b1,0000000..a3b8011
mode 100644,000000..100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java
@@@ -1,1034 -1,0 +1,959 @@@
 +/*
 + * 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.io.IOException;
 +import java.util.ArrayList;
 +import java.util.Collections;
 +import java.util.Comparator;
 +import java.util.HashMap;
 +import java.util.HashSet;
 +import java.util.Iterator;
 +import java.util.List;
 +import java.util.Map;
 +import java.util.Map.Entry;
 +import java.util.Set;
 +import java.util.SortedMap;
 +import java.util.TreeMap;
 +import java.util.concurrent.TimeUnit;
 +
 +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.BatchWriter;
 +import org.apache.accumulo.core.client.BatchWriterConfig;
 +import org.apache.accumulo.core.client.Connector;
 +import org.apache.accumulo.core.client.Instance;
 +import org.apache.accumulo.core.client.IsolatedScanner;
 +import org.apache.accumulo.core.client.MutationsRejectedException;
 +import org.apache.accumulo.core.client.Scanner;
 +import org.apache.accumulo.core.client.TableNotFoundException;
 +import org.apache.accumulo.core.client.impl.BatchWriterImpl;
 +import org.apache.accumulo.core.client.impl.ScannerImpl;
 +import org.apache.accumulo.core.client.impl.Writer;
 +import org.apache.accumulo.core.data.Key;
 +import org.apache.accumulo.core.data.KeyExtent;
 +import org.apache.accumulo.core.data.Mutation;
 +import org.apache.accumulo.core.data.PartialKey;
 +import org.apache.accumulo.core.data.Range;
 +import org.apache.accumulo.core.data.Value;
- import org.apache.accumulo.core.master.state.tables.TableState;
 +import org.apache.accumulo.core.metadata.MetadataTable;
 +import org.apache.accumulo.core.metadata.RootTable;
 +import org.apache.accumulo.core.metadata.schema.DataFileValue;
 +import org.apache.accumulo.core.metadata.schema.MetadataSchema;
 +import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
 +import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ChoppedColumnFamily;
 +import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ClonedColumnFamily;
 +import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
 +import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LogColumnFamily;
 +import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ScanFileColumnFamily;
 +import org.apache.accumulo.core.security.Authorizations;
 +import org.apache.accumulo.core.security.Credentials;
 +import org.apache.accumulo.core.tabletserver.log.LogEntry;
 +import org.apache.accumulo.core.tabletserver.thrift.ConstraintViolationException;
 +import org.apache.accumulo.core.util.ColumnFQ;
 +import org.apache.accumulo.core.util.FastFormat;
 +import org.apache.accumulo.core.util.Pair;
 +import org.apache.accumulo.core.util.UtilWaitThread;
 +import org.apache.accumulo.core.zookeeper.ZooUtil;
 +import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 +import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
 +import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy;
 +import org.apache.accumulo.server.ServerConstants;
 +import org.apache.accumulo.server.client.HdfsZooInstance;
 +import org.apache.accumulo.server.fs.FileRef;
 +import org.apache.accumulo.server.fs.VolumeManager;
 +import org.apache.accumulo.server.fs.VolumeManager.FileType;
 +import org.apache.accumulo.server.fs.VolumeManagerImpl;
 +import org.apache.accumulo.server.security.SystemCredentials;
- import org.apache.accumulo.server.tables.TableManager;
 +import org.apache.accumulo.server.zookeeper.ZooLock;
 +import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 +import org.apache.hadoop.fs.FileStatus;
 +import org.apache.hadoop.fs.Path;
 +import org.apache.hadoop.io.Text;
 +import org.apache.log4j.Logger;
 +import org.apache.zookeeper.KeeperException;
 +
 +/**
 + * provides a reference to the metadata table for updates by tablet servers
 + */
 +public class MetadataTableUtil {
 +
 +  private static final Text EMPTY_TEXT = new Text();
 +  private static Map<Credentials,Writer> root_tables = new HashMap<Credentials,Writer>();
 +  private static Map<Credentials,Writer> metadata_tables = new HashMap<Credentials,Writer>();
 +  private static final Logger log = Logger.getLogger(MetadataTableUtil.class);
 +
-   private static final int SAVE_ROOT_TABLET_RETRIES = 3;
- 
 +  private MetadataTableUtil() {}
 +
 +  public synchronized static Writer getMetadataTable(Credentials credentials) {
 +    Writer metadataTable = metadata_tables.get(credentials);
 +    if (metadataTable == null) {
 +      metadataTable = new Writer(HdfsZooInstance.getInstance(), credentials, MetadataTable.ID);
 +      metadata_tables.put(credentials, metadataTable);
 +    }
 +    return metadataTable;
 +  }
 +
-   public synchronized static Writer getRootTable(Credentials credentials) {
++  private synchronized static Writer getRootTable(Credentials credentials) {
 +    Writer rootTable = root_tables.get(credentials);
 +    if (rootTable == null) {
 +      rootTable = new Writer(HdfsZooInstance.getInstance(), credentials, RootTable.ID);
 +      root_tables.put(credentials, rootTable);
 +    }
 +    return rootTable;
 +  }
 +
-   public static void putLockID(ZooLock zooLock, Mutation m) {
++  private static void putLockID(ZooLock zooLock, Mutation m) {
 +    TabletsSection.ServerColumnFamily.LOCK_COLUMN.put(m, new Value(zooLock.getLockID().serialize(ZooUtil.getRoot(HdfsZooInstance.getInstance()) + "/")
 +        .getBytes()));
 +  }
 +
-   public static void update(Credentials credentials, Mutation m, KeyExtent extent) {
++  private static void update(Credentials credentials, Mutation m, KeyExtent extent) {
 +    update(credentials, null, m, extent);
 +  }
 +
 +  public static void update(Credentials credentials, ZooLock zooLock, Mutation m, KeyExtent extent) {
 +    Writer t = extent.isMeta() ? getRootTable(credentials) : getMetadataTable(credentials);
 +    if (zooLock != null)
 +      putLockID(zooLock, m);
 +    while (true) {
 +      try {
 +        t.update(m);
 +        return;
 +      } catch (AccumuloException e) {
 +        log.error(e, e);
 +      } catch (AccumuloSecurityException e) {
 +        log.error(e, e);
 +      } catch (ConstraintViolationException e) {
 +        log.error(e, e);
 +      } catch (TableNotFoundException e) {
 +        log.error(e, e);
 +      }
 +      UtilWaitThread.sleep(1000);
 +    }
 +
 +  }
 +
 +  public static void updateTabletFlushID(KeyExtent extent, long flushID, Credentials credentials, ZooLock zooLock) {
 +    if (!extent.isRootTablet()) {
 +      Mutation m = new Mutation(extent.getMetadataEntry());
 +      TabletsSection.ServerColumnFamily.FLUSH_COLUMN.put(m, new Value((flushID + "").getBytes()));
 +      update(credentials, zooLock, m, extent);
 +    }
 +  }
 +
 +  public static void updateTabletCompactID(KeyExtent extent, long compactID, Credentials credentials, ZooLock zooLock) {
 +    if (!extent.isRootTablet()) {
 +      Mutation m = new Mutation(extent.getMetadataEntry());
 +      TabletsSection.ServerColumnFamily.COMPACT_COLUMN.put(m, new Value((compactID + "").getBytes()));
 +      update(credentials, zooLock, m, extent);
 +    }
 +  }
 +
 +  public static void updateTabletDataFile(long tid, KeyExtent extent, Map<FileRef,DataFileValue> estSizes, String time, Credentials credentials, ZooLock zooLock) {
 +    Mutation m = new Mutation(extent.getMetadataEntry());
 +    byte[] tidBytes = Long.toString(tid).getBytes();
 +
 +    for (Entry<FileRef,DataFileValue> entry : estSizes.entrySet()) {
 +      Text file = entry.getKey().meta();
 +      m.put(DataFileColumnFamily.NAME, file, new Value(entry.getValue().encode()));
 +      m.put(TabletsSection.BulkFileColumnFamily.NAME, file, new Value(tidBytes));
 +    }
 +    TabletsSection.ServerColumnFamily.TIME_COLUMN.put(m, new Value(time.getBytes()));
 +    update(credentials, zooLock, m, extent);
 +  }
 +
 +  public static void addTablet(KeyExtent extent, String path, Credentials credentials, char timeType, ZooLock lock) {
 +    Mutation m = extent.getPrevRowUpdateMutation();
 +
 +    TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN.put(m, new Value(path.getBytes()));
 +    TabletsSection.ServerColumnFamily.TIME_COLUMN.put(m, new Value((timeType + "0").getBytes()));
 +
 +    update(credentials, lock, m, extent);
 +  }
 +
 +  public static void updateTabletPrevEndRow(KeyExtent extent, Credentials credentials) {
 +    Mutation m = extent.getPrevRowUpdateMutation(); //
 +    update(credentials, m, extent);
 +  }
 +
-   /**
-    * convenience method for reading entries from the metadata table
-    */
-   public static SortedMap<KeyExtent,Text> getMetadataDirectoryEntries(SortedMap<Key,Value> entries) {
-     Key key;
-     Value val;
-     Text datafile = null;
-     Value prevRow = null;
-     KeyExtent ke;
- 
-     SortedMap<KeyExtent,Text> results = new TreeMap<KeyExtent,Text>();
- 
-     Text lastRowFromKey = new Text();
- 
-     // text obj below is meant to be reused in loop for efficiency
-     Text colf = new Text();
-     Text colq = new Text();
- 
-     for (Entry<Key,Value> entry : entries.entrySet()) {
-       key = entry.getKey();
-       val = entry.getValue();
- 
-       if (key.compareRow(lastRowFromKey) != 0) {
-         prevRow = null;
-         datafile = null;
-         key.getRow(lastRowFromKey);
-       }
- 
-       colf = key.getColumnFamily(colf);
-       colq = key.getColumnQualifier(colq);
- 
-       // interpret the row id as a key extent
-       if (TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN.equals(colf, colq))
-         datafile = new Text(val.toString());
- 
-       else if (TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN.equals(colf, colq))
-         prevRow = new Value(val);
- 
-       if (datafile != null && prevRow != null) {
-         ke = new KeyExtent(key.getRow(), prevRow);
-         results.put(ke, datafile);
- 
-         datafile = null;
-         prevRow = null;
-       }
-     }
-     return results;
-   }
- 
-   public static boolean recordRootTabletLocation(String address) {
-     IZooReaderWriter zoo = ZooReaderWriter.getInstance();
-     for (int i = 0; i < SAVE_ROOT_TABLET_RETRIES; i++) {
-       try {
-         log.info("trying to write root tablet location to ZooKeeper as " + address);
-         String zRootLocPath = ZooUtil.getRoot(HdfsZooInstance.getInstance()) + RootTable.ZROOT_TABLET_LOCATION;
-         zoo.putPersistentData(zRootLocPath, address.getBytes(), NodeExistsPolicy.OVERWRITE);
-         return true;
-       } catch (Exception e) {
-         log.error("Master: unable to save root tablet location in zookeeper. exception: " + e, e);
-       }
-     }
-     log.error("Giving up after " + SAVE_ROOT_TABLET_RETRIES + " retries");
-     return false;
-   }
- 
 +  public static SortedMap<FileRef,DataFileValue> getDataFileSizes(KeyExtent extent, Credentials credentials) throws IOException {
 +    TreeMap<FileRef,DataFileValue> sizes = new TreeMap<FileRef,DataFileValue>();
 +
 +    Scanner mdScanner = new ScannerImpl(HdfsZooInstance.getInstance(), credentials, MetadataTable.ID, Authorizations.EMPTY);
 +    mdScanner.fetchColumnFamily(DataFileColumnFamily.NAME);
 +    Text row = extent.getMetadataEntry();
 +    VolumeManager fs = VolumeManagerImpl.get();
 +
 +    Key endKey = new Key(row, DataFileColumnFamily.NAME, new Text(""));
 +    endKey = endKey.followingKey(PartialKey.ROW_COLFAM);
 +
 +    mdScanner.setRange(new Range(new Key(row), endKey));
 +    for (Entry<Key,Value> entry : mdScanner) {
 +
 +      if (!entry.getKey().getRow().equals(row))
 +        break;
 +      DataFileValue dfv = new DataFileValue(entry.getValue().get());
 +      sizes.put(new FileRef(fs, entry.getKey()), dfv);
 +    }
 +
 +    return sizes;
 +  }
 +
 +  public static void rollBackSplit(Text metadataEntry, Text oldPrevEndRow, Credentials credentials, ZooLock zooLock) {
 +    KeyExtent ke = new KeyExtent(metadataEntry, oldPrevEndRow);
 +    Mutation m = ke.getPrevRowUpdateMutation();
 +    TabletsSection.TabletColumnFamily.SPLIT_RATIO_COLUMN.putDelete(m);
 +    TabletsSection.TabletColumnFamily.OLD_PREV_ROW_COLUMN.putDelete(m);
 +    update(credentials, zooLock, m, new KeyExtent(metadataEntry, (Text) null));
 +  }
 +
 +  public static void splitTablet(KeyExtent extent, Text oldPrevEndRow, double splitRatio, Credentials credentials, ZooLock zooLock) {
 +    Mutation m = extent.getPrevRowUpdateMutation(); //
 +
 +    TabletsSection.TabletColumnFamily.SPLIT_RATIO_COLUMN.put(m, new Value(Double.toString(splitRatio).getBytes()));
 +
 +    TabletsSection.TabletColumnFamily.OLD_PREV_ROW_COLUMN.put(m, KeyExtent.encodePrevEndRow(oldPrevEndRow));
 +    ChoppedColumnFamily.CHOPPED_COLUMN.putDelete(m);
 +    update(credentials, zooLock, m, extent);
 +  }
 +
 +  public static void finishSplit(Text metadataEntry, Map<FileRef,DataFileValue> datafileSizes, List<FileRef> highDatafilesToRemove, Credentials credentials,
 +      ZooLock zooLock) {
 +    Mutation m = new Mutation(metadataEntry);
 +    TabletsSection.TabletColumnFamily.SPLIT_RATIO_COLUMN.putDelete(m);
 +    TabletsSection.TabletColumnFamily.OLD_PREV_ROW_COLUMN.putDelete(m);
 +    ChoppedColumnFamily.CHOPPED_COLUMN.putDelete(m);
 +
 +    for (Entry<FileRef,DataFileValue> entry : datafileSizes.entrySet()) {
 +      m.put(DataFileColumnFamily.NAME, entry.getKey().meta(), new Value(entry.getValue().encode()));
 +    }
 +
 +    for (FileRef pathToRemove : highDatafilesToRemove) {
 +      m.putDelete(DataFileColumnFamily.NAME, pathToRemove.meta());
 +    }
 +
 +    update(credentials, zooLock, m, new KeyExtent(metadataEntry, (Text) null));
 +  }
 +
 +  public static void finishSplit(KeyExtent extent, Map<FileRef,DataFileValue> datafileSizes, List<FileRef> highDatafilesToRemove, Credentials credentials,
 +      ZooLock zooLock) {
 +    finishSplit(extent.getMetadataEntry(), datafileSizes, highDatafilesToRemove, credentials, zooLock);
 +  }
 +
 +  public static void addDeleteEntries(KeyExtent extent, Set<FileRef> datafilesToDelete, Credentials credentials) throws IOException {
 +
 +    String tableId = extent.getTableId().toString();
 +
 +    // TODO could use batch writer,would need to handle failure and retry like update does - ACCUMULO-1294
 +    for (FileRef pathToRemove : datafilesToDelete) {
 +      update(credentials, createDeleteMutation(tableId, pathToRemove.path().toString()), extent);
 +    }
 +  }
 +
 +  public static void addDeleteEntry(String tableId, String path) throws IOException {
 +    update(SystemCredentials.get(), createDeleteMutation(tableId, path), new KeyExtent(new Text(tableId), null, null));
 +  }
 +
 +  public static Mutation createDeleteMutation(String tableId, String pathToRemove) throws IOException {
 +    if (!pathToRemove.contains(":")) {
 +      if (pathToRemove.startsWith("../"))
 +        pathToRemove = pathToRemove.substring(2);
 +      else
 +        pathToRemove = "/" + tableId + pathToRemove;
 +    }
 +
 +    Path path = VolumeManagerImpl.get().getFullPath(FileType.TABLE, pathToRemove);
 +    Mutation delFlag = new Mutation(new Text(MetadataSchema.DeletesSection.getRowPrefix() + path.toString()));
 +    delFlag.put(EMPTY_TEXT, EMPTY_TEXT, new Value(new byte[] {}));
 +    return delFlag;
 +  }
 +
 +  public static void removeScanFiles(KeyExtent extent, Set<FileRef> scanFiles, Credentials credentials, ZooLock zooLock) {
 +    Mutation m = new Mutation(extent.getMetadataEntry());
 +
 +    for (FileRef pathToRemove : scanFiles)
 +      m.putDelete(ScanFileColumnFamily.NAME, pathToRemove.meta());
 +
 +    update(credentials, zooLock, m, extent);
 +  }
 +
 +  public static void splitDatafiles(Text table, Text midRow, double splitRatio, Map<FileRef,FileUtil.FileInfo> firstAndLastRows,
 +      SortedMap<FileRef,DataFileValue> datafiles, SortedMap<FileRef,DataFileValue> lowDatafileSizes, SortedMap<FileRef,DataFileValue> highDatafileSizes,
 +      List<FileRef> highDatafilesToRemove) {
 +
 +    for (Entry<FileRef,DataFileValue> entry : datafiles.entrySet()) {
 +
 +      Text firstRow = null;
 +      Text lastRow = null;
 +
 +      boolean rowsKnown = false;
 +
 +      FileUtil.FileInfo mfi = firstAndLastRows.get(entry.getKey());
 +
 +      if (mfi != null) {
 +        firstRow = mfi.getFirstRow();
 +        lastRow = mfi.getLastRow();
 +        rowsKnown = true;
 +      }
 +
 +      if (rowsKnown && firstRow.compareTo(midRow) > 0) {
 +        // only in high
 +        long highSize = entry.getValue().getSize();
 +        long highEntries = entry.getValue().getNumEntries();
 +        highDatafileSizes.put(entry.getKey(), new DataFileValue(highSize, highEntries, entry.getValue().getTime()));
 +      } else if (rowsKnown && lastRow.compareTo(midRow) <= 0) {
 +        // only in low
 +        long lowSize = entry.getValue().getSize();
 +        long lowEntries = entry.getValue().getNumEntries();
 +        lowDatafileSizes.put(entry.getKey(), new DataFileValue(lowSize, lowEntries, entry.getValue().getTime()));
 +
 +        highDatafilesToRemove.add(entry.getKey());
 +      } else {
 +        long lowSize = (long) Math.floor((entry.getValue().getSize() * splitRatio));
 +        long lowEntries = (long) Math.floor((entry.getValue().getNumEntries() * splitRatio));
 +        lowDatafileSizes.put(entry.getKey(), new DataFileValue(lowSize, lowEntries, entry.getValue().getTime()));
 +
 +        long highSize = (long) Math.ceil((entry.getValue().getSize() * (1.0 - splitRatio)));
 +        long highEntries = (long) Math.ceil((entry.getValue().getNumEntries() * (1.0 - splitRatio)));
 +        highDatafileSizes.put(entry.getKey(), new DataFileValue(highSize, highEntries, entry.getValue().getTime()));
 +      }
 +    }
 +  }
 +
 +  public static void deleteTable(String tableId, boolean insertDeletes, Credentials credentials, ZooLock lock) throws AccumuloException, IOException {
 +    Scanner ms = new ScannerImpl(HdfsZooInstance.getInstance(), credentials, MetadataTable.ID, Authorizations.EMPTY);
 +    Text tableIdText = new Text(tableId);
 +    BatchWriter bw = new BatchWriterImpl(HdfsZooInstance.getInstance(), credentials, MetadataTable.ID, new BatchWriterConfig().setMaxMemory(1000000)
 +        .setMaxLatency(120000l, TimeUnit.MILLISECONDS).setMaxWriteThreads(2));
 +
 +    // scan metadata for our table and delete everything we find
 +    Mutation m = null;
 +    ms.setRange(new KeyExtent(tableIdText, null, null).toMetadataRange());
 +
 +    // insert deletes before deleting data from !METADATA... this makes the code fault tolerant
 +    if (insertDeletes) {
 +
 +      ms.fetchColumnFamily(DataFileColumnFamily.NAME);
 +      TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN.fetch(ms);
 +
 +      for (Entry<Key,Value> cell : ms) {
 +        Key key = cell.getKey();
 +
 +        if (key.getColumnFamily().equals(DataFileColumnFamily.NAME)) {
 +          FileRef ref = new FileRef(VolumeManagerImpl.get(), key);
 +          bw.addMutation(createDeleteMutation(tableId, ref.meta().toString()));
 +        }
 +
 +        if (TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN.hasColumns(key)) {
 +          bw.addMutation(createDeleteMutation(tableId, cell.getValue().toString()));
 +        }
 +      }
 +
 +      bw.flush();
 +
 +      ms.clearColumns();
 +    }
 +
 +    for (Entry<Key,Value> cell : ms) {
 +      Key key = cell.getKey();
 +
 +      if (m == null) {
 +        m = new Mutation(key.getRow());
 +        if (lock != null)
 +          putLockID(lock, m);
 +      }
 +
 +      if (key.getRow().compareTo(m.getRow(), 0, m.getRow().length) != 0) {
 +        bw.addMutation(m);
 +        m = new Mutation(key.getRow());
 +        if (lock != null)
 +          putLockID(lock, m);
 +      }
 +      m.putDelete(key.getColumnFamily(), key.getColumnQualifier());
 +    }
 +
 +    if (m != null)
 +      bw.addMutation(m);
 +
 +    bw.close();
 +  }
 +
 +  static String getZookeeperLogLocation() {
 +    return ZooUtil.getRoot(HdfsZooInstance.getInstance()) + RootTable.ZROOT_TABLET_WALOGS;
 +  }
 +
 +  public static void addLogEntry(Credentials credentials, LogEntry entry, ZooLock zooLock) {
 +    if (entry.extent.isRootTablet()) {
 +      String root = getZookeeperLogLocation();
 +      while (true) {
 +        try {
 +          IZooReaderWriter zoo = ZooReaderWriter.getInstance();
 +          if (zoo.isLockHeld(zooLock.getLockID())) {
 +            String[] parts = entry.filename.split("/");
 +            String uniqueId = parts[parts.length - 1];
 +            zoo.putPersistentData(root + "/" + uniqueId, entry.toBytes(), NodeExistsPolicy.OVERWRITE);
 +          }
 +          break;
 +        } catch (KeeperException e) {
 +          log.error(e, e);
 +        } catch (InterruptedException e) {
 +          log.error(e, e);
 +        } catch (IOException e) {
 +          log.error(e, e);
 +        }
 +        UtilWaitThread.sleep(1000);
 +      }
 +    } else {
 +      Mutation m = new Mutation(entry.getRow());
 +      m.put(entry.getColumnFamily(), entry.getColumnQualifier(), entry.getValue());
 +      update(credentials, zooLock, m, entry.extent);
 +    }
 +  }
 +
 +  public static String getRootTabletDir() throws IOException {
 +    IZooReaderWriter zoo = ZooReaderWriter.getInstance();
 +    String zpath = ZooUtil.getRoot(HdfsZooInstance.getInstance()) + RootTable.ZROOT_TABLET_PATH;
 +    try {
 +      return new String(zoo.getData(zpath, null), Constants.UTF8);
 +    } catch (KeeperException e) {
 +      throw new IOException(e);
 +    } catch (InterruptedException e) {
 +      throw new IOException(e);
 +    }
 +  }
 +
 +  public static Pair<List<LogEntry>,SortedMap<FileRef,DataFileValue>> getFileAndLogEntries(Credentials credentials, KeyExtent extent) throws KeeperException,
 +      InterruptedException, IOException {
 +    ArrayList<LogEntry> result = new ArrayList<LogEntry>();
 +    TreeMap<FileRef,DataFileValue> sizes = new TreeMap<FileRef,DataFileValue>();
 +
 +    VolumeManager fs = VolumeManagerImpl.get();
 +    if (extent.isRootTablet()) {
 +      getRootLogEntries(result);
 +      Path rootDir = new Path(getRootTabletDir());
 +      FileStatus[] files = fs.listStatus(rootDir);
 +      for (FileStatus fileStatus : files) {
 +        if (fileStatus.getPath().toString().endsWith("_tmp")) {
 +          continue;
 +        }
 +        DataFileValue dfv = new DataFileValue(0, 0);
 +        sizes.put(new FileRef(fileStatus.getPath().toString(), fileStatus.getPath()), dfv);
 +      }
 +
 +    } else {
 +      String systemTableToCheck = extent.isMeta() ? RootTable.ID : MetadataTable.ID;
 +      Scanner scanner = new ScannerImpl(HdfsZooInstance.getInstance(), credentials, systemTableToCheck, Authorizations.EMPTY);
 +      scanner.fetchColumnFamily(LogColumnFamily.NAME);
 +      scanner.fetchColumnFamily(DataFileColumnFamily.NAME);
 +      scanner.setRange(extent.toMetadataRange());
 +
 +      for (Entry<Key,Value> entry : scanner) {
 +        if (!entry.getKey().getRow().equals(extent.getMetadataEntry())) {
 +          throw new RuntimeException("Unexpected row " + entry.getKey().getRow() + " expected " + extent.getMetadataEntry());
 +        }
 +
 +        if (entry.getKey().getColumnFamily().equals(LogColumnFamily.NAME)) {
 +          result.add(LogEntry.fromKeyValue(entry.getKey(), entry.getValue()));
 +        } else if (entry.getKey().getColumnFamily().equals(DataFileColumnFamily.NAME)) {
 +          DataFileValue dfv = new DataFileValue(entry.getValue().get());
 +          sizes.put(new FileRef(fs, entry.getKey()), dfv);
 +        } else {
 +          throw new RuntimeException("Unexpected col fam " + entry.getKey().getColumnFamily());
 +        }
 +      }
 +    }
 +
 +    return new Pair<List<LogEntry>,SortedMap<FileRef,DataFileValue>>(result, sizes);
 +  }
 +
 +  public static List<LogEntry> getLogEntries(Credentials credentials, KeyExtent extent) throws IOException, KeeperException, InterruptedException {
 +    log.info("Scanning logging entries for " + extent);
 +    ArrayList<LogEntry> result = new ArrayList<LogEntry>();
 +    if (extent.equals(RootTable.EXTENT)) {
 +      log.info("Getting logs for root tablet from zookeeper");
 +      getRootLogEntries(result);
 +    } else {
 +      log.info("Scanning metadata for logs used for tablet " + extent);
 +      Scanner scanner = getTabletLogScanner(credentials, extent);
 +      Text pattern = extent.getMetadataEntry();
 +      for (Entry<Key,Value> entry : scanner) {
 +        Text row = entry.getKey().getRow();
 +        if (entry.getKey().getColumnFamily().equals(LogColumnFamily.NAME)) {
 +          if (row.equals(pattern)) {
 +            result.add(LogEntry.fromKeyValue(entry.getKey(), entry.getValue()));
 +          }
 +        }
 +      }
 +    }
 +
 +    Collections.sort(result, new Comparator<LogEntry>() {
 +      @Override
 +      public int compare(LogEntry o1, LogEntry o2) {
 +        long diff = o1.timestamp - o2.timestamp;
 +        if (diff < 0)
 +          return -1;
 +        if (diff > 0)
 +          return 1;
 +        return 0;
 +      }
 +    });
 +    log.info("Returning logs " + result + " for extent " + extent);
 +    return result;
 +  }
 +
 +  private static void getRootLogEntries(ArrayList<LogEntry> result) throws KeeperException, InterruptedException, IOException {
 +    IZooReaderWriter zoo = ZooReaderWriter.getInstance();
 +    String root = getZookeeperLogLocation();
 +    // there's a little race between getting the children and fetching
 +    // the data. The log can be removed in between.
 +    while (true) {
 +      result.clear();
 +      for (String child : zoo.getChildren(root)) {
 +        LogEntry e = new LogEntry();
 +        try {
 +          e.fromBytes(zoo.getData(root + "/" + child, null));
 +          // upgrade from !0;!0<< -> !!R<<
 +          e.extent = RootTable.EXTENT;
 +          result.add(e);
 +        } catch (KeeperException.NoNodeException ex) {
 +          continue;
 +        }
 +      }
 +      break;
 +    }
 +  }
 +
 +  private static Scanner getTabletLogScanner(Credentials credentials, KeyExtent extent) {
 +    String tableId = MetadataTable.ID;
 +    if (extent.isMeta())
 +      tableId = RootTable.ID;
 +    Scanner scanner = new ScannerImpl(HdfsZooInstance.getInstance(), credentials, tableId, Authorizations.EMPTY);
 +    scanner.fetchColumnFamily(LogColumnFamily.NAME);
 +    Text start = extent.getMetadataEntry();
 +    Key endKey = new Key(start, LogColumnFamily.NAME);
 +    endKey = endKey.followingKey(PartialKey.ROW_COLFAM);
 +    scanner.setRange(new Range(new Key(start), endKey));
 +    return scanner;
 +  }
 +
-   static class LogEntryIterator implements Iterator<LogEntry> {
++  private static class LogEntryIterator implements Iterator<LogEntry> {
 +
 +    Iterator<LogEntry> zookeeperEntries = null;
 +    Iterator<LogEntry> rootTableEntries = null;
 +    Iterator<Entry<Key,Value>> metadataEntries = null;
 +
 +    LogEntryIterator(Credentials creds) throws IOException, KeeperException, InterruptedException {
 +      zookeeperEntries = getLogEntries(creds, RootTable.EXTENT).iterator();
 +      rootTableEntries = getLogEntries(creds, new KeyExtent(new Text(MetadataTable.ID), null, null)).iterator();
 +      try {
 +        Scanner scanner = HdfsZooInstance.getInstance().getConnector(creds.getPrincipal(), creds.getToken())
 +            .createScanner(MetadataTable.NAME, Authorizations.EMPTY);
 +        log.info("Setting range to " + MetadataSchema.TabletsSection.getRange());
 +        scanner.setRange(MetadataSchema.TabletsSection.getRange());
 +        scanner.fetchColumnFamily(LogColumnFamily.NAME);
 +        metadataEntries = scanner.iterator();
 +      } catch (Exception ex) {
 +        throw new IOException(ex);
 +      }
 +    }
 +
 +    @Override
 +    public boolean hasNext() {
 +      return zookeeperEntries.hasNext() || rootTableEntries.hasNext() || metadataEntries.hasNext();
 +    }
 +
 +    @Override
 +    public LogEntry next() {
 +      if (zookeeperEntries.hasNext()) {
 +        return zookeeperEntries.next();
 +      }
 +      if (rootTableEntries.hasNext()) {
 +        return rootTableEntries.next();
 +      }
 +      Entry<Key,Value> entry = metadataEntries.next();
 +      return LogEntry.fromKeyValue(entry.getKey(), entry.getValue());
 +    }
 +
 +    @Override
 +    public void remove() {
 +      throw new UnsupportedOperationException();
 +    }
 +  }
 +
 +  public static Iterator<LogEntry> getLogEntries(Credentials creds) throws IOException, KeeperException, InterruptedException {
 +    return new LogEntryIterator(creds);
 +  }
 +
 +  public static void removeUnusedWALEntries(KeyExtent extent, List<LogEntry> logEntries, ZooLock zooLock) {
-     for (LogEntry entry : logEntries) {
-       if (entry.extent.isRootTablet()) {
++    if (extent.isRootTablet()) {
++      for (LogEntry entry : logEntries) {
 +        String root = getZookeeperLogLocation();
 +        while (true) {
 +          try {
 +            IZooReaderWriter zoo = ZooReaderWriter.getInstance();
 +            if (zoo.isLockHeld(zooLock.getLockID()))
 +              zoo.recursiveDelete(root + "/" + entry.filename, NodeMissingPolicy.SKIP);
 +            break;
 +          } catch (Exception e) {
 +            log.error(e, e);
 +          }
 +          UtilWaitThread.sleep(1000);
 +        }
-       } else {
-         Mutation m = new Mutation(entry.extent.getMetadataEntry());
++      }
++    } else {
++      Mutation m = new Mutation(extent.getMetadataEntry());
++      for (LogEntry entry : logEntries) {
 +        m.putDelete(LogColumnFamily.NAME, new Text(entry.toString()));
-         update(SystemCredentials.get(), zooLock, m, entry.extent);
 +      }
++      update(SystemCredentials.get(), zooLock, m, extent);
 +    }
 +  }
 +
 +  private static void getFiles(Set<String> files, Map<Key,Value> tablet, String srcTableId) {
 +    for (Entry<Key,Value> entry : tablet.entrySet()) {
 +      if (entry.getKey().getColumnFamily().equals(DataFileColumnFamily.NAME)) {
 +        String cf = entry.getKey().getColumnQualifier().toString();
 +        if (srcTableId != null && !cf.startsWith("../") && !cf.contains(":")) {
 +          cf = "../" + srcTableId + entry.getKey().getColumnQualifier();
 +        }
 +        files.add(cf);
 +      }
 +    }
 +  }
 +
 +  private static Mutation createCloneMutation(String srcTableId, String tableId, Map<Key,Value> tablet) {
 +
 +    KeyExtent ke = new KeyExtent(tablet.keySet().iterator().next().getRow(), (Text) null);
 +    Mutation m = new Mutation(KeyExtent.getMetadataEntry(new Text(tableId), ke.getEndRow()));
 +
 +    for (Entry<Key,Value> entry : tablet.entrySet()) {
 +      if (entry.getKey().getColumnFamily().equals(DataFileColumnFamily.NAME)) {
 +        String cf = entry.getKey().getColumnQualifier().toString();
 +        if (!cf.startsWith("../") && !cf.contains(":"))
 +          cf = "../" + srcTableId + entry.getKey().getColumnQualifier();
 +        m.put(entry.getKey().getColumnFamily(), new Text(cf), entry.getValue());
 +      } else if (entry.getKey().getColumnFamily().equals(TabletsSection.CurrentLocationColumnFamily.NAME)) {
 +        m.put(TabletsSection.LastLocationColumnFamily.NAME, entry.getKey().getColumnQualifier(), entry.getValue());
 +      } else if (entry.getKey().getColumnFamily().equals(TabletsSection.LastLocationColumnFamily.NAME)) {
 +        // skip
 +      } else {
 +        m.put(entry.getKey().getColumnFamily(), entry.getKey().getColumnQualifier(), entry.getValue());
 +      }
 +    }
 +    return m;
 +  }
 +
 +  private static Scanner createCloneScanner(String tableId, Connector conn) throws TableNotFoundException {
 +    Scanner mscanner = new IsolatedScanner(conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY));
 +    mscanner.setRange(new KeyExtent(new Text(tableId), null, null).toMetadataRange());
 +    mscanner.fetchColumnFamily(DataFileColumnFamily.NAME);
 +    mscanner.fetchColumnFamily(TabletsSection.CurrentLocationColumnFamily.NAME);
 +    mscanner.fetchColumnFamily(TabletsSection.LastLocationColumnFamily.NAME);
 +    mscanner.fetchColumnFamily(ClonedColumnFamily.NAME);
 +    TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN.fetch(mscanner);
 +    TabletsSection.ServerColumnFamily.TIME_COLUMN.fetch(mscanner);
 +    return mscanner;
 +  }
 +
 +  static void initializeClone(String srcTableId, String tableId, Connector conn, BatchWriter bw) throws TableNotFoundException, MutationsRejectedException {
 +    TabletIterator ti = new TabletIterator(createCloneScanner(srcTableId, conn), new KeyExtent(new Text(srcTableId), null, null).toMetadataRange(), true, true);
 +
 +    if (!ti.hasNext())
 +      throw new RuntimeException(" table deleted during clone?  srcTableId = " + srcTableId);
 +
 +    while (ti.hasNext())
 +      bw.addMutation(createCloneMutation(srcTableId, tableId, ti.next()));
 +
 +    bw.flush();
 +  }
 +
-   static int compareEndRows(Text endRow1, Text endRow2) {
++  private static int compareEndRows(Text endRow1, Text endRow2) {
 +    return new KeyExtent(new Text("0"), endRow1, null).compareTo(new KeyExtent(new Text("0"), endRow2, null));
 +  }
 +
 +  static int checkClone(String srcTableId, String tableId, Connector conn, BatchWriter bw) throws TableNotFoundException, MutationsRejectedException {
 +    TabletIterator srcIter = new TabletIterator(createCloneScanner(srcTableId, conn), new KeyExtent(new Text(srcTableId), null, null).toMetadataRange(), true,
 +        true);
 +    TabletIterator cloneIter = new TabletIterator(createCloneScanner(tableId, conn), new KeyExtent(new Text(tableId), null, null).toMetadataRange(), true, true);
 +
 +    if (!cloneIter.hasNext() || !srcIter.hasNext())
 +      throw new RuntimeException(" table deleted during clone?  srcTableId = " + srcTableId + " tableId=" + tableId);
 +
 +    int rewrites = 0;
 +
 +    while (cloneIter.hasNext()) {
 +      Map<Key,Value> cloneTablet = cloneIter.next();
 +      Text cloneEndRow = new KeyExtent(cloneTablet.keySet().iterator().next().getRow(), (Text) null).getEndRow();
 +      HashSet<String> cloneFiles = new HashSet<String>();
 +
 +      boolean cloneSuccessful = false;
 +      for (Entry<Key,Value> entry : cloneTablet.entrySet()) {
 +        if (entry.getKey().getColumnFamily().equals(ClonedColumnFamily.NAME)) {
 +          cloneSuccessful = true;
 +          break;
 +        }
 +      }
 +
 +      if (!cloneSuccessful)
 +        getFiles(cloneFiles, cloneTablet, null);
 +
 +      List<Map<Key,Value>> srcTablets = new ArrayList<Map<Key,Value>>();
 +      Map<Key,Value> srcTablet = srcIter.next();
 +      srcTablets.add(srcTablet);
 +
 +      Text srcEndRow = new KeyExtent(srcTablet.keySet().iterator().next().getRow(), (Text) null).getEndRow();
 +
 +      int cmp = compareEndRows(cloneEndRow, srcEndRow);
 +      if (cmp < 0)
 +        throw new TabletIterator.TabletDeletedException("Tablets deleted from src during clone : " + cloneEndRow + " " + srcEndRow);
 +
 +      HashSet<String> srcFiles = new HashSet<String>();
 +      if (!cloneSuccessful)
 +        getFiles(srcFiles, srcTablet, srcTableId);
 +
 +      while (cmp > 0) {
 +        srcTablet = srcIter.next();
 +        srcTablets.add(srcTablet);
 +        srcEndRow = new KeyExtent(srcTablet.keySet().iterator().next().getRow(), (Text) null).getEndRow();
 +        cmp = compareEndRows(cloneEndRow, srcEndRow);
 +        if (cmp < 0)
 +          throw new TabletIterator.TabletDeletedException("Tablets deleted from src during clone : " + cloneEndRow + " " + srcEndRow);
 +
 +        if (!cloneSuccessful)
 +          getFiles(srcFiles, srcTablet, srcTableId);
 +      }
 +
 +      if (cloneSuccessful)
 +        continue;
 +
 +      if (!srcFiles.containsAll(cloneFiles)) {
 +        // delete existing cloned tablet entry
 +        Mutation m = new Mutation(cloneTablet.keySet().iterator().next().getRow());
 +
 +        for (Entry<Key,Value> entry : cloneTablet.entrySet()) {
 +          Key k = entry.getKey();
 +          m.putDelete(k.getColumnFamily(), k.getColumnQualifier(), k.getTimestamp());
 +        }
 +
 +        bw.addMutation(m);
 +
 +        for (Map<Key,Value> st : srcTablets)
 +          bw.addMutation(createCloneMutation(srcTableId, tableId, st));
 +
 +        rewrites++;
 +      } else {
 +        // write out marker that this tablet was successfully cloned
 +        Mutation m = new Mutation(cloneTablet.keySet().iterator().next().getRow());
 +        m.put(ClonedColumnFamily.NAME, new Text(""), new Value("OK".getBytes()));
 +        bw.addMutation(m);
 +      }
 +    }
 +
 +    bw.flush();
 +    return rewrites;
 +  }
 +
 +  public static void cloneTable(Instance instance, String srcTableId, String tableId, VolumeManager volumeManager) throws Exception {
 +
 +    Connector conn = instance.getConnector(SystemCredentials.get().getPrincipal(), SystemCredentials.get().getToken());
 +    BatchWriter bw = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
 +
 +    while (true) {
 +
 +      try {
 +        initializeClone(srcTableId, tableId, conn, bw);
 +
 +        // the following loop looks changes in the file that occurred during the copy.. if files were dereferenced then they could have been GCed
 +
 +        while (true) {
 +          int rewrites = checkClone(srcTableId, tableId, conn, bw);
 +
 +          if (rewrites == 0)
 +            break;
 +        }
 +
 +        bw.flush();
 +        break;
 +
 +      } catch (TabletIterator.TabletDeletedException tde) {
 +        // tablets were merged in the src table
 +        bw.flush();
 +
 +        // delete what we have cloned and try again
 +        deleteTable(tableId, false, SystemCredentials.get(), null);
 +
 +        log.debug("Tablets merged in table " + srcTableId + " while attempting to clone, trying again");
 +
 +        UtilWaitThread.sleep(100);
 +      }
 +    }
 +
 +    // delete the clone markers and create directory entries
 +    Scanner mscanner = conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
 +    mscanner.setRange(new KeyExtent(new Text(tableId), null, null).toMetadataRange());
 +    mscanner.fetchColumnFamily(ClonedColumnFamily.NAME);
 +
 +    int dirCount = 0;
 +
 +    for (Entry<Key,Value> entry : mscanner) {
 +      Key k = entry.getKey();
 +      Mutation m = new Mutation(k.getRow());
 +      m.putDelete(k.getColumnFamily(), k.getColumnQualifier());
 +      String dir = volumeManager.choose(ServerConstants.getTablesDirs()) + "/" + tableId
 +          + new String(FastFormat.toZeroPaddedString(dirCount++, 8, 16, "/c-".getBytes()));
 +      TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN.put(m, new Value(dir.getBytes()));
 +      bw.addMutation(m);
 +    }
 +
 +    bw.close();
 +
 +  }
 +
 +  public static void chopped(KeyExtent extent, ZooLock zooLock) {
 +    Mutation m = new Mutation(extent.getMetadataEntry());
 +    ChoppedColumnFamily.CHOPPED_COLUMN.put(m, new Value("chopped".getBytes()));
 +    update(SystemCredentials.get(), zooLock, m, extent);
 +  }
 +
 +  public static void removeBulkLoadEntries(Connector conn, String tableId, long tid) throws Exception {
 +    Scanner mscanner = new IsolatedScanner(conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY));
 +    mscanner.setRange(new KeyExtent(new Text(tableId), null, null).toMetadataRange());
 +    mscanner.fetchColumnFamily(TabletsSection.BulkFileColumnFamily.NAME);
 +    BatchWriter bw = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
 +    for (Entry<Key,Value> entry : mscanner) {
 +      log.debug("Looking at entry " + entry + " with tid " + tid);
 +      if (Long.parseLong(entry.getValue().toString()) == tid) {
 +        log.debug("deleting entry " + entry);
 +        Mutation m = new Mutation(entry.getKey().getRow());
 +        m.putDelete(entry.getKey().getColumnFamily(), entry.getKey().getColumnQualifier());
 +        bw.addMutation(m);
 +      }
 +    }
 +    bw.close();
 +  }
 +
 +  public static List<FileRef> getBulkFilesLoaded(Connector conn, KeyExtent extent, long tid) throws IOException {
 +    List<FileRef> result = new ArrayList<FileRef>();
 +    try {
 +      VolumeManager fs = VolumeManagerImpl.get();
 +      Scanner mscanner = new IsolatedScanner(conn.createScanner(extent.isMeta() ? RootTable.NAME : MetadataTable.NAME, Authorizations.EMPTY));
 +      mscanner.setRange(extent.toMetadataRange());
 +      mscanner.fetchColumnFamily(TabletsSection.BulkFileColumnFamily.NAME);
 +      for (Entry<Key,Value> entry : mscanner) {
 +        if (Long.parseLong(entry.getValue().toString()) == tid) {
 +          result.add(new FileRef(fs, entry.getKey()));
 +        }
 +      }
 +      return result;
 +    } catch (TableNotFoundException ex) {
 +      // unlikely
 +      throw new RuntimeException("Onos! teh metadata table has vanished!!");
 +    }
 +  }
 +
 +  public static Map<FileRef,Long> getBulkFilesLoaded(Credentials credentials, KeyExtent extent) throws IOException {
 +    Text metadataRow = extent.getMetadataEntry();
 +    Map<FileRef,Long> ret = new HashMap<FileRef,Long>();
 +
 +    VolumeManager fs = VolumeManagerImpl.get();
 +    Scanner scanner = new ScannerImpl(HdfsZooInstance.getInstance(), credentials, extent.isMeta() ? RootTable.ID : MetadataTable.ID, Authorizations.EMPTY);
 +    scanner.setRange(new Range(metadataRow));
 +    scanner.fetchColumnFamily(TabletsSection.BulkFileColumnFamily.NAME);
 +    for (Entry<Key,Value> entry : scanner) {
 +      Long tid = Long.parseLong(entry.getValue().toString());
 +      ret.put(new FileRef(fs, entry.getKey()), tid);
 +    }
 +    return ret;
 +  }
 +
 +  public static void addBulkLoadInProgressFlag(String path) {
 +
 +    Mutation m = new Mutation(MetadataSchema.BlipSection.getRowPrefix() + path);
 +    m.put(EMPTY_TEXT, EMPTY_TEXT, new Value(new byte[] {}));
 +
 +    // new KeyExtent is only added to force update to write to the metadata table, not the root table
 +    // because bulk loads aren't supported to the metadata table
 +    update(SystemCredentials.get(), m, new KeyExtent(new Text("anythingNotMetadata"), null, null));
 +  }
 +
 +  public static void removeBulkLoadInProgressFlag(String path) {
 +
 +    Mutation m = new Mutation(MetadataSchema.BlipSection.getRowPrefix() + path);
 +    m.putDelete(EMPTY_TEXT, EMPTY_TEXT);
 +
 +    // new KeyExtent is only added to force update to write to the metadata table, not the root table
 +    // because bulk loads aren't supported to the metadata table
 +    update(SystemCredentials.get(), m, new KeyExtent(new Text("anythingNotMetadata"), null, null));
 +  }
 +
 +  public static void moveMetaDeleteMarkers(Instance instance, Credentials creds) {
 +    // move old delete markers to new location, to standardize table schema between all metadata tables
 +    byte[] EMPTY_BYTES = new byte[0];
 +    Scanner scanner = new ScannerImpl(instance, creds, RootTable.ID, Authorizations.EMPTY);
 +    String oldDeletesPrefix = "!!~del";
 +    Range oldDeletesRange = new Range(oldDeletesPrefix, true, "!!~dem", false);
 +    scanner.setRange(oldDeletesRange);
 +    for (Entry<Key,Value> entry : scanner) {
 +      String row = entry.getKey().getRow().toString();
 +      if (row.startsWith(oldDeletesPrefix)) {
 +        String filename = row.substring(oldDeletesPrefix.length());
 +        // add the new entry first
 +        log.info("Moving " + filename + " marker in " + RootTable.NAME);
 +        Mutation m = new Mutation(MetadataSchema.DeletesSection.getRowPrefix() + filename);
 +        m.put(EMPTY_BYTES, EMPTY_BYTES, EMPTY_BYTES);
 +        update(creds, m, RootTable.EXTENT);
 +        // remove the old entry
 +        m = new Mutation(entry.getKey().getRow());
 +        m.putDelete(EMPTY_BYTES, EMPTY_BYTES);
 +        update(creds, m, RootTable.OLD_EXTENT);
 +      } else {
 +        break;
 +      }
 +    }
 +
 +  }
 +
 +  public static SortedMap<Text,SortedMap<ColumnFQ,Value>> getTabletEntries(SortedMap<Key,Value> tabletKeyValues, List<ColumnFQ> columns) {
 +    TreeMap<Text,SortedMap<ColumnFQ,Value>> tabletEntries = new TreeMap<Text,SortedMap<ColumnFQ,Value>>();
 +
 +    HashSet<ColumnFQ> colSet = null;
 +    if (columns != null) {
 +      colSet = new HashSet<ColumnFQ>(columns);
 +    }
 +
 +    for (Entry<Key,Value> entry : tabletKeyValues.entrySet()) {
 +
 +      if (columns != null && !colSet.contains(new ColumnFQ(entry.getKey()))) {
 +        continue;
 +      }
 +
 +      Text row = entry.getKey().getRow();
 +
 +      SortedMap<ColumnFQ,Value> colVals = tabletEntries.get(row);
 +      if (colVals == null) {
 +        colVals = new TreeMap<ColumnFQ,Value>();
 +        tabletEntries.put(row, colVals);
 +      }
 +
 +      colVals.put(new ColumnFQ(entry.getKey()), entry.getValue());
 +    }
 +
 +    return tabletEntries;
 +  }
- 
-   public static void convertRootTabletToRootTable(Instance instance, SystemCredentials systemCredentials) throws KeeperException, InterruptedException {
-     ZooReaderWriter zoo = ZooReaderWriter.getInstance();
-     if (zoo.exists(ZooUtil.getRoot(instance) + "/tables/" + RootTable.ID))
-       return;
-     TableManager.prepareNewTableState(instance.getInstanceID(), RootTable.ID, RootTable.NAME, TableState.ONLINE, NodeExistsPolicy.FAIL);
-   }
- 
 +}


[4/8] git commit: ACCUMULO-1914 make WALog cleanup after recovery an atomic mutation

Posted by ec...@apache.org.
ACCUMULO-1914 make WALog cleanup after recovery an atomic mutation


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

Branch: refs/heads/1.6.0-SNAPSHOT
Commit: 041270b778db9345c5276ef91fb4f95f3e9e0bbe
Parents: 404e955 8bd6e0e
Author: Eric Newton <er...@gmail.com>
Authored: Thu Nov 21 12:34:59 2013 -0500
Committer: Eric Newton <er...@gmail.com>
Committed: Thu Nov 21 12:34:59 2013 -0500

----------------------------------------------------------------------
 .../org/apache/accumulo/server/monitor/Monitor.java | 16 ++++------------
 .../apache/accumulo/server/util/MetadataTable.java  | 12 +++++++-----
 2 files changed, 11 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/041270b7/server/src/main/java/org/apache/accumulo/server/monitor/Monitor.java
----------------------------------------------------------------------
diff --cc server/src/main/java/org/apache/accumulo/server/monitor/Monitor.java
index 8602c4c,0000000..8236b5f
mode 100644,000000..100644
--- a/server/src/main/java/org/apache/accumulo/server/monitor/Monitor.java
+++ b/server/src/main/java/org/apache/accumulo/server/monitor/Monitor.java
@@@ -1,667 -1,0 +1,659 @@@
 +/*
 + * 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.monitor;
 +
 +import java.net.InetSocketAddress;
 +import java.util.ArrayList;
 +import java.util.Collections;
 +import java.util.HashMap;
 +import java.util.HashSet;
 +import java.util.LinkedList;
 +import java.util.List;
 +import java.util.Map;
 +import java.util.Map.Entry;
 +import java.util.Set;
 +
 +import org.apache.accumulo.trace.instrument.Tracer;
 +import org.apache.accumulo.core.Constants;
 +import org.apache.accumulo.core.client.Instance;
 +import org.apache.accumulo.core.client.impl.MasterClient;
 +import org.apache.accumulo.core.conf.AccumuloConfiguration;
 +import org.apache.accumulo.core.conf.Property;
 +import org.apache.accumulo.core.file.FileUtil;
 +import org.apache.accumulo.core.gc.thrift.GCMonitorService;
 +import org.apache.accumulo.core.gc.thrift.GCStatus;
 +import org.apache.accumulo.core.master.thrift.Compacting;
 +import org.apache.accumulo.core.master.thrift.MasterClientService;
 +import org.apache.accumulo.core.master.thrift.MasterMonitorInfo;
 +import org.apache.accumulo.core.master.thrift.TableInfo;
 +import org.apache.accumulo.core.master.thrift.TabletServerStatus;
 +import org.apache.accumulo.core.security.SecurityUtil;
 +import org.apache.accumulo.core.util.CachedConfiguration;
 +import org.apache.accumulo.core.util.Daemon;
 +import org.apache.accumulo.core.util.LoggingRunnable;
 +import org.apache.accumulo.core.util.Pair;
 +import org.apache.accumulo.core.util.ServerServices;
 +import org.apache.accumulo.core.util.ServerServices.Service;
 +import org.apache.accumulo.core.util.ThriftUtil;
 +import org.apache.accumulo.core.util.UtilWaitThread;
 +import org.apache.accumulo.core.zookeeper.ZooUtil;
 +import org.apache.accumulo.server.Accumulo;
 +import org.apache.accumulo.server.client.HdfsZooInstance;
 +import org.apache.accumulo.server.conf.ServerConfiguration;
 +import org.apache.accumulo.server.monitor.servlets.DefaultServlet;
 +import org.apache.accumulo.server.monitor.servlets.GcStatusServlet;
 +import org.apache.accumulo.server.monitor.servlets.JSONServlet;
 +import org.apache.accumulo.server.monitor.servlets.LogServlet;
 +import org.apache.accumulo.server.monitor.servlets.MasterServlet;
 +import org.apache.accumulo.server.monitor.servlets.OperationServlet;
 +import org.apache.accumulo.server.monitor.servlets.ProblemServlet;
 +import org.apache.accumulo.server.monitor.servlets.ShellServlet;
 +import org.apache.accumulo.server.monitor.servlets.TServersServlet;
 +import org.apache.accumulo.server.monitor.servlets.TablesServlet;
 +import org.apache.accumulo.server.monitor.servlets.VisServlet;
 +import org.apache.accumulo.server.monitor.servlets.XMLServlet;
 +import org.apache.accumulo.server.monitor.servlets.trace.ListType;
 +import org.apache.accumulo.server.monitor.servlets.trace.ShowTrace;
 +import org.apache.accumulo.server.monitor.servlets.trace.Summary;
 +import org.apache.accumulo.server.problems.ProblemReports;
 +import org.apache.accumulo.server.problems.ProblemType;
 +import org.apache.accumulo.server.security.SecurityConstants;
 +import org.apache.accumulo.server.util.EmbeddedWebServer;
++import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 +import org.apache.hadoop.fs.FileSystem;
 +import org.apache.log4j.Logger;
- import org.apache.zookeeper.WatchedEvent;
- import org.apache.zookeeper.Watcher;
- import org.apache.zookeeper.ZooKeeper;
 +
 +/**
 + * Serve master statistics with an embedded web server.
 + */
 +public class Monitor {
 +  private static final Logger log = Logger.getLogger(Monitor.class);
 +  
 +  public static final int REFRESH_TIME = 5;
 +  private static long lastRecalc = 0L;
 +  private static double totalIngestRate = 0.0;
 +  private static double totalIngestByteRate = 0.0;
 +  private static double totalQueryRate = 0.0;
 +  private static double totalScanRate = 0.0;
 +  private static double totalQueryByteRate = 0.0;
 +  private static long totalEntries = 0L;
 +  private static int totalTabletCount = 0;
 +  private static int onlineTabletCount = 0;
 +  private static long totalHoldTime = 0;
 +  private static long totalLookups = 0;
 +  private static int totalTables = 0;
 +  
 +  private static class MaxList<T> extends LinkedList<Pair<Long,T>> {
 +    private static final long serialVersionUID = 1L;
 +    
 +    private long maxDelta;
 +    
 +    public MaxList(long maxDelta) {
 +      this.maxDelta = maxDelta;
 +    }
 +    
 +    @Override
 +    public boolean add(Pair<Long,T> obj) {
 +      boolean result = super.add(obj);
 +      
 +      if (obj.getFirst() - get(0).getFirst() > maxDelta)
 +        remove(0);
 +      
 +      return result;
 +    }
 +    
 +  }
 +  
 +  private static final int MAX_TIME_PERIOD = 60 * 60 * 1000;
 +  private static final List<Pair<Long,Double>> loadOverTime = Collections.synchronizedList(new MaxList<Double>(MAX_TIME_PERIOD));
 +  private static final List<Pair<Long,Double>> ingestRateOverTime = Collections.synchronizedList(new MaxList<Double>(MAX_TIME_PERIOD));
 +  private static final List<Pair<Long,Double>> ingestByteRateOverTime = Collections.synchronizedList(new MaxList<Double>(MAX_TIME_PERIOD));
 +  private static final List<Pair<Long,Integer>> recoveriesOverTime = Collections.synchronizedList(new MaxList<Integer>(MAX_TIME_PERIOD));
 +  private static final List<Pair<Long,Integer>> minorCompactionsOverTime = Collections.synchronizedList(new MaxList<Integer>(MAX_TIME_PERIOD));
 +  private static final List<Pair<Long,Integer>> majorCompactionsOverTime = Collections.synchronizedList(new MaxList<Integer>(MAX_TIME_PERIOD));
 +  private static final List<Pair<Long,Double>> lookupsOverTime = Collections.synchronizedList(new MaxList<Double>(MAX_TIME_PERIOD));
 +  private static final List<Pair<Long,Integer>> queryRateOverTime = Collections.synchronizedList(new MaxList<Integer>(MAX_TIME_PERIOD));
 +  private static final List<Pair<Long,Integer>> scanRateOverTime = Collections.synchronizedList(new MaxList<Integer>(MAX_TIME_PERIOD));
 +  private static final List<Pair<Long,Double>> queryByteRateOverTime = Collections.synchronizedList(new MaxList<Double>(MAX_TIME_PERIOD));
 +  private static final List<Pair<Long,Double>> indexCacheHitRateOverTime = Collections.synchronizedList(new MaxList<Double>(MAX_TIME_PERIOD));
 +  private static final List<Pair<Long,Double>> dataCacheHitRateOverTime = Collections.synchronizedList(new MaxList<Double>(MAX_TIME_PERIOD));
 +  private static EventCounter lookupRateTracker = new EventCounter();
 +  private static EventCounter indexCacheHitTracker = new EventCounter();
 +  private static EventCounter indexCacheRequestTracker = new EventCounter();
 +  private static EventCounter dataCacheHitTracker = new EventCounter();
 +  private static EventCounter dataCacheRequestTracker = new EventCounter();
 +  
 +  private static volatile boolean fetching = false;
 +  private static MasterMonitorInfo mmi;
 +  private static Map<String,Map<ProblemType,Integer>> problemSummary = Collections.emptyMap();
 +  private static Exception problemException;
 +  private static GCStatus gcStatus;
 +  
 +  private static Instance instance;
 +  
 +  private static ServerConfiguration config;
 +  
 +  private static EmbeddedWebServer server;
 +  
 +  public static Map<String,Double> summarizeTableStats(MasterMonitorInfo mmi) {
 +    Map<String,Double> compactingByTable = new HashMap<String,Double>();
 +    if (mmi != null && mmi.tServerInfo != null) {
 +      for (TabletServerStatus status : mmi.tServerInfo) {
 +        if (status != null && status.tableMap != null) {
 +          for (String table : status.tableMap.keySet()) {
 +            Double holdTime = compactingByTable.get(table);
 +            compactingByTable.put(table, Math.max(holdTime == null ? 0. : holdTime.doubleValue(), status.holdTime));
 +          }
 +        }
 +      }
 +    }
 +    return compactingByTable;
 +  }
 +  
 +  public static void add(TableInfo total, TableInfo more) {
 +    if (total.minors == null)
 +      total.minors = new Compacting();
 +    if (total.majors == null)
 +      total.majors = new Compacting();
 +    if (total.scans == null)
 +      total.scans = new Compacting();
 +    if (more.minors != null) {
 +      total.minors.running += more.minors.running;
 +      total.minors.queued += more.minors.queued;
 +    }
 +    if (more.majors != null) {
 +      total.majors.running += more.majors.running;
 +      total.majors.queued += more.majors.queued;
 +    }
 +    if (more.scans != null) {
 +      total.scans.running += more.scans.running;
 +      total.scans.queued += more.scans.queued;
 +    }
 +    total.onlineTablets += more.onlineTablets;
 +    total.recs += more.recs;
 +    total.recsInMemory += more.recsInMemory;
 +    total.tablets += more.tablets;
 +    total.ingestRate += more.ingestRate;
 +    total.ingestByteRate += more.ingestByteRate;
 +    total.queryRate += more.queryRate;
 +    total.queryByteRate += more.queryByteRate;
 +    total.scanRate += more.scanRate;
 +  }
 +  
 +  public static TableInfo summarizeTableStats(TabletServerStatus status) {
 +    TableInfo summary = new TableInfo();
 +    summary.majors = new Compacting();
 +    summary.minors = new Compacting();
 +    summary.scans = new Compacting();
 +    for (TableInfo rates : status.tableMap.values()) {
 +      add(summary, rates);
 +    }
 +    return summary;
 +  }
 +  
 +  private static class EventCounter {
 +    
 +    Map<String,Pair<Long,Long>> prevSamples = new HashMap<String,Pair<Long,Long>>();
 +    Map<String,Pair<Long,Long>> samples = new HashMap<String,Pair<Long,Long>>();
 +    Set<String> serversUpdated = new HashSet<String>();
 +    
 +    void startingUpdates() {
 +      serversUpdated.clear();
 +    }
 +    
 +    void updateTabletServer(String name, long sampleTime, long numEvents) {
 +      Pair<Long,Long> newSample = new Pair<Long,Long>(sampleTime, numEvents);
 +      Pair<Long,Long> lastSample = samples.get(name);
 +      
 +      if (lastSample == null || !lastSample.equals(newSample)) {
 +        samples.put(name, newSample);
 +        if (lastSample != null) {
 +          prevSamples.put(name, lastSample);
 +        }
 +      }
 +      serversUpdated.add(name);
 +    }
 +    
 +    void finishedUpdating() {
 +      // remove any tablet servers not updated
 +      samples.keySet().retainAll(serversUpdated);
 +      prevSamples.keySet().retainAll(serversUpdated);
 +    }
 +    
 +    double calculateRate() {
 +      double totalRate = 0;
 +      
 +      for (Entry<String,Pair<Long,Long>> entry : prevSamples.entrySet()) {
 +        Pair<Long,Long> prevSample = entry.getValue();
 +        Pair<Long,Long> sample = samples.get(entry.getKey());
 +        
 +        totalRate += (sample.getSecond() - prevSample.getSecond()) / ((sample.getFirst() - prevSample.getFirst()) / (double) 1000);
 +      }
 +      
 +      return totalRate;
 +    }
 +    
 +    long calculateCount() {
 +      long count = 0;
 +      
 +      for (Entry<String,Pair<Long,Long>> entry : prevSamples.entrySet()) {
 +        Pair<Long,Long> prevSample = entry.getValue();
 +        Pair<Long,Long> sample = samples.get(entry.getKey());
 +        
 +        count += sample.getSecond() - prevSample.getSecond();
 +      }
 +      
 +      return count;
 +    }
 +  }
 +  
 +  public static void fetchData() {
 +    double totalIngestRate = 0.;
 +    double totalIngestByteRate = 0.;
 +    double totalQueryRate = 0.;
 +    double totalQueryByteRate = 0.;
 +    double totalScanRate = 0.;
 +    long totalEntries = 0;
 +    int totalTabletCount = 0;
 +    int onlineTabletCount = 0;
 +    long totalHoldTime = 0;
 +    long totalLookups = 0;
 +    boolean retry = true;
 +    
 +    // only recalc every so often
 +    long currentTime = System.currentTimeMillis();
 +    if (currentTime - lastRecalc < REFRESH_TIME * 1000)
 +      return;
 +    
 +    synchronized (Monitor.class) {
 +      if (fetching)
 +        return;
 +      fetching = true;
 +    }
 +    
 +    try {
 +      while (retry) {
 +        MasterClientService.Iface client = null;
 +        try {
 +          client = MasterClient.getConnection(HdfsZooInstance.getInstance());
 +          if (client != null) {
 +            mmi = client.getMasterStats(Tracer.traceInfo(), SecurityConstants.getSystemCredentials());
 +            retry = false;
 +          } else {
 +            mmi = null;
 +          }
 +          Monitor.gcStatus = fetchGcStatus();
 +        } catch (Exception e) {
 +          mmi = null;
 +          log.info("Error fetching stats: " + e);
 +        } finally {
 +          if (client != null) {
 +            MasterClient.close(client);
 +          }
 +        }
 +        if (mmi == null)
 +          UtilWaitThread.sleep(1000);
 +      }
 +      if (mmi != null) {
 +        int majorCompactions = 0;
 +        int minorCompactions = 0;
 +        
 +        lookupRateTracker.startingUpdates();
 +        indexCacheHitTracker.startingUpdates();
 +        indexCacheRequestTracker.startingUpdates();
 +        dataCacheHitTracker.startingUpdates();
 +        dataCacheRequestTracker.startingUpdates();
 +        
 +        for (TabletServerStatus server : mmi.tServerInfo) {
 +          TableInfo summary = Monitor.summarizeTableStats(server);
 +          totalIngestRate += summary.ingestRate;
 +          totalIngestByteRate += summary.ingestByteRate;
 +          totalQueryRate += summary.queryRate;
 +          totalScanRate += summary.scanRate;
 +          totalQueryByteRate += summary.queryByteRate;
 +          totalEntries += summary.recs;
 +          totalHoldTime += server.holdTime;
 +          totalLookups += server.lookups;
 +          majorCompactions += summary.majors.running;
 +          minorCompactions += summary.minors.running;
 +          lookupRateTracker.updateTabletServer(server.name, server.lastContact, server.lookups);
 +          indexCacheHitTracker.updateTabletServer(server.name, server.lastContact, server.indexCacheHits);
 +          indexCacheRequestTracker.updateTabletServer(server.name, server.lastContact, server.indexCacheRequest);
 +          dataCacheHitTracker.updateTabletServer(server.name, server.lastContact, server.dataCacheHits);
 +          dataCacheRequestTracker.updateTabletServer(server.name, server.lastContact, server.dataCacheRequest);
 +        }
 +        
 +        lookupRateTracker.finishedUpdating();
 +        indexCacheHitTracker.finishedUpdating();
 +        indexCacheRequestTracker.finishedUpdating();
 +        dataCacheHitTracker.finishedUpdating();
 +        dataCacheRequestTracker.finishedUpdating();
 +        
 +        int totalTables = 0;
 +        for (TableInfo tInfo : mmi.tableMap.values()) {
 +          totalTabletCount += tInfo.tablets;
 +          onlineTabletCount += tInfo.onlineTablets;
 +          totalTables++;
 +        }
 +        Monitor.totalIngestRate = totalIngestRate;
 +        Monitor.totalTables = totalTables;
 +        totalIngestByteRate = totalIngestByteRate / 1000000.0;
 +        Monitor.totalIngestByteRate = totalIngestByteRate;
 +        Monitor.totalQueryRate = totalQueryRate;
 +        Monitor.totalScanRate = totalScanRate;
 +        totalQueryByteRate = totalQueryByteRate / 1000000.0;
 +        Monitor.totalQueryByteRate = totalQueryByteRate;
 +        Monitor.totalEntries = totalEntries;
 +        Monitor.totalTabletCount = totalTabletCount;
 +        Monitor.onlineTabletCount = onlineTabletCount;
 +        Monitor.totalHoldTime = totalHoldTime;
 +        Monitor.totalLookups = totalLookups;
 +        
 +        ingestRateOverTime.add(new Pair<Long,Double>(currentTime, totalIngestRate));
 +        ingestByteRateOverTime.add(new Pair<Long,Double>(currentTime, totalIngestByteRate));
 +        
 +        double totalLoad = 0.;
 +        for (TabletServerStatus status : mmi.tServerInfo) {
 +          if (status != null)
 +            totalLoad += status.osLoad;
 +        }
 +        loadOverTime.add(new Pair<Long,Double>(currentTime, totalLoad));
 +        
 +        minorCompactionsOverTime.add(new Pair<Long,Integer>(currentTime, minorCompactions));
 +        majorCompactionsOverTime.add(new Pair<Long,Integer>(currentTime, majorCompactions));
 +        
 +        lookupsOverTime.add(new Pair<Long,Double>(currentTime, lookupRateTracker.calculateRate()));
 +        
 +        queryRateOverTime.add(new Pair<Long,Integer>(currentTime, (int) totalQueryRate));
 +        queryByteRateOverTime.add(new Pair<Long,Double>(currentTime, totalQueryByteRate));
 +        
 +        scanRateOverTime.add(new Pair<Long,Integer>(currentTime, (int) totalScanRate));
 +        
 +        calcCacheHitRate(indexCacheHitRateOverTime, currentTime, indexCacheHitTracker, indexCacheRequestTracker);
 +        calcCacheHitRate(dataCacheHitRateOverTime, currentTime, dataCacheHitTracker, dataCacheRequestTracker);
 +      }
 +      try {
 +        Monitor.problemSummary = ProblemReports.getInstance().summarize();
 +        Monitor.problemException = null;
 +      } catch (Exception e) {
 +        log.info("Failed to obtain problem reports ", e);
 +        Monitor.problemSummary = Collections.emptyMap();
 +        Monitor.problemException = e;
 +      }
 +      
 +    } finally {
 +      synchronized (Monitor.class) {
 +        fetching = false;
 +        lastRecalc = currentTime;
 +      }
 +    }
 +  }
 +  
 +  private static void calcCacheHitRate(List<Pair<Long,Double>> hitRate, long currentTime, EventCounter cacheHits, EventCounter cacheReq) {
 +    long req = cacheReq.calculateCount();
 +    if (req > 0)
 +      hitRate.add(new Pair<Long,Double>(currentTime, cacheHits.calculateCount() / (double) cacheReq.calculateCount()));
 +    else
 +      hitRate.add(new Pair<Long,Double>(currentTime, null));
 +  }
 +  
 +  private static GCStatus fetchGcStatus() {
 +    GCStatus result = null;
 +    InetSocketAddress address = null;
 +    try {
 +      // Read the gc location from its lock
-       Instance instance = HdfsZooInstance.getInstance();
-       String zooKeepers = instance.getZooKeepers();
-       log.debug("connecting to zookeepers " + zooKeepers);
-       ZooKeeper zk = new ZooKeeper(zooKeepers, (int) config.getConfiguration().getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT), new Watcher() {
-         @Override
-         public void process(WatchedEvent event) {}
-       });
++      ZooReaderWriter zk = ZooReaderWriter.getInstance();
 +      try {
 +        String path = ZooUtil.getRoot(HdfsZooInstance.getInstance()) + Constants.ZGC_LOCK;
 +        List<String> locks = zk.getChildren(path, null);
 +        if (locks != null && locks.size() > 0) {
 +          Collections.sort(locks);
-           address = new ServerServices(new String(zk.getData(path + "/" + locks.get(0), null, null))).getAddress(Service.GC_CLIENT);
-           GCMonitorService.Client client = ThriftUtil.getClient(new GCMonitorService.Client.Factory(), address, config.getConfiguration());
++          address = new ServerServices(new String(zk.getData(path + "/" + locks.get(0), null))).getAddress(Service.GC_CLIENT);
++          GCMonitorService.Client client = ThriftUtil.getClient(new GCMonitorService.Client.Factory(), address, ServerConfiguration.getSystemConfiguration(instance));
 +          try {
 +            result = client.getStatus(Tracer.traceInfo(), SecurityConstants.getSystemCredentials());
 +          } finally {
 +            ThriftUtil.returnClient(client);
 +          }
 +        }
 +      } finally {
 +        zk.close();
 +      }
 +    } catch (Exception ex) {
 +      log.warn("Unable to contact the garbage collector at " + address, ex);
 +    }
 +    return result;
 +  }
 +  
 +  public static void main(String[] args) throws Exception {
 +    SecurityUtil.serverLogin();
 +    
 +    FileSystem fs = FileUtil.getFileSystem(CachedConfiguration.getInstance(), ServerConfiguration.getSiteConfiguration());
 +    String hostname = Accumulo.getLocalAddress(args);
 +    instance = HdfsZooInstance.getInstance();
 +    config = new ServerConfiguration(instance);
 +    Accumulo.init(fs, config, "monitor");
 +    Monitor monitor = new Monitor();
 +    Accumulo.enableTracing(hostname, "monitor");
 +    monitor.run(hostname);
 +  }
 +  
 +  private static long START_TIME;
 +  
 +  public void run(String hostname) {
 +    Monitor.START_TIME = System.currentTimeMillis();
 +    int port = config.getConfiguration().getPort(Property.MONITOR_PORT);
 +    try {
 +      log.debug("Creating monitor on port " + port);
 +      server = new EmbeddedWebServer(hostname, port);
 +    } catch (Throwable ex) {
 +      log.error("Unable to start embedded web server", ex);
 +      throw new RuntimeException(ex);
 +    }
 +    
 +    server.addServlet(DefaultServlet.class, "/");
 +    server.addServlet(OperationServlet.class, "/op");
 +    server.addServlet(MasterServlet.class, "/master");
 +    server.addServlet(TablesServlet.class, "/tables");
 +    server.addServlet(TServersServlet.class, "/tservers");
 +    server.addServlet(ProblemServlet.class, "/problems");
 +    server.addServlet(GcStatusServlet.class, "/gc");
 +    server.addServlet(LogServlet.class, "/log");
 +    server.addServlet(XMLServlet.class, "/xml");
 +    server.addServlet(JSONServlet.class, "/json");
 +    server.addServlet(VisServlet.class, "/vis");
 +    server.addServlet(Summary.class, "/trace/summary");
 +    server.addServlet(ListType.class, "/trace/listType");
 +    server.addServlet(ShowTrace.class, "/trace/show");
 +    if (server.isUsingSsl())
 +      server.addServlet(ShellServlet.class, "/shell");
 +    LogService.startLogListener(Monitor.getSystemConfiguration());
 +    server.start();
 +    
 +    new Daemon(new LoggingRunnable(log, new ZooKeeperStatus()), "ZooKeeperStatus").start();
 +    
 +    // need to regularly fetch data so plot data is updated
 +    new Daemon(new LoggingRunnable(log, new Runnable() {
 +      
 +      @Override
 +      public void run() {
 +        while (true) {
 +          try {
 +            Monitor.fetchData();
 +          } catch (Exception e) {
 +            log.warn(e.getMessage(), e);
 +          }
 +          
 +          UtilWaitThread.sleep(333);
 +        }
 +        
 +      }
 +    }), "Data fetcher").start();
 +  }
 +  
 +  public static MasterMonitorInfo getMmi() {
 +    return mmi;
 +  }
 +  
 +  public static int getTotalTables() {
 +    return totalTables;
 +  }
 +  
 +  public static int getTotalTabletCount() {
 +    return totalTabletCount;
 +  }
 +  
 +  public static int getOnlineTabletCount() {
 +    return onlineTabletCount;
 +  }
 +  
 +  public static long getTotalEntries() {
 +    return totalEntries;
 +  }
 +  
 +  public static double getTotalIngestRate() {
 +    return totalIngestRate;
 +  }
 +  
 +  public static double getTotalIngestByteRate() {
 +    return totalIngestByteRate;
 +  }
 +  
 +  public static double getTotalQueryRate() {
 +    return totalQueryRate;
 +  }
 +  
 +  public static double getTotalScanRate() {
 +    return totalScanRate;
 +  }
 +  
 +  public static double getTotalQueryByteRate() {
 +    return totalQueryByteRate;
 +  }
 +  
 +  public static long getTotalHoldTime() {
 +    return totalHoldTime;
 +  }
 +  
 +  public static Exception getProblemException() {
 +    return problemException;
 +  }
 +  
 +  public static Map<String,Map<ProblemType,Integer>> getProblemSummary() {
 +    return problemSummary;
 +  }
 +  
 +  public static GCStatus getGcStatus() {
 +    return gcStatus;
 +  }
 +  
 +  public static long getTotalLookups() {
 +    return totalLookups;
 +  }
 +  
 +  public static long getStartTime() {
 +    return START_TIME;
 +  }
 +  
 +  public static List<Pair<Long,Double>> getLoadOverTime() {
 +    synchronized (loadOverTime) {
 +      return new ArrayList<Pair<Long,Double>>(loadOverTime);
 +    }
 +  }
 +  
 +  public static List<Pair<Long,Double>> getIngestRateOverTime() {
 +    synchronized (ingestRateOverTime) {
 +      return new ArrayList<Pair<Long,Double>>(ingestRateOverTime);
 +    }
 +  }
 +  
 +  public static List<Pair<Long,Double>> getIngestByteRateOverTime() {
 +    synchronized (ingestByteRateOverTime) {
 +      return new ArrayList<Pair<Long,Double>>(ingestByteRateOverTime);
 +    }
 +  }
 +  
 +  public static List<Pair<Long,Integer>> getRecoveriesOverTime() {
 +    synchronized (recoveriesOverTime) {
 +      return new ArrayList<Pair<Long,Integer>>(recoveriesOverTime);
 +    }
 +  }
 +  
 +  public static List<Pair<Long,Integer>> getMinorCompactionsOverTime() {
 +    synchronized (minorCompactionsOverTime) {
 +      return new ArrayList<Pair<Long,Integer>>(minorCompactionsOverTime);
 +    }
 +  }
 +  
 +  public static List<Pair<Long,Integer>> getMajorCompactionsOverTime() {
 +    synchronized (majorCompactionsOverTime) {
 +      return new ArrayList<Pair<Long,Integer>>(majorCompactionsOverTime);
 +    }
 +  }
 +  
 +  public static List<Pair<Long,Double>> getLookupsOverTime() {
 +    synchronized (lookupsOverTime) {
 +      return new ArrayList<Pair<Long,Double>>(lookupsOverTime);
 +    }
 +  }
 +  
 +  public static double getLookupRate() {
 +    return lookupRateTracker.calculateRate();
 +  }
 +  
 +  public static List<Pair<Long,Integer>> getQueryRateOverTime() {
 +    synchronized (queryRateOverTime) {
 +      return new ArrayList<Pair<Long,Integer>>(queryRateOverTime);
 +    }
 +  }
 +  
 +  public static List<Pair<Long,Integer>> getScanRateOverTime() {
 +    synchronized (scanRateOverTime) {
 +      return new ArrayList<Pair<Long,Integer>>(scanRateOverTime);
 +    }
 +  }
 +  
 +  public static List<Pair<Long,Double>> getQueryByteRateOverTime() {
 +    synchronized (queryByteRateOverTime) {
 +      return new ArrayList<Pair<Long,Double>>(queryByteRateOverTime);
 +    }
 +  }
 +  
 +  public static List<Pair<Long,Double>> getIndexCacheHitRateOverTime() {
 +    synchronized (indexCacheHitRateOverTime) {
 +      return new ArrayList<Pair<Long,Double>>(indexCacheHitRateOverTime);
 +    }
 +  }
 +  
 +  public static List<Pair<Long,Double>> getDataCacheHitRateOverTime() {
 +    synchronized (dataCacheHitRateOverTime) {
 +      return new ArrayList<Pair<Long,Double>>(dataCacheHitRateOverTime);
 +    }
 +  }
 +  
 +  public static AccumuloConfiguration getSystemConfiguration() {
 +    return config.getConfiguration();
 +  }
 +  
 +  public static Instance getInstance() {
 +    return instance;
 +  }
 +  
 +  public static boolean isUsingSsl() {
 +    return server.isUsingSsl();
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/041270b7/server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java
----------------------------------------------------------------------
diff --cc server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java
index aa5fe48,0000000..f859523
mode 100644,000000..100644
--- a/server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java
+++ b/server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java
@@@ -1,1257 -1,0 +1,1259 @@@
 +/*
 + * 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.io.IOException;
 +import java.util.ArrayList;
 +import java.util.Arrays;
 +import java.util.Collection;
 +import java.util.Collections;
 +import java.util.Comparator;
 +import java.util.HashMap;
 +import java.util.HashSet;
 +import java.util.Iterator;
 +import java.util.List;
 +import java.util.Map;
 +import java.util.Map.Entry;
 +import java.util.Set;
 +import java.util.SortedMap;
 +import java.util.TreeMap;
 +import java.util.concurrent.TimeUnit;
 +
 +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.BatchWriter;
 +import org.apache.accumulo.core.client.BatchWriterConfig;
 +import org.apache.accumulo.core.client.Connector;
 +import org.apache.accumulo.core.client.Instance;
 +import org.apache.accumulo.core.client.IsolatedScanner;
 +import org.apache.accumulo.core.client.MutationsRejectedException;
 +import org.apache.accumulo.core.client.Scanner;
 +import org.apache.accumulo.core.client.TableNotFoundException;
 +import org.apache.accumulo.core.client.impl.BatchWriterImpl;
 +import org.apache.accumulo.core.client.impl.ScannerImpl;
 +import org.apache.accumulo.core.client.impl.Writer;
 +import org.apache.accumulo.core.data.Key;
 +import org.apache.accumulo.core.data.KeyExtent;
 +import org.apache.accumulo.core.data.Mutation;
 +import org.apache.accumulo.core.data.PartialKey;
 +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.security.CredentialHelper;
 +import org.apache.accumulo.core.security.thrift.TCredentials;
 +import org.apache.accumulo.core.tabletserver.thrift.ConstraintViolationException;
 +import org.apache.accumulo.core.util.CachedConfiguration;
 +import org.apache.accumulo.core.util.ColumnFQ;
 +import org.apache.accumulo.core.util.FastFormat;
 +import org.apache.accumulo.core.util.Pair;
 +import org.apache.accumulo.core.util.StringUtil;
 +import org.apache.accumulo.core.util.UtilWaitThread;
 +import org.apache.accumulo.core.zookeeper.ZooUtil;
 +import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 +import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
 +import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy;
 +import org.apache.accumulo.server.ServerConstants;
 +import org.apache.accumulo.server.client.HdfsZooInstance;
 +import org.apache.accumulo.server.conf.ServerConfiguration;
 +import org.apache.accumulo.server.master.state.TServerInstance;
 +import org.apache.accumulo.server.security.SecurityConstants;
 +import org.apache.accumulo.server.trace.TraceFileSystem;
 +import org.apache.accumulo.server.zookeeper.ZooLock;
 +import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 +import org.apache.hadoop.fs.FileStatus;
 +import org.apache.hadoop.fs.FileSystem;
 +import org.apache.hadoop.fs.Path;
 +import org.apache.hadoop.io.DataInputBuffer;
 +import org.apache.hadoop.io.DataOutputBuffer;
 +import org.apache.hadoop.io.Text;
 +import org.apache.log4j.Logger;
 +import org.apache.zookeeper.KeeperException;
 +
 +/**
 + * provides a reference to the metadata table for updates by tablet servers
 + */
 +public class MetadataTable extends org.apache.accumulo.core.util.MetadataTable {
 +  
 +  private static final Text EMPTY_TEXT = new Text();
 +  private static Map<TCredentials,Writer> metadata_tables = new HashMap<TCredentials,Writer>();
 +  private static final Logger log = Logger.getLogger(MetadataTable.class);
 +  
 +  private static final int SAVE_ROOT_TABLET_RETRIES = 3;
 +  
 +  private MetadataTable() {
 +    
 +  }
 +  
 +  public synchronized static Writer getMetadataTable(TCredentials credentials) {
 +    Writer metadataTable = metadata_tables.get(credentials);
 +    if (metadataTable == null) {
 +      metadataTable = new Writer(HdfsZooInstance.getInstance(), credentials, Constants.METADATA_TABLE_ID);
 +      metadata_tables.put(credentials, metadataTable);
 +    }
 +    return metadataTable;
 +  }
 +  
 +  public static void putLockID(ZooLock zooLock, Mutation m) {
 +    Constants.METADATA_LOCK_COLUMN.put(m, new Value(zooLock.getLockID().serialize(ZooUtil.getRoot(HdfsZooInstance.getInstance()) + "/").getBytes()));
 +  }
 +  
 +  public static void update(TCredentials credentials, Mutation m) {
 +    update(credentials, null, m);
 +  }
 +  
 +  public static void update(TCredentials credentials, ZooLock zooLock, Mutation m) {
 +    Writer t;
 +    t = getMetadataTable(credentials);
 +    if (zooLock != null)
 +      putLockID(zooLock, m);
 +    while (true) {
 +      try {
 +        t.update(m);
 +        return;
 +      } catch (AccumuloException e) {
 +        log.error(e, e);
 +      } catch (AccumuloSecurityException e) {
 +        log.error(e, e);
 +      } catch (ConstraintViolationException e) {
 +        log.error(e, e);
 +      } catch (TableNotFoundException e) {
 +        log.error(e, e);
 +      }
 +      UtilWaitThread.sleep(1000);
 +    }
 +    
 +  }
 +  
 +  /**
 +   * new data file update function adds one data file to a tablet's list
 +   * 
 +   * path should be relative to the table directory
 +   * 
 +   * @param time
 +   * @param filesInUseByScans
 +   * @param zooLock
 +   * @param flushId
 +   * 
 +   */
 +  public static void updateTabletDataFile(KeyExtent extent, String path, String mergeFile, DataFileValue dfv, String time, TCredentials credentials,
 +      Set<String> filesInUseByScans, String address, ZooLock zooLock, Set<String> unusedWalLogs, TServerInstance lastLocation, long flushId) {
 +    if (extent.equals(Constants.ROOT_TABLET_EXTENT)) {
 +      if (unusedWalLogs != null) {
 +        IZooReaderWriter zk = ZooReaderWriter.getInstance();
 +        // unusedWalLogs will contain the location/name of each log in a log set
 +        // the log set is stored under one of the log names, but not both
 +        // find the entry under one of the names and delete it.
 +        String root = getZookeeperLogLocation();
 +        boolean foundEntry = false;
 +        for (String entry : unusedWalLogs) {
 +          String[] parts = entry.split("/");
 +          String zpath = root + "/" + parts[1];
 +          while (true) {
 +            try {
 +              if (zk.exists(zpath)) {
 +                zk.recursiveDelete(zpath, NodeMissingPolicy.SKIP);
 +                foundEntry = true;
 +              }
 +              break;
 +            } catch (KeeperException e) {
 +              log.error(e, e);
 +            } catch (InterruptedException e) {
 +              log.error(e, e);
 +            }
 +            UtilWaitThread.sleep(1000);
 +          }
 +        }
 +        if (unusedWalLogs.size() > 0 && !foundEntry)
 +          log.warn("WALog entry for root tablet did not exist " + unusedWalLogs);
 +      }
 +      return;
 +    }
 +    
 +    Mutation m = new Mutation(extent.getMetadataEntry());
 +    
 +    if (dfv.getNumEntries() > 0) {
 +      m.put(Constants.METADATA_DATAFILE_COLUMN_FAMILY, new Text(path), new Value(dfv.encode()));
 +      Constants.METADATA_TIME_COLUMN.put(m, new Value(time.getBytes()));
 +      // stuff in this location
 +      TServerInstance self = getTServerInstance(address, zooLock);
 +      self.putLastLocation(m);
 +      // erase the old location
 +      if (lastLocation != null && !lastLocation.equals(self))
 +        lastLocation.clearLastLocation(m);
 +    }
 +    if (unusedWalLogs != null) {
 +      for (String entry : unusedWalLogs) {
 +        m.putDelete(Constants.METADATA_LOG_COLUMN_FAMILY, new Text(entry));
 +      }
 +    }
 +    
 +    for (String scanFile : filesInUseByScans)
 +      m.put(Constants.METADATA_SCANFILE_COLUMN_FAMILY, new Text(scanFile), new Value("".getBytes()));
 +    
 +    if (mergeFile != null)
 +      m.putDelete(Constants.METADATA_DATAFILE_COLUMN_FAMILY, new Text(mergeFile));
 +    
 +    Constants.METADATA_FLUSH_COLUMN.put(m, new Value((flushId + "").getBytes()));
 +    
 +    update(credentials, zooLock, m);
 +    
 +  }
 +  
 +  private static TServerInstance getTServerInstance(String address, ZooLock zooLock) {
 +    while (true) {
 +      try {
 +        return new TServerInstance(address, zooLock.getSessionId());
 +      } catch (KeeperException e) {
 +        log.error(e, e);
 +      } catch (InterruptedException e) {
 +        log.error(e, e);
 +      }
 +      UtilWaitThread.sleep(1000);
 +    }
 +  }
 +  
 +  public static void updateTabletFlushID(KeyExtent extent, long flushID, TCredentials credentials, ZooLock zooLock) {
 +    if (!extent.isRootTablet()) {
 +      Mutation m = new Mutation(extent.getMetadataEntry());
 +      Constants.METADATA_FLUSH_COLUMN.put(m, new Value((flushID + "").getBytes()));
 +      update(credentials, zooLock, m);
 +    }
 +  }
 +  
 +  public static void updateTabletCompactID(KeyExtent extent, long compactID, TCredentials credentials, ZooLock zooLock) {
 +    if (!extent.isRootTablet()) {
 +      Mutation m = new Mutation(extent.getMetadataEntry());
 +      Constants.METADATA_COMPACT_COLUMN.put(m, new Value((compactID + "").getBytes()));
 +      update(credentials, zooLock, m);
 +    }
 +  }
 +  
 +  public static void updateTabletDataFile(long tid, KeyExtent extent, Map<String,DataFileValue> estSizes, String time, TCredentials credentials, ZooLock zooLock) {
 +    Mutation m = new Mutation(extent.getMetadataEntry());
 +    byte[] tidBytes = Long.toString(tid).getBytes();
 +    
 +    for (Entry<String,DataFileValue> entry : estSizes.entrySet()) {
 +      Text file = new Text(entry.getKey());
 +      m.put(Constants.METADATA_DATAFILE_COLUMN_FAMILY, file, new Value(entry.getValue().encode()));
 +      m.put(Constants.METADATA_BULKFILE_COLUMN_FAMILY, file, new Value(tidBytes));
 +    }
 +    Constants.METADATA_TIME_COLUMN.put(m, new Value(time.getBytes()));
 +    update(credentials, zooLock, m);
 +  }
 +  
 +  public static void addTablet(KeyExtent extent, String path, TCredentials credentials, char timeType, ZooLock lock) {
 +    Mutation m = extent.getPrevRowUpdateMutation();
 +    
 +    Constants.METADATA_DIRECTORY_COLUMN.put(m, new Value(path.getBytes()));
 +    Constants.METADATA_TIME_COLUMN.put(m, new Value((timeType + "0").getBytes()));
 +    
 +    update(credentials, lock, m);
 +  }
 +  
 +  public static void updateTabletPrevEndRow(KeyExtent extent, TCredentials credentials) {
 +    Mutation m = extent.getPrevRowUpdateMutation(); //
 +    update(credentials, m);
 +  }
 +  
 +  /**
 +   * convenience method for reading entries from the metadata table
 +   */
 +  public static SortedMap<KeyExtent,Text> getMetadataDirectoryEntries(SortedMap<Key,Value> entries) {
 +    Key key;
 +    Value val;
 +    Text datafile = null;
 +    Value prevRow = null;
 +    KeyExtent ke;
 +    
 +    SortedMap<KeyExtent,Text> results = new TreeMap<KeyExtent,Text>();
 +    
 +    Text lastRowFromKey = new Text();
 +    
 +    // text obj below is meant to be reused in loop for efficiency
 +    Text colf = new Text();
 +    Text colq = new Text();
 +    
 +    for (Entry<Key,Value> entry : entries.entrySet()) {
 +      key = entry.getKey();
 +      val = entry.getValue();
 +      
 +      if (key.compareRow(lastRowFromKey) != 0) {
 +        prevRow = null;
 +        datafile = null;
 +        key.getRow(lastRowFromKey);
 +      }
 +      
 +      colf = key.getColumnFamily(colf);
 +      colq = key.getColumnQualifier(colq);
 +      
 +      // interpret the row id as a key extent
 +      if (Constants.METADATA_DIRECTORY_COLUMN.equals(colf, colq))
 +        datafile = new Text(val.toString());
 +      
 +      else if (Constants.METADATA_PREV_ROW_COLUMN.equals(colf, colq))
 +        prevRow = new Value(val);
 +      
 +      if (datafile != null && prevRow != null) {
 +        ke = new KeyExtent(key.getRow(), prevRow);
 +        results.put(ke, datafile);
 +        
 +        datafile = null;
 +        prevRow = null;
 +      }
 +    }
 +    return results;
 +  }
 +  
 +  public static boolean recordRootTabletLocation(String address) {
 +    IZooReaderWriter zoo = ZooReaderWriter.getInstance();
 +    for (int i = 0; i < SAVE_ROOT_TABLET_RETRIES; i++) {
 +      try {
 +        log.info("trying to write root tablet location to ZooKeeper as " + address);
 +        String zRootLocPath = ZooUtil.getRoot(HdfsZooInstance.getInstance()) + Constants.ZROOT_TABLET_LOCATION;
 +        zoo.putPersistentData(zRootLocPath, address.getBytes(), NodeExistsPolicy.OVERWRITE);
 +        return true;
 +      } catch (Exception e) {
 +        log.error("Master: unable to save root tablet location in zookeeper. exception: " + e, e);
 +      }
 +    }
 +    log.error("Giving up after " + SAVE_ROOT_TABLET_RETRIES + " retries");
 +    return false;
 +  }
 +  
 +  public static SortedMap<String,DataFileValue> getDataFileSizes(KeyExtent extent, TCredentials credentials) {
 +    TreeMap<String,DataFileValue> sizes = new TreeMap<String,DataFileValue>();
 +    
 +    Scanner mdScanner = new ScannerImpl(HdfsZooInstance.getInstance(), credentials, Constants.METADATA_TABLE_ID, Constants.NO_AUTHS);
 +    mdScanner.fetchColumnFamily(Constants.METADATA_DATAFILE_COLUMN_FAMILY);
 +    Text row = extent.getMetadataEntry();
 +    
 +    Key endKey = new Key(row, Constants.METADATA_DATAFILE_COLUMN_FAMILY, new Text(""));
 +    endKey = endKey.followingKey(PartialKey.ROW_COLFAM);
 +    
 +    mdScanner.setRange(new Range(new Key(row), endKey));
 +    for (Entry<Key,Value> entry : mdScanner) {
 +      
 +      if (!entry.getKey().getRow().equals(row))
 +        break;
 +      DataFileValue dfv = new DataFileValue(entry.getValue().get());
 +      sizes.put(entry.getKey().getColumnQualifier().toString(), dfv);
 +    }
 +    
 +    return sizes;
 +  }
 +  
 +  public static void addNewTablet(KeyExtent extent, String path, TServerInstance location, Map<String,DataFileValue> datafileSizes,
 +      Map<String,Long> bulkLoadedFiles, TCredentials credentials, String time, long lastFlushID, long lastCompactID, ZooLock zooLock) {
 +    Mutation m = extent.getPrevRowUpdateMutation();
 +    
 +    Constants.METADATA_DIRECTORY_COLUMN.put(m, new Value(path.getBytes()));
 +    Constants.METADATA_TIME_COLUMN.put(m, new Value(time.getBytes()));
 +    if (lastFlushID > 0)
 +      Constants.METADATA_FLUSH_COLUMN.put(m, new Value(("" + lastFlushID).getBytes()));
 +    if (lastCompactID > 0)
 +      Constants.METADATA_COMPACT_COLUMN.put(m, new Value(("" + lastCompactID).getBytes()));
 +    
 +    if (location != null) {
 +      m.put(Constants.METADATA_CURRENT_LOCATION_COLUMN_FAMILY, location.asColumnQualifier(), location.asMutationValue());
 +      m.putDelete(Constants.METADATA_FUTURE_LOCATION_COLUMN_FAMILY, location.asColumnQualifier());
 +    }
 +    
 +    for (Entry<String,DataFileValue> entry : datafileSizes.entrySet()) {
 +      m.put(Constants.METADATA_DATAFILE_COLUMN_FAMILY, new Text(entry.getKey()), new Value(entry.getValue().encode()));
 +    }
 +    
 +    for (Entry<String,Long> entry : bulkLoadedFiles.entrySet()) {
 +      byte[] tidBytes = Long.toString(entry.getValue()).getBytes();
 +      m.put(Constants.METADATA_BULKFILE_COLUMN_FAMILY, new Text(entry.getKey()), new Value(tidBytes));
 +    }
 +    
 +    update(credentials, zooLock, m);
 +  }
 +  
 +  public static void rollBackSplit(Text metadataEntry, Text oldPrevEndRow, TCredentials credentials, ZooLock zooLock) {
 +    KeyExtent ke = new KeyExtent(metadataEntry, oldPrevEndRow);
 +    Mutation m = ke.getPrevRowUpdateMutation();
 +    Constants.METADATA_SPLIT_RATIO_COLUMN.putDelete(m);
 +    Constants.METADATA_OLD_PREV_ROW_COLUMN.putDelete(m);
 +    update(credentials, zooLock, m);
 +  }
 +
 +  public static void splitTablet(KeyExtent extent, Text oldPrevEndRow, double splitRatio, TCredentials credentials, ZooLock zooLock) {
 +    Mutation m = extent.getPrevRowUpdateMutation(); //
 +    
 +    Constants.METADATA_SPLIT_RATIO_COLUMN.put(m, new Value(Double.toString(splitRatio).getBytes()));
 +    
 +    Constants.METADATA_OLD_PREV_ROW_COLUMN.put(m, KeyExtent.encodePrevEndRow(oldPrevEndRow));
 +    Constants.METADATA_CHOPPED_COLUMN.putDelete(m);
 +    update(credentials, zooLock, m);
 +  }
 +  
 +  public static void finishSplit(Text metadataEntry, Map<String,DataFileValue> datafileSizes, List<String> highDatafilesToRemove, TCredentials credentials,
 +      ZooLock zooLock) {
 +    Mutation m = new Mutation(metadataEntry);
 +    Constants.METADATA_SPLIT_RATIO_COLUMN.putDelete(m);
 +    Constants.METADATA_OLD_PREV_ROW_COLUMN.putDelete(m);
 +    Constants.METADATA_CHOPPED_COLUMN.putDelete(m);
 +    
 +    for (Entry<String,DataFileValue> entry : datafileSizes.entrySet()) {
 +      m.put(Constants.METADATA_DATAFILE_COLUMN_FAMILY, new Text(entry.getKey()), new Value(entry.getValue().encode()));
 +    }
 +    
 +    for (String pathToRemove : highDatafilesToRemove) {
 +      m.putDelete(Constants.METADATA_DATAFILE_COLUMN_FAMILY, new Text(pathToRemove));
 +    }
 +    
 +    update(credentials, zooLock, m);
 +  }
 +  
 +  public static void finishSplit(KeyExtent extent, Map<String,DataFileValue> datafileSizes, List<String> highDatafilesToRemove, TCredentials credentials,
 +      ZooLock zooLock) {
 +    finishSplit(extent.getMetadataEntry(), datafileSizes, highDatafilesToRemove, credentials, zooLock);
 +  }
 +  
 +  public static void replaceDatafiles(KeyExtent extent, Set<String> datafilesToDelete, Set<String> scanFiles, String path, Long compactionId,
 +      DataFileValue size, TCredentials credentials, String address, TServerInstance lastLocation, ZooLock zooLock) {
 +    replaceDatafiles(extent, datafilesToDelete, scanFiles, path, compactionId, size, credentials, address, lastLocation, zooLock, true);
 +  }
 +  
 +  public static void replaceDatafiles(KeyExtent extent, Set<String> datafilesToDelete, Set<String> scanFiles, String path, Long compactionId,
 +      DataFileValue size, TCredentials credentials, String address, TServerInstance lastLocation, ZooLock zooLock, boolean insertDeleteFlags) {
 +    
 +    if (insertDeleteFlags) {
 +      // add delete flags for those paths before the data file reference is removed
 +      addDeleteEntries(extent, datafilesToDelete, credentials);
 +    }
 +    
 +    // replace data file references to old mapfiles with the new mapfiles
 +    Mutation m = new Mutation(extent.getMetadataEntry());
 +    
 +    for (String pathToRemove : datafilesToDelete)
 +      m.putDelete(Constants.METADATA_DATAFILE_COLUMN_FAMILY, new Text(pathToRemove));
 +    
 +    for (String scanFile : scanFiles)
 +      m.put(Constants.METADATA_SCANFILE_COLUMN_FAMILY, new Text(scanFile), new Value("".getBytes()));
 +    
 +    if (size.getNumEntries() > 0)
 +      m.put(Constants.METADATA_DATAFILE_COLUMN_FAMILY, new Text(path), new Value(size.encode()));
 +    
 +    if (compactionId != null)
 +      Constants.METADATA_COMPACT_COLUMN.put(m, new Value(("" + compactionId).getBytes()));
 +    
 +    TServerInstance self = getTServerInstance(address, zooLock);
 +    self.putLastLocation(m);
 +    
 +    // remove the old location
 +    if (lastLocation != null && !lastLocation.equals(self))
 +      lastLocation.clearLastLocation(m);
 +    
 +    update(credentials, zooLock, m);
 +  }
 +  
 +  public static void addDeleteEntries(KeyExtent extent, Set<String> datafilesToDelete, TCredentials credentials) {
 +    
 +    String tableId = extent.getTableId().toString();
 +    
 +    // TODO could use batch writer,would need to handle failure and retry like update does - ACCUMULO-1294
 +    for (String pathToRemove : datafilesToDelete)
 +      update(credentials, createDeleteMutation(tableId, pathToRemove));
 +  }
 +  
 +  public static void addDeleteEntry(String tableId, String path) {
 +    update(SecurityConstants.getSystemCredentials(), createDeleteMutation(tableId, path));
 +  }
 +  
 +  public static Mutation createDeleteMutation(String tableId, String pathToRemove) {
 +    Mutation delFlag;
 +    String prefix = Constants.METADATA_DELETE_FLAG_PREFIX;
 +    if (tableId.equals(Constants.METADATA_TABLE_ID))
 +      prefix = Constants.METADATA_DELETE_FLAG_FOR_METADATA_PREFIX;
 +
 +    if (pathToRemove.startsWith("../"))
 +      delFlag = new Mutation(new Text(prefix + pathToRemove.substring(2)));
 +    else
 +      delFlag = new Mutation(new Text(prefix + "/" + tableId + pathToRemove));
 +
 +    delFlag.put(EMPTY_TEXT, EMPTY_TEXT, new Value(new byte[] {}));
 +    return delFlag;
 +  }
 +  
 +  public static void removeScanFiles(KeyExtent extent, Set<String> scanFiles, TCredentials credentials, ZooLock zooLock) {
 +    Mutation m = new Mutation(extent.getMetadataEntry());
 +    
 +    for (String pathToRemove : scanFiles)
 +      m.putDelete(Constants.METADATA_SCANFILE_COLUMN_FAMILY, new Text(pathToRemove));
 +    
 +    update(credentials, zooLock, m);
 +  }
 +  
 +  private static KeyExtent fixSplit(Text table, Text metadataEntry, Text metadataPrevEndRow, Value oper, double splitRatio, TServerInstance tserver,
 +      TCredentials credentials, String time, long initFlushID, long initCompactID, ZooLock lock) throws AccumuloException {
 +    if (metadataPrevEndRow == null)
 +      // something is wrong, this should not happen... if a tablet is split, it will always have a
 +      // prev end row....
 +      throw new AccumuloException("Split tablet does not have prev end row, something is amiss, extent = " + metadataEntry);
 +    
 +    // check to see if prev tablet exist in metadata tablet
 +    Key prevRowKey = new Key(new Text(KeyExtent.getMetadataEntry(table, metadataPrevEndRow)));
 +
 +    ScannerImpl scanner2 = new ScannerImpl(HdfsZooInstance.getInstance(), credentials, Constants.METADATA_TABLE_ID, Constants.NO_AUTHS);
 +    scanner2.setRange(new Range(prevRowKey, prevRowKey.followingKey(PartialKey.ROW)));
 +    
 +    if (!scanner2.iterator().hasNext()) {
 +      log.info("Rolling back incomplete split " + metadataEntry + " " + metadataPrevEndRow);
 +      rollBackSplit(metadataEntry, KeyExtent.decodePrevEndRow(oper), credentials, lock);
 +      return new KeyExtent(metadataEntry, KeyExtent.decodePrevEndRow(oper));
 +    } else {
 +      log.info("Finishing incomplete split " + metadataEntry + " " + metadataPrevEndRow);
 +
 +      List<String> highDatafilesToRemove = new ArrayList<String>();
 +
 +      Scanner scanner3 = new ScannerImpl(HdfsZooInstance.getInstance(), credentials, Constants.METADATA_TABLE_ID, Constants.NO_AUTHS);
 +      Key rowKey = new Key(metadataEntry);
 +      
 +      SortedMap<String,DataFileValue> origDatafileSizes = new TreeMap<String,DataFileValue>();
 +      SortedMap<String,DataFileValue> highDatafileSizes = new TreeMap<String,DataFileValue>();
 +      SortedMap<String,DataFileValue> lowDatafileSizes = new TreeMap<String,DataFileValue>();
 +      scanner3.fetchColumnFamily(Constants.METADATA_DATAFILE_COLUMN_FAMILY);
 +      scanner3.setRange(new Range(rowKey, rowKey.followingKey(PartialKey.ROW)));
 +      
 +      for (Entry<Key,Value> entry : scanner3) {
 +        if (entry.getKey().compareColumnFamily(Constants.METADATA_DATAFILE_COLUMN_FAMILY) == 0) {
 +          origDatafileSizes.put(entry.getKey().getColumnQualifier().toString(), new DataFileValue(entry.getValue().get()));
 +        }
 +      }
 +      
 +      splitDatafiles(table, metadataPrevEndRow, splitRatio, new HashMap<String,FileUtil.FileInfo>(), origDatafileSizes, lowDatafileSizes, highDatafileSizes,
 +          highDatafilesToRemove);
 +    
 +      MetadataTable.finishSplit(metadataEntry, highDatafileSizes, highDatafilesToRemove, credentials, lock);
 +      
 +      return new KeyExtent(metadataEntry, KeyExtent.encodePrevEndRow(metadataPrevEndRow));
 +    }
 +
 +
 +  }
 +  
 +  public static void splitDatafiles(Text table, Text midRow, double splitRatio, Map<String,FileUtil.FileInfo> firstAndLastRows,
 +      SortedMap<String,DataFileValue> datafiles, SortedMap<String,DataFileValue> lowDatafileSizes, SortedMap<String,DataFileValue> highDatafileSizes,
 +      List<String> highDatafilesToRemove) {
 +    
 +    for (Entry<String,DataFileValue> entry : datafiles.entrySet()) {
 +      
 +      Text firstRow = null;
 +      Text lastRow = null;
 +      
 +      boolean rowsKnown = false;
 +      
 +      FileUtil.FileInfo mfi = firstAndLastRows.get(entry.getKey());
 +      
 +      if (mfi != null) {
 +        firstRow = mfi.getFirstRow();
 +        lastRow = mfi.getLastRow();
 +        rowsKnown = true;
 +      }
 +      
 +      if (rowsKnown && firstRow.compareTo(midRow) > 0) {
 +        // only in high
 +        long highSize = entry.getValue().getSize();
 +        long highEntries = entry.getValue().getNumEntries();
 +        highDatafileSizes.put(entry.getKey(), new DataFileValue(highSize, highEntries, entry.getValue().getTime()));
 +      } else if (rowsKnown && lastRow.compareTo(midRow) <= 0) {
 +        // only in low
 +        long lowSize = entry.getValue().getSize();
 +        long lowEntries = entry.getValue().getNumEntries();
 +        lowDatafileSizes.put(entry.getKey(), new DataFileValue(lowSize, lowEntries, entry.getValue().getTime()));
 +        
 +        highDatafilesToRemove.add(entry.getKey());
 +      } else {
 +        long lowSize = (long) Math.floor((entry.getValue().getSize() * splitRatio));
 +        long lowEntries = (long) Math.floor((entry.getValue().getNumEntries() * splitRatio));
 +        lowDatafileSizes.put(entry.getKey(), new DataFileValue(lowSize, lowEntries, entry.getValue().getTime()));
 +        
 +        long highSize = (long) Math.ceil((entry.getValue().getSize() * (1.0 - splitRatio)));
 +        long highEntries = (long) Math.ceil((entry.getValue().getNumEntries() * (1.0 - splitRatio)));
 +        highDatafileSizes.put(entry.getKey(), new DataFileValue(highSize, highEntries, entry.getValue().getTime()));
 +      }
 +    }
 +  }
 +  
 +  public static KeyExtent fixSplit(Text metadataEntry, SortedMap<ColumnFQ,Value> columns, TServerInstance tserver, TCredentials credentials, ZooLock lock)
 +      throws AccumuloException {
 +    log.info("Incomplete split " + metadataEntry + " attempting to fix");
 +    
 +    Value oper = columns.get(Constants.METADATA_OLD_PREV_ROW_COLUMN);
 +    
 +    if (columns.get(Constants.METADATA_SPLIT_RATIO_COLUMN) == null) {
 +      throw new IllegalArgumentException("Metadata entry does not have split ratio (" + metadataEntry + ")");
 +    }
 +    
 +    double splitRatio = Double.parseDouble(new String(columns.get(Constants.METADATA_SPLIT_RATIO_COLUMN).get()));
 +    
 +    Value prevEndRowIBW = columns.get(Constants.METADATA_PREV_ROW_COLUMN);
 +    
 +    if (prevEndRowIBW == null) {
 +      throw new IllegalArgumentException("Metadata entry does not have prev row (" + metadataEntry + ")");
 +    }
 +    
 +    Value time = columns.get(Constants.METADATA_TIME_COLUMN);
 +    
 +    if (time == null) {
 +      throw new IllegalArgumentException("Metadata entry does not have time (" + metadataEntry + ")");
 +    }
 +    
 +    Value flushID = columns.get(Constants.METADATA_FLUSH_COLUMN);
 +    long initFlushID = -1;
 +    if (flushID != null)
 +      initFlushID = Long.parseLong(flushID.toString());
 +    
 +    Value compactID = columns.get(Constants.METADATA_COMPACT_COLUMN);
 +    long initCompactID = -1;
 +    if (compactID != null)
 +      initCompactID = Long.parseLong(compactID.toString());
 +    
 +    Text metadataPrevEndRow = KeyExtent.decodePrevEndRow(prevEndRowIBW);
 +    
 +    Text table = (new KeyExtent(metadataEntry, (Text) null)).getTableId();
 +    
 +    return fixSplit(table, metadataEntry, metadataPrevEndRow, oper, splitRatio, tserver, credentials, time.toString(), initFlushID, initCompactID, lock);
 +  }
 +  
 +  public static void deleteTable(String tableId, boolean insertDeletes, TCredentials credentials, ZooLock lock) throws AccumuloException {
 +    Scanner ms = new ScannerImpl(HdfsZooInstance.getInstance(), credentials, Constants.METADATA_TABLE_ID, Constants.NO_AUTHS);
 +    Text tableIdText = new Text(tableId);
 +    BatchWriter bw = new BatchWriterImpl(HdfsZooInstance.getInstance(), credentials, Constants.METADATA_TABLE_ID, new BatchWriterConfig().setMaxMemory(1000000)
 +        .setMaxLatency(120000l, TimeUnit.MILLISECONDS).setMaxWriteThreads(2));
 +    
 +    // scan metadata for our table and delete everything we find
 +    Mutation m = null;
 +    ms.setRange(new KeyExtent(tableIdText, null, null).toMetadataRange());
 +    
 +    // insert deletes before deleting data from !METADATA... this makes the code fault tolerant
 +    if (insertDeletes) {
 +      
 +      ms.fetchColumnFamily(Constants.METADATA_DATAFILE_COLUMN_FAMILY);
 +      Constants.METADATA_DIRECTORY_COLUMN.fetch(ms);
 +      
 +      for (Entry<Key,Value> cell : ms) {
 +        Key key = cell.getKey();
 +        
 +        if (key.getColumnFamily().equals(Constants.METADATA_DATAFILE_COLUMN_FAMILY)) {
 +          String relPath = key.getColumnQualifier().toString();
 +          // only insert deletes for files owned by this table
 +          if (!relPath.startsWith("../"))
 +            bw.addMutation(createDeleteMutation(tableId, relPath));
 +        }
 +        
 +        if (Constants.METADATA_DIRECTORY_COLUMN.hasColumns(key)) {
 +          bw.addMutation(createDeleteMutation(tableId, cell.getValue().toString()));
 +        }
 +      }
 +      
 +      bw.flush();
 +      
 +      ms.clearColumns();
 +    }
 +    
 +    for (Entry<Key,Value> cell : ms) {
 +      Key key = cell.getKey();
 +      
 +      if (m == null) {
 +        m = new Mutation(key.getRow());
 +        if (lock != null)
 +          putLockID(lock, m);
 +      }
 +      
 +      if (key.getRow().compareTo(m.getRow(), 0, m.getRow().length) != 0) {
 +        bw.addMutation(m);
 +        m = new Mutation(key.getRow());
 +        if (lock != null)
 +          putLockID(lock, m);
 +      }
 +      m.putDelete(key.getColumnFamily(), key.getColumnQualifier());
 +    }
 +    
 +    if (m != null)
 +      bw.addMutation(m);
 +    
 +    bw.close();
 +  }
 +  
 +  public static class LogEntry {
 +    public KeyExtent extent;
 +    public long timestamp;
 +    public String server;
 +    public String filename;
 +    public int tabletId;
 +    public Collection<String> logSet;
 +    
 +    @Override
 +    public String toString() {
 +      return extent.toString() + " " + filename + " (" + tabletId + ")";
 +    }
 +    
 +    public String getName() {
 +      return server + "/" + filename;
 +    }
 +    
 +    public byte[] toBytes() throws IOException {
 +      DataOutputBuffer out = new DataOutputBuffer();
 +      extent.write(out);
 +      out.writeLong(timestamp);
 +      out.writeUTF(server);
 +      out.writeUTF(filename);
 +      out.write(tabletId);
 +      out.write(logSet.size());
 +      for (String s : logSet) {
 +        out.writeUTF(s);
 +      }
 +      return Arrays.copyOf(out.getData(), out.getLength());
 +    }
 +    
 +    public void fromBytes(byte bytes[]) throws IOException {
 +      DataInputBuffer inp = new DataInputBuffer();
 +      inp.reset(bytes, bytes.length);
 +      extent = new KeyExtent();
 +      extent.readFields(inp);
 +      timestamp = inp.readLong();
 +      server = inp.readUTF();
 +      filename = inp.readUTF();
 +      tabletId = inp.read();
 +      int count = inp.read();
 +      ArrayList<String> logSet = new ArrayList<String>(count);
 +      for (int i = 0; i < count; i++)
 +        logSet.add(inp.readUTF());
 +      this.logSet = logSet;
 +    }
 +    
 +  }
 +  
 +  private static String getZookeeperLogLocation() {
 +    return ZooUtil.getRoot(HdfsZooInstance.getInstance()) + Constants.ZROOT_TABLET_WALOGS;
 +  }
 +  
 +  public static void addLogEntry(TCredentials credentials, LogEntry entry, ZooLock zooLock) {
 +    if (entry.extent.isRootTablet()) {
 +      String root = getZookeeperLogLocation();
 +      while (true) {
 +        try {
 +          IZooReaderWriter zoo = ZooReaderWriter.getInstance();
 +          if (zoo.isLockHeld(zooLock.getLockID()))
 +            zoo.putPersistentData(root + "/" + entry.filename, entry.toBytes(), NodeExistsPolicy.OVERWRITE);
 +          break;
 +        } catch (KeeperException e) {
 +          log.error(e, e);
 +        } catch (InterruptedException e) {
 +          log.error(e, e);
 +        } catch (IOException e) {
 +          log.error(e, e);
 +        }
 +        UtilWaitThread.sleep(1000);
 +      }
 +    } else {
 +      String value = StringUtil.join(entry.logSet, ";") + "|" + entry.tabletId;
 +      Mutation m = new Mutation(entry.extent.getMetadataEntry());
 +      m.put(Constants.METADATA_LOG_COLUMN_FAMILY, new Text(entry.server + "/" + entry.filename), new Value(value.getBytes()));
 +      update(credentials, zooLock, m);
 +    }
 +  }
 +  
 +  public static LogEntry entryFromKeyValue(Key key, Value value) {
 +    MetadataTable.LogEntry e = new MetadataTable.LogEntry();
 +    e.extent = new KeyExtent(key.getRow(), EMPTY_TEXT);
 +    String[] parts = key.getColumnQualifier().toString().split("/");
 +    e.server = parts[0];
 +    e.filename = parts[1];
 +    parts = value.toString().split("\\|");
 +    e.tabletId = Integer.parseInt(parts[1]);
 +    e.logSet = Arrays.asList(parts[0].split(";"));
 +    e.timestamp = key.getTimestamp();
 +    return e;
 +  }
 +  
 +  public static Pair<List<LogEntry>,SortedMap<String,DataFileValue>> getFileAndLogEntries(TCredentials credentials, KeyExtent extent) throws KeeperException,
 +      InterruptedException, IOException {
 +    ArrayList<LogEntry> result = new ArrayList<LogEntry>();
 +    TreeMap<String,DataFileValue> sizes = new TreeMap<String,DataFileValue>();
 +    
 +    if (extent.isRootTablet()) {
 +      getRootLogEntries(result);
 +      FileSystem fs = TraceFileSystem.wrap(FileUtil.getFileSystem(CachedConfiguration.getInstance(), ServerConfiguration.getSiteConfiguration()));
 +      FileStatus[] files = fs.listStatus(new Path(ServerConstants.getRootTabletDir()));
 +      
 +      for (FileStatus fileStatus : files) {
 +        if (fileStatus.getPath().toString().endsWith("_tmp")) {
 +          continue;
 +        }
 +        DataFileValue dfv = new DataFileValue(0, 0);
 +        sizes.put(Constants.ZROOT_TABLET + "/" + fileStatus.getPath().getName(), dfv);
 +      }
 +      
 +    } else {
 +      Scanner scanner = new ScannerImpl(HdfsZooInstance.getInstance(), credentials, Constants.METADATA_TABLE_ID, Constants.NO_AUTHS);
 +      scanner.fetchColumnFamily(Constants.METADATA_LOG_COLUMN_FAMILY);
 +      scanner.fetchColumnFamily(Constants.METADATA_DATAFILE_COLUMN_FAMILY);
 +      scanner.setRange(extent.toMetadataRange());
 +      
 +      for (Entry<Key,Value> entry : scanner) {
 +        if (!entry.getKey().getRow().equals(extent.getMetadataEntry())) {
 +          throw new RuntimeException("Unexpected row " + entry.getKey().getRow() + " expected " + extent.getMetadataEntry());
 +        }
 +        
 +        if (entry.getKey().getColumnFamily().equals(Constants.METADATA_LOG_COLUMN_FAMILY)) {
 +          result.add(entryFromKeyValue(entry.getKey(), entry.getValue()));
 +        } else if (entry.getKey().getColumnFamily().equals(Constants.METADATA_DATAFILE_COLUMN_FAMILY)) {
 +          DataFileValue dfv = new DataFileValue(entry.getValue().get());
 +          sizes.put(entry.getKey().getColumnQualifier().toString(), dfv);
 +        } else {
 +          throw new RuntimeException("Unexpected col fam " + entry.getKey().getColumnFamily());
 +        }
 +      }
 +    }
 +    
 +    return new Pair<List<LogEntry>,SortedMap<String,DataFileValue>>(result, sizes);
 +  }
 +  
 +  public static List<LogEntry> getLogEntries(TCredentials credentials, KeyExtent extent) throws IOException, KeeperException, InterruptedException {
 +    log.info("Scanning logging entries for " + extent);
 +    ArrayList<LogEntry> result = new ArrayList<LogEntry>();
 +    if (extent.equals(Constants.ROOT_TABLET_EXTENT)) {
 +      log.info("Getting logs for root tablet from zookeeper");
 +      getRootLogEntries(result);
 +    } else {
 +      log.info("Scanning metadata for logs used for tablet " + extent);
 +      Scanner scanner = getTabletLogScanner(credentials, extent);
 +      Text pattern = extent.getMetadataEntry();
 +      for (Entry<Key,Value> entry : scanner) {
 +        Text row = entry.getKey().getRow();
 +        if (entry.getKey().getColumnFamily().equals(Constants.METADATA_LOG_COLUMN_FAMILY)) {
 +          if (row.equals(pattern)) {
 +            result.add(entryFromKeyValue(entry.getKey(), entry.getValue()));
 +          }
 +        }
 +      }
 +    }
 +    
 +    Collections.sort(result, new Comparator<LogEntry>() {
 +      @Override
 +      public int compare(LogEntry o1, LogEntry o2) {
 +        long diff = o1.timestamp - o2.timestamp;
 +        if (diff < 0)
 +          return -1;
 +        if (diff > 0)
 +          return 1;
 +        return 0;
 +      }
 +    });
 +    log.info("Returning logs " + result + " for extent " + extent);
 +    return result;
 +  }
 +  
 +  private static void getRootLogEntries(ArrayList<LogEntry> result) throws KeeperException, InterruptedException, IOException {
 +    IZooReaderWriter zoo = ZooReaderWriter.getInstance();
 +    String root = getZookeeperLogLocation();
 +    // there's a little race between getting the children and fetching 
 +    // the data.  The log can be removed in between.
 +    while (true) {
 +      result.clear();
 +      for (String child : zoo.getChildren(root)) {
 +        LogEntry e = new LogEntry();
 +        try {
 +          e.fromBytes(zoo.getData(root + "/" + child, null));
 +          result.add(e);
 +        } catch (KeeperException.NoNodeException ex) {
 +          continue;
 +        }
 +      }
 +      break;
 +    }
 +  }
 +  
 +  private static Scanner getTabletLogScanner(TCredentials credentials, KeyExtent extent) {
 +    Scanner scanner = new ScannerImpl(HdfsZooInstance.getInstance(), credentials, Constants.METADATA_TABLE_ID, Constants.NO_AUTHS);
 +    scanner.fetchColumnFamily(Constants.METADATA_LOG_COLUMN_FAMILY);
 +    Text start = extent.getMetadataEntry();
 +    Key endKey = new Key(start, Constants.METADATA_LOG_COLUMN_FAMILY);
 +    endKey = endKey.followingKey(PartialKey.ROW_COLFAM);
 +    scanner.setRange(new Range(new Key(start), endKey));
 +    return scanner;
 +  }
 +  
 +  static class LogEntryIterator implements Iterator<LogEntry> {
 +    
 +    Iterator<LogEntry> rootTabletEntries = null;
 +    Iterator<Entry<Key,Value>> metadataEntries = null;
 +    
 +    LogEntryIterator(TCredentials creds) throws IOException, KeeperException, InterruptedException {
 +      rootTabletEntries = getLogEntries(creds, Constants.ROOT_TABLET_EXTENT).iterator();
 +      try {
 +        Scanner scanner = HdfsZooInstance.getInstance().getConnector(creds.getPrincipal(), CredentialHelper.extractToken(creds))
 +            .createScanner(Constants.METADATA_TABLE_NAME, Constants.NO_AUTHS);
 +        scanner.fetchColumnFamily(Constants.METADATA_LOG_COLUMN_FAMILY);
 +        metadataEntries = scanner.iterator();
 +      } catch (Exception ex) {
 +        throw new IOException(ex);
 +      }
 +    }
 +    
 +    @Override
 +    public boolean hasNext() {
 +      return rootTabletEntries.hasNext() || metadataEntries.hasNext();
 +    }
 +    
 +    @Override
 +    public LogEntry next() {
 +      if (rootTabletEntries.hasNext()) {
 +        return rootTabletEntries.next();
 +      }
 +      Entry<Key,Value> entry = metadataEntries.next();
 +      return entryFromKeyValue(entry.getKey(), entry.getValue());
 +    }
 +    
 +    @Override
 +    public void remove() {
 +      throw new UnsupportedOperationException();
 +    }
 +  }
 +  
 +  public static Iterator<LogEntry> getLogEntries(TCredentials creds) throws IOException, KeeperException, InterruptedException {
 +    return new LogEntryIterator(creds);
 +  }
 +  
 +  public static void removeUnusedWALEntries(KeyExtent extent, List<LogEntry> logEntries, ZooLock zooLock) {
-     for (LogEntry entry : logEntries) {
-       if (entry.extent.isRootTablet()) {
++      if (extent.isRootTablet()) {
++        for (LogEntry entry : logEntries) {
 +        String root = getZookeeperLogLocation();
 +        while (true) {
 +          try {
 +            IZooReaderWriter zoo = ZooReaderWriter.getInstance();
 +            if (zoo.isLockHeld(zooLock.getLockID()))
 +              zoo.recursiveDelete(root + "/" + entry.filename, NodeMissingPolicy.SKIP);
 +            break;
 +          } catch (Exception e) {
 +            log.error(e, e);
 +          }
 +          UtilWaitThread.sleep(1000);
 +        }
++        }
 +      } else {
-         Mutation m = new Mutation(entry.extent.getMetadataEntry());
-         m.putDelete(Constants.METADATA_LOG_COLUMN_FAMILY, new Text(entry.server + "/" + entry.filename));
++        Mutation m = new Mutation(extent.getMetadataEntry());
++        for (LogEntry entry : logEntries) {
++          m.putDelete(Constants.METADATA_LOG_COLUMN_FAMILY, new Text(entry.server + "/" + entry.filename));
++        }
 +        update(SecurityConstants.getSystemCredentials(), zooLock, m);
-       }
 +    }
 +  }
 +  
 +  private static void getFiles(Set<String> files, Map<Key,Value> tablet, String srcTableId) {
 +    for (Entry<Key,Value> entry : tablet.entrySet()) {
 +      if (entry.getKey().getColumnFamily().equals(Constants.METADATA_DATAFILE_COLUMN_FAMILY)) {
 +        String cf = entry.getKey().getColumnQualifier().toString();
 +        if (srcTableId != null && !cf.startsWith("../"))
 +          cf = "../" + srcTableId + entry.getKey().getColumnQualifier();
 +        files.add(cf);
 +      }
 +    }
 +  }
 +  
 +  private static Mutation createCloneMutation(String srcTableId, String tableId, Map<Key,Value> tablet) {
 +    
 +    KeyExtent ke = new KeyExtent(tablet.keySet().iterator().next().getRow(), (Text) null);
 +    Mutation m = new Mutation(KeyExtent.getMetadataEntry(new Text(tableId), ke.getEndRow()));
 +    
 +    for (Entry<Key,Value> entry : tablet.entrySet()) {
 +      if (entry.getKey().getColumnFamily().equals(Constants.METADATA_DATAFILE_COLUMN_FAMILY)) {
 +        String cf = entry.getKey().getColumnQualifier().toString();
 +        if (!cf.startsWith("../"))
 +          cf = "../" + srcTableId + entry.getKey().getColumnQualifier();
 +        m.put(entry.getKey().getColumnFamily(), new Text(cf), entry.getValue());
 +      } else if (entry.getKey().getColumnFamily().equals(Constants.METADATA_CURRENT_LOCATION_COLUMN_FAMILY)) {
 +        m.put(Constants.METADATA_LAST_LOCATION_COLUMN_FAMILY, entry.getKey().getColumnQualifier(), entry.getValue());
 +      } else if (entry.getKey().getColumnFamily().equals(Constants.METADATA_LAST_LOCATION_COLUMN_FAMILY)) {
 +        // skip
 +      } else {
 +        m.put(entry.getKey().getColumnFamily(), entry.getKey().getColumnQualifier(), entry.getValue());
 +      }
 +    }
 +    return m;
 +  }
 +  
 +  private static Scanner createCloneScanner(String tableId, Connector conn) throws TableNotFoundException {
 +    Scanner mscanner = new IsolatedScanner(conn.createScanner(Constants.METADATA_TABLE_NAME, Constants.NO_AUTHS));
 +    mscanner.setRange(new KeyExtent(new Text(tableId), null, null).toMetadataRange());
 +    mscanner.fetchColumnFamily(Constants.METADATA_DATAFILE_COLUMN_FAMILY);
 +    mscanner.fetchColumnFamily(Constants.METADATA_CURRENT_LOCATION_COLUMN_FAMILY);
 +    mscanner.fetchColumnFamily(Constants.METADATA_LAST_LOCATION_COLUMN_FAMILY);
 +    mscanner.fetchColumnFamily(Constants.METADATA_CLONED_COLUMN_FAMILY);
 +    Constants.METADATA_PREV_ROW_COLUMN.fetch(mscanner);
 +    Constants.METADATA_TIME_COLUMN.fetch(mscanner);
 +    return mscanner;
 +  }
 +  
 +  static void initializeClone(String srcTableId, String tableId, Connector conn, BatchWriter bw) throws TableNotFoundException, MutationsRejectedException {
 +    TabletIterator ti = new TabletIterator(createCloneScanner(srcTableId, conn), new KeyExtent(new Text(srcTableId), null, null).toMetadataRange(), true, true);
 +    
 +    if (!ti.hasNext())
 +      throw new RuntimeException(" table deleted during clone?  srcTableId = " + srcTableId);
 +    
 +    while (ti.hasNext())
 +      bw.addMutation(createCloneMutation(srcTableId, tableId, ti.next()));
 +    
 +    bw.flush();
 +  }
 +  
 +  static int compareEndRows(Text endRow1, Text endRow2) {
 +    return new KeyExtent(new Text("0"), endRow1, null).compareTo(new KeyExtent(new Text("0"), endRow2, null));
 +  }
 +  
 +  static int checkClone(String srcTableId, String tableId, Connector conn, BatchWriter bw) throws TableNotFoundException, MutationsRejectedException {
 +    TabletIterator srcIter = new TabletIterator(createCloneScanner(srcTableId, conn), new KeyExtent(new Text(srcTableId), null, null).toMetadataRange(), true,
 +        true);
 +    TabletIterator cloneIter = new TabletIterator(createCloneScanner(tableId, conn), new KeyExtent(new Text(tableId), null, null).toMetadataRange(), true, true);
 +    
 +    if (!cloneIter.hasNext() || !srcIter.hasNext())
 +      throw new RuntimeException(" table deleted during clone?  srcTableId = " + srcTableId + " tableId=" + tableId);
 +    
 +    int rewrites = 0;
 +    
 +    while (cloneIter.hasNext()) {
 +      Map<Key,Value> cloneTablet = cloneIter.next();
 +      Text cloneEndRow = new KeyExtent(cloneTablet.keySet().iterator().next().getRow(), (Text) null).getEndRow();
 +      HashSet<String> cloneFiles = new HashSet<String>();
 +      
 +      boolean cloneSuccessful = false;
 +      for (Entry<Key,Value> entry : cloneTablet.entrySet()) {
 +        if (entry.getKey().getColumnFamily().equals(Constants.METADATA_CLONED_COLUMN_FAMILY)) {
 +          cloneSuccessful = true;
 +          break;
 +        }
 +      }
 +      
 +      if (!cloneSuccessful)
 +        getFiles(cloneFiles, cloneTablet, null);
 +      
 +      List<Map<Key,Value>> srcTablets = new ArrayList<Map<Key,Value>>();
 +      Map<Key,Value> srcTablet = srcIter.next();
 +      srcTablets.add(srcTablet);
 +      
 +      Text srcEndRow = new KeyExtent(srcTablet.keySet().iterator().next().getRow(), (Text) null).getEndRow();
 +      
 +      int cmp = compareEndRows(cloneEndRow, srcEndRow);
 +      if (cmp < 0)
 +        throw new TabletIterator.TabletDeletedException("Tablets deleted from src during clone : " + cloneEndRow + " " + srcEndRow);
 +      
 +      HashSet<String> srcFiles = new HashSet<String>();
 +      if (!cloneSuccessful)
 +        getFiles(srcFiles, srcTablet, srcTableId);
 +      
 +      while (cmp > 0) {
 +        srcTablet = srcIter.next();
 +        srcTablets.add(srcTablet);
 +        srcEndRow = new KeyExtent(srcTablet.keySet().iterator().next().getRow(), (Text) null).getEndRow();
 +        cmp = compareEndRows(cloneEndRow, srcEndRow);
 +        if (cmp < 0)
 +          throw new TabletIterator.TabletDeletedException("Tablets deleted from src during clone : " + cloneEndRow + " " + srcEndRow);
 +        
 +        if (!cloneSuccessful)
 +          getFiles(srcFiles, srcTablet, srcTableId);
 +      }
 +      
 +      if (cloneSuccessful)
 +        continue;
 +      
 +      if (!srcFiles.containsAll(cloneFiles)) {
 +        // delete existing cloned tablet entry
 +        Mutation m = new Mutation(cloneTablet.keySet().iterator().next().getRow());
 +        
 +        for (Entry<Key,Value> entry : cloneTablet.entrySet()) {
 +          Key k = entry.getKey();
 +          m.putDelete(k.getColumnFamily(), k.getColumnQualifier(), k.getTimestamp());
 +        }
 +        
 +        bw.addMutation(m);
 +        
 +        for (Map<Key,Value> st : srcTablets)
 +          bw.addMutation(createCloneMutation(srcTableId, tableId, st));
 +        
 +        rewrites++;
 +      } else {
 +        // write out marker that this tablet was successfully cloned
 +        Mutation m = new Mutation(cloneTablet.keySet().iterator().next().getRow());
 +        m.put(Constants.METADATA_CLONED_COLUMN_FAMILY, new Text(""), new Value("OK".getBytes()));
 +        bw.addMutation(m);
 +      }
 +    }
 +    
 +    bw.flush();
 +    return rewrites;
 +  }
 +  
 +  public static void cloneTable(Instance instance, String srcTableId, String tableId) throws Exception {
 +    
 +    Connector conn = instance.getConnector(SecurityConstants.SYSTEM_PRINCIPAL, SecurityConstants.getSystemToken());
 +    BatchWriter bw = conn.createBatchWriter(Constants.METADATA_TABLE_NAME, new BatchWriterConfig());
 +    
 +    while (true) {
 +      
 +      try {
 +        initializeClone(srcTableId, tableId, conn, bw);
 +        
 +        // the following loop looks changes in the file that occurred during the copy.. if files were dereferenced then they could have been GCed
 +        
 +        while (true) {
 +          int rewrites = checkClone(srcTableId, tableId, conn, bw);
 +          
 +          if (rewrites == 0)
 +            break;
 +        }
 +        
 +        bw.flush();
 +        break;
 +        
 +      } catch (TabletIterator.TabletDeletedException tde) {
 +        // tablets were merged in the src table
 +        bw.flush();
 +        
 +        // delete what we have cloned and try again
 +        deleteTable(tableId, false, SecurityConstants.getSystemCredentials(), null);
 +        
 +        log.debug("Tablets merged in table " + srcTableId + " while attempting to clone, trying again");
 +        
 +        UtilWaitThread.sleep(100);
 +      }
 +    }
 +    
 +    // delete the clone markers and create directory entries
 +    Scanner mscanner = conn.createScanner(Constants.METADATA_TABLE_NAME, Constants.NO_AUTHS);
 +    mscanner.setRange(new KeyExtent(new Text(tableId), null, null).toMetadataRange());
 +    mscanner.fetchColumnFamily(Constants.METADATA_CLONED_COLUMN_FAMILY);
 +    
 +    int dirCount = 0;
 +    
 +    for (Entry<Key,Value> entry : mscanner) {
 +      Key k = entry.getKey();
 +      Mutation m = new Mutation(k.getRow());
 +      m.putDelete(k.getColumnFamily(), k.getColumnQualifier());
 +      Constants.METADATA_DIRECTORY_COLUMN.put(m, new Value(FastFormat.toZeroPaddedString(dirCount++, 8, 16, "/c-".getBytes())));
 +      bw.addMutation(m);
 +    }
 +    
 +    bw.close();
 +    
 +  }
 +  
 +  public static void chopped(KeyExtent extent, ZooLock zooLock) {
 +    Mutation m = new Mutation(extent.getMetadataEntry());
 +    Constants.METADATA_CHOPPED_COLUMN.put(m, new Value("chopped".getBytes()));
 +    update(SecurityConstants.getSystemCredentials(), zooLock, m);
 +  }
 +  
 +  public static void removeBulkLoadEntries(Connector conn, String tableId, long tid) throws Exception {
 +    Scanner mscanner = new IsolatedScanner(conn.createScanner(Constants.METADATA_TABLE_NAME, Constants.NO_AUTHS));
 +    mscanner.setRange(new KeyExtent(new Text(tableId), null, null).toMetadataRange());
 +    mscanner.fetchColumnFamily(Constants.METADATA_BULKFILE_COLUMN_FAMILY);
 +    BatchWriter bw = conn.createBatchWriter(Constants.METADATA_TABLE_NAME, new BatchWriterConfig());
 +    for (Entry<Key,Value> entry : mscanner) {
 +      log.debug("Looking at entry " + entry + " with tid " + tid);
 +      if (Long.parseLong(entry.getValue().toString()) == tid) {
 +        log.debug("deleting entry " + entry);
 +        Mutation m = new Mutation(entry.getKey().getRow());
 +        m.putDelete(entry.getKey().getColumnFamily(), entry.getKey().getColumnQualifier());
 +        bw.addMutation(m);
 +      }
 +    }
 +    bw.close();
 +  }
 +  
 +  public static List<String> getBulkFilesLoaded(Connector conn, KeyExtent extent, long tid) {
 +    List<String> result = new ArrayList<String>();
 +    try {
 +      Scanner mscanner = new IsolatedScanner(conn.createScanner(Constants.METADATA_TABLE_NAME, Constants.NO_AUTHS));
 +      mscanner.setRange(extent.toMetadataRange());
 +      mscanner.fetchColumnFamily(Constants.METADATA_BULKFILE_COLUMN_FAMILY);
 +      for (Entry<Key,Value> entry : mscanner) {
 +        if (Long.parseLong(entry.getValue().toString()) == tid) {
 +          result.add(entry.getKey().getColumnQualifier().toString());
 +        }
 +      }
 +      return result;
 +    } catch (TableNotFoundException ex) {
 +      // unlikely
 +      throw new RuntimeException("Onos! teh metadata table has vanished!!");
 +    }
 +  }
 +  
 +  public static Map<String,Long> getBulkFilesLoaded(TCredentials credentials, KeyExtent extent) {
 +    return getBulkFilesLoaded(credentials, extent.getMetadataEntry());
 +  }
 +  
 +  public static Map<String,Long> getBulkFilesLoaded(TCredentials credentials, Text metadataRow) {
 +    
 +    Map<String,Long> ret = new HashMap<String,Long>();
 +    
 +    Scanner scanner = new ScannerImpl(HdfsZooInstance.getInstance(), credentials, Constants.METADATA_TABLE_ID, Constants.NO_AUTHS);
 +    scanner.setRange(new Range(metadataRow));
 +    scanner.fetchColumnFamily(Constants.METADATA_BULKFILE_COLUMN_FAMILY);
 +    for (Entry<Key,Value> entry : scanner) {
 +      String file = entry.getKey().getColumnQualifier().toString();
 +      Long tid = Long.parseLong(entry.getValue().toString());
 +      
 +      ret.put(file, tid);
 +    }
 +    return ret;
 +  }
 +  
 +  public static void addBulkLoadInProgressFlag(String path) {
 +    
 +    Mutation m = new Mutation(Constants.METADATA_BLIP_FLAG_PREFIX + path);
 +    m.put(EMPTY_TEXT, EMPTY_TEXT, new Value(new byte[] {}));
 +    
 +    update(SecurityConstants.getSystemCredentials(), m);
 +  }
 +  
 +  public static void removeBulkLoadInProgressFlag(String path) {
 +    
 +    Mutation m = new Mutation(Constants.METADATA_BLIP_FLAG_PREFIX + path);
 +    m.putDelete(EMPTY_TEXT, EMPTY_TEXT);
 +    
 +    update(SecurityConstants.getSystemCredentials(), m);
 +  }
 +
 +  public static void moveMetaDeleteMarkers(Instance instance, TCredentials creds) {
 +    // move delete markers from the normal delete keyspace to the root tablet delete keyspace if the files are for the !METADATA table
 +    Scanner scanner = new ScannerImpl(instance, creds, Constants.METADATA_TABLE_ID, Constants.NO_AUTHS);
 +    scanner.setRange(new Range(Constants.METADATA_DELETES_KEYSPACE));
 +    for (Entry<Key,Value> entry : scanner) {
 +      String row = entry.getKey().getRow().toString();
 +      if (row.startsWith(Constants.METADATA_DELETE_FLAG_PREFIX + "/" + Constants.METADATA_TABLE_ID)) {
 +        String filename = row.substring(Constants.METADATA_DELETE_FLAG_PREFIX.length());
 +        // add the new entry first
 +        log.info("Moving " + filename + " marker to the root tablet");
 +        Mutation m = new Mutation(Constants.METADATA_DELETE_FLAG_FOR_METADATA_PREFIX + filename);
 +        m.put(new byte[]{}, new byte[]{}, new byte[]{});
 +        update(creds, m);
 +        // remove the old entry
 +        m = new Mutation(entry.getKey().getRow());
 +        m.putDelete(new byte[]{}, new byte[]{});
 +        update(creds, m);
 +      } else {
 +        break;
 +      }
 +    }
 +    
 +  }
 +}


[8/8] git commit: Merge branch '1.6.0-SNAPSHOT'

Posted by ec...@apache.org.
Merge branch '1.6.0-SNAPSHOT'


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

Branch: refs/heads/master
Commit: 9b41155822f69d145317fb63d6a4088a56858eba
Parents: 1fba05a fe46a60
Author: Eric Newton <er...@gmail.com>
Authored: Thu Nov 21 12:45:31 2013 -0500
Committer: Eric Newton <er...@gmail.com>
Committed: Thu Nov 21 12:45:31 2013 -0500

----------------------------------------------------------------------
 .../apache/accumulo/core/client/Instance.java   |  1 +
 .../client/admin/InstanceOperationsImpl.java    |  7 +-
 .../core/client/admin/TableOperationsImpl.java  |  5 +-
 .../core/client/impl/ConditionalWriterImpl.java |  6 +-
 .../accumulo/core/client/impl/MasterClient.java |  2 +-
 .../core/client/impl/OfflineScanner.java        |  4 +-
 .../core/client/impl/ScannerIterator.java       |  2 +-
 .../accumulo/core/client/impl/ServerClient.java |  4 +-
 .../client/impl/ServerConfigurationFactory.java | 30 ++++++
 .../impl/TabletServerBatchReaderIterator.java   |  2 +-
 .../client/impl/TabletServerBatchWriter.java    |  6 +-
 .../accumulo/core/client/impl/Writer.java       |  2 +-
 .../core/metadata/MetadataLocationObtainer.java |  7 +-
 .../accumulo/server/client/BulkImporter.java    |  9 +-
 .../accumulo/server/util/MetadataTableUtil.java | 99 +++-----------------
 .../gc/GarbageCollectWriteAheadLogs.java        |  3 +-
 .../accumulo/gc/SimpleGarbageCollector.java     | 13 +--
 .../java/org/apache/accumulo/master/Master.java |  3 +-
 18 files changed, 84 insertions(+), 121 deletions(-)
----------------------------------------------------------------------



[5/8] git commit: ACCUMULO-1914 make WALog cleanup after recovery an atomic mutation

Posted by ec...@apache.org.
ACCUMULO-1914 make WALog cleanup after recovery an atomic mutation


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

Branch: refs/heads/master
Commit: 041270b778db9345c5276ef91fb4f95f3e9e0bbe
Parents: 404e955 8bd6e0e
Author: Eric Newton <er...@gmail.com>
Authored: Thu Nov 21 12:34:59 2013 -0500
Committer: Eric Newton <er...@gmail.com>
Committed: Thu Nov 21 12:34:59 2013 -0500

----------------------------------------------------------------------
 .../org/apache/accumulo/server/monitor/Monitor.java | 16 ++++------------
 .../apache/accumulo/server/util/MetadataTable.java  | 12 +++++++-----
 2 files changed, 11 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/041270b7/server/src/main/java/org/apache/accumulo/server/monitor/Monitor.java
----------------------------------------------------------------------
diff --cc server/src/main/java/org/apache/accumulo/server/monitor/Monitor.java
index 8602c4c,0000000..8236b5f
mode 100644,000000..100644
--- a/server/src/main/java/org/apache/accumulo/server/monitor/Monitor.java
+++ b/server/src/main/java/org/apache/accumulo/server/monitor/Monitor.java
@@@ -1,667 -1,0 +1,659 @@@
 +/*
 + * 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.monitor;
 +
 +import java.net.InetSocketAddress;
 +import java.util.ArrayList;
 +import java.util.Collections;
 +import java.util.HashMap;
 +import java.util.HashSet;
 +import java.util.LinkedList;
 +import java.util.List;
 +import java.util.Map;
 +import java.util.Map.Entry;
 +import java.util.Set;
 +
 +import org.apache.accumulo.trace.instrument.Tracer;
 +import org.apache.accumulo.core.Constants;
 +import org.apache.accumulo.core.client.Instance;
 +import org.apache.accumulo.core.client.impl.MasterClient;
 +import org.apache.accumulo.core.conf.AccumuloConfiguration;
 +import org.apache.accumulo.core.conf.Property;
 +import org.apache.accumulo.core.file.FileUtil;
 +import org.apache.accumulo.core.gc.thrift.GCMonitorService;
 +import org.apache.accumulo.core.gc.thrift.GCStatus;
 +import org.apache.accumulo.core.master.thrift.Compacting;
 +import org.apache.accumulo.core.master.thrift.MasterClientService;
 +import org.apache.accumulo.core.master.thrift.MasterMonitorInfo;
 +import org.apache.accumulo.core.master.thrift.TableInfo;
 +import org.apache.accumulo.core.master.thrift.TabletServerStatus;
 +import org.apache.accumulo.core.security.SecurityUtil;
 +import org.apache.accumulo.core.util.CachedConfiguration;
 +import org.apache.accumulo.core.util.Daemon;
 +import org.apache.accumulo.core.util.LoggingRunnable;
 +import org.apache.accumulo.core.util.Pair;
 +import org.apache.accumulo.core.util.ServerServices;
 +import org.apache.accumulo.core.util.ServerServices.Service;
 +import org.apache.accumulo.core.util.ThriftUtil;
 +import org.apache.accumulo.core.util.UtilWaitThread;
 +import org.apache.accumulo.core.zookeeper.ZooUtil;
 +import org.apache.accumulo.server.Accumulo;
 +import org.apache.accumulo.server.client.HdfsZooInstance;
 +import org.apache.accumulo.server.conf.ServerConfiguration;
 +import org.apache.accumulo.server.monitor.servlets.DefaultServlet;
 +import org.apache.accumulo.server.monitor.servlets.GcStatusServlet;
 +import org.apache.accumulo.server.monitor.servlets.JSONServlet;
 +import org.apache.accumulo.server.monitor.servlets.LogServlet;
 +import org.apache.accumulo.server.monitor.servlets.MasterServlet;
 +import org.apache.accumulo.server.monitor.servlets.OperationServlet;
 +import org.apache.accumulo.server.monitor.servlets.ProblemServlet;
 +import org.apache.accumulo.server.monitor.servlets.ShellServlet;
 +import org.apache.accumulo.server.monitor.servlets.TServersServlet;
 +import org.apache.accumulo.server.monitor.servlets.TablesServlet;
 +import org.apache.accumulo.server.monitor.servlets.VisServlet;
 +import org.apache.accumulo.server.monitor.servlets.XMLServlet;
 +import org.apache.accumulo.server.monitor.servlets.trace.ListType;
 +import org.apache.accumulo.server.monitor.servlets.trace.ShowTrace;
 +import org.apache.accumulo.server.monitor.servlets.trace.Summary;
 +import org.apache.accumulo.server.problems.ProblemReports;
 +import org.apache.accumulo.server.problems.ProblemType;
 +import org.apache.accumulo.server.security.SecurityConstants;
 +import org.apache.accumulo.server.util.EmbeddedWebServer;
++import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 +import org.apache.hadoop.fs.FileSystem;
 +import org.apache.log4j.Logger;
- import org.apache.zookeeper.WatchedEvent;
- import org.apache.zookeeper.Watcher;
- import org.apache.zookeeper.ZooKeeper;
 +
 +/**
 + * Serve master statistics with an embedded web server.
 + */
 +public class Monitor {
 +  private static final Logger log = Logger.getLogger(Monitor.class);
 +  
 +  public static final int REFRESH_TIME = 5;
 +  private static long lastRecalc = 0L;
 +  private static double totalIngestRate = 0.0;
 +  private static double totalIngestByteRate = 0.0;
 +  private static double totalQueryRate = 0.0;
 +  private static double totalScanRate = 0.0;
 +  private static double totalQueryByteRate = 0.0;
 +  private static long totalEntries = 0L;
 +  private static int totalTabletCount = 0;
 +  private static int onlineTabletCount = 0;
 +  private static long totalHoldTime = 0;
 +  private static long totalLookups = 0;
 +  private static int totalTables = 0;
 +  
 +  private static class MaxList<T> extends LinkedList<Pair<Long,T>> {
 +    private static final long serialVersionUID = 1L;
 +    
 +    private long maxDelta;
 +    
 +    public MaxList(long maxDelta) {
 +      this.maxDelta = maxDelta;
 +    }
 +    
 +    @Override
 +    public boolean add(Pair<Long,T> obj) {
 +      boolean result = super.add(obj);
 +      
 +      if (obj.getFirst() - get(0).getFirst() > maxDelta)
 +        remove(0);
 +      
 +      return result;
 +    }
 +    
 +  }
 +  
 +  private static final int MAX_TIME_PERIOD = 60 * 60 * 1000;
 +  private static final List<Pair<Long,Double>> loadOverTime = Collections.synchronizedList(new MaxList<Double>(MAX_TIME_PERIOD));
 +  private static final List<Pair<Long,Double>> ingestRateOverTime = Collections.synchronizedList(new MaxList<Double>(MAX_TIME_PERIOD));
 +  private static final List<Pair<Long,Double>> ingestByteRateOverTime = Collections.synchronizedList(new MaxList<Double>(MAX_TIME_PERIOD));
 +  private static final List<Pair<Long,Integer>> recoveriesOverTime = Collections.synchronizedList(new MaxList<Integer>(MAX_TIME_PERIOD));
 +  private static final List<Pair<Long,Integer>> minorCompactionsOverTime = Collections.synchronizedList(new MaxList<Integer>(MAX_TIME_PERIOD));
 +  private static final List<Pair<Long,Integer>> majorCompactionsOverTime = Collections.synchronizedList(new MaxList<Integer>(MAX_TIME_PERIOD));
 +  private static final List<Pair<Long,Double>> lookupsOverTime = Collections.synchronizedList(new MaxList<Double>(MAX_TIME_PERIOD));
 +  private static final List<Pair<Long,Integer>> queryRateOverTime = Collections.synchronizedList(new MaxList<Integer>(MAX_TIME_PERIOD));
 +  private static final List<Pair<Long,Integer>> scanRateOverTime = Collections.synchronizedList(new MaxList<Integer>(MAX_TIME_PERIOD));
 +  private static final List<Pair<Long,Double>> queryByteRateOverTime = Collections.synchronizedList(new MaxList<Double>(MAX_TIME_PERIOD));
 +  private static final List<Pair<Long,Double>> indexCacheHitRateOverTime = Collections.synchronizedList(new MaxList<Double>(MAX_TIME_PERIOD));
 +  private static final List<Pair<Long,Double>> dataCacheHitRateOverTime = Collections.synchronizedList(new MaxList<Double>(MAX_TIME_PERIOD));
 +  private static EventCounter lookupRateTracker = new EventCounter();
 +  private static EventCounter indexCacheHitTracker = new EventCounter();
 +  private static EventCounter indexCacheRequestTracker = new EventCounter();
 +  private static EventCounter dataCacheHitTracker = new EventCounter();
 +  private static EventCounter dataCacheRequestTracker = new EventCounter();
 +  
 +  private static volatile boolean fetching = false;
 +  private static MasterMonitorInfo mmi;
 +  private static Map<String,Map<ProblemType,Integer>> problemSummary = Collections.emptyMap();
 +  private static Exception problemException;
 +  private static GCStatus gcStatus;
 +  
 +  private static Instance instance;
 +  
 +  private static ServerConfiguration config;
 +  
 +  private static EmbeddedWebServer server;
 +  
 +  public static Map<String,Double> summarizeTableStats(MasterMonitorInfo mmi) {
 +    Map<String,Double> compactingByTable = new HashMap<String,Double>();
 +    if (mmi != null && mmi.tServerInfo != null) {
 +      for (TabletServerStatus status : mmi.tServerInfo) {
 +        if (status != null && status.tableMap != null) {
 +          for (String table : status.tableMap.keySet()) {
 +            Double holdTime = compactingByTable.get(table);
 +            compactingByTable.put(table, Math.max(holdTime == null ? 0. : holdTime.doubleValue(), status.holdTime));
 +          }
 +        }
 +      }
 +    }
 +    return compactingByTable;
 +  }
 +  
 +  public static void add(TableInfo total, TableInfo more) {
 +    if (total.minors == null)
 +      total.minors = new Compacting();
 +    if (total.majors == null)
 +      total.majors = new Compacting();
 +    if (total.scans == null)
 +      total.scans = new Compacting();
 +    if (more.minors != null) {
 +      total.minors.running += more.minors.running;
 +      total.minors.queued += more.minors.queued;
 +    }
 +    if (more.majors != null) {
 +      total.majors.running += more.majors.running;
 +      total.majors.queued += more.majors.queued;
 +    }
 +    if (more.scans != null) {
 +      total.scans.running += more.scans.running;
 +      total.scans.queued += more.scans.queued;
 +    }
 +    total.onlineTablets += more.onlineTablets;
 +    total.recs += more.recs;
 +    total.recsInMemory += more.recsInMemory;
 +    total.tablets += more.tablets;
 +    total.ingestRate += more.ingestRate;
 +    total.ingestByteRate += more.ingestByteRate;
 +    total.queryRate += more.queryRate;
 +    total.queryByteRate += more.queryByteRate;
 +    total.scanRate += more.scanRate;
 +  }
 +  
 +  public static TableInfo summarizeTableStats(TabletServerStatus status) {
 +    TableInfo summary = new TableInfo();
 +    summary.majors = new Compacting();
 +    summary.minors = new Compacting();
 +    summary.scans = new Compacting();
 +    for (TableInfo rates : status.tableMap.values()) {
 +      add(summary, rates);
 +    }
 +    return summary;
 +  }
 +  
 +  private static class EventCounter {
 +    
 +    Map<String,Pair<Long,Long>> prevSamples = new HashMap<String,Pair<Long,Long>>();
 +    Map<String,Pair<Long,Long>> samples = new HashMap<String,Pair<Long,Long>>();
 +    Set<String> serversUpdated = new HashSet<String>();
 +    
 +    void startingUpdates() {
 +      serversUpdated.clear();
 +    }
 +    
 +    void updateTabletServer(String name, long sampleTime, long numEvents) {
 +      Pair<Long,Long> newSample = new Pair<Long,Long>(sampleTime, numEvents);
 +      Pair<Long,Long> lastSample = samples.get(name);
 +      
 +      if (lastSample == null || !lastSample.equals(newSample)) {
 +        samples.put(name, newSample);
 +        if (lastSample != null) {
 +          prevSamples.put(name, lastSample);
 +        }
 +      }
 +      serversUpdated.add(name);
 +    }
 +    
 +    void finishedUpdating() {
 +      // remove any tablet servers not updated
 +      samples.keySet().retainAll(serversUpdated);
 +      prevSamples.keySet().retainAll(serversUpdated);
 +    }
 +    
 +    double calculateRate() {
 +      double totalRate = 0;
 +      
 +      for (Entry<String,Pair<Long,Long>> entry : prevSamples.entrySet()) {
 +        Pair<Long,Long> prevSample = entry.getValue();
 +        Pair<Long,Long> sample = samples.get(entry.getKey());
 +        
 +        totalRate += (sample.getSecond() - prevSample.getSecond()) / ((sample.getFirst() - prevSample.getFirst()) / (double) 1000);
 +      }
 +      
 +      return totalRate;
 +    }
 +    
 +    long calculateCount() {
 +      long count = 0;
 +      
 +      for (Entry<String,Pair<Long,Long>> entry : prevSamples.entrySet()) {
 +        Pair<Long,Long> prevSample = entry.getValue();
 +        Pair<Long,Long> sample = samples.get(entry.getKey());
 +        
 +        count += sample.getSecond() - prevSample.getSecond();
 +      }
 +      
 +      return count;
 +    }
 +  }
 +  
 +  public static void fetchData() {
 +    double totalIngestRate = 0.;
 +    double totalIngestByteRate = 0.;
 +    double totalQueryRate = 0.;
 +    double totalQueryByteRate = 0.;
 +    double totalScanRate = 0.;
 +    long totalEntries = 0;
 +    int totalTabletCount = 0;
 +    int onlineTabletCount = 0;
 +    long totalHoldTime = 0;
 +    long totalLookups = 0;
 +    boolean retry = true;
 +    
 +    // only recalc every so often
 +    long currentTime = System.currentTimeMillis();
 +    if (currentTime - lastRecalc < REFRESH_TIME * 1000)
 +      return;
 +    
 +    synchronized (Monitor.class) {
 +      if (fetching)
 +        return;
 +      fetching = true;
 +    }
 +    
 +    try {
 +      while (retry) {
 +        MasterClientService.Iface client = null;
 +        try {
 +          client = MasterClient.getConnection(HdfsZooInstance.getInstance());
 +          if (client != null) {
 +            mmi = client.getMasterStats(Tracer.traceInfo(), SecurityConstants.getSystemCredentials());
 +            retry = false;
 +          } else {
 +            mmi = null;
 +          }
 +          Monitor.gcStatus = fetchGcStatus();
 +        } catch (Exception e) {
 +          mmi = null;
 +          log.info("Error fetching stats: " + e);
 +        } finally {
 +          if (client != null) {
 +            MasterClient.close(client);
 +          }
 +        }
 +        if (mmi == null)
 +          UtilWaitThread.sleep(1000);
 +      }
 +      if (mmi != null) {
 +        int majorCompactions = 0;
 +        int minorCompactions = 0;
 +        
 +        lookupRateTracker.startingUpdates();
 +        indexCacheHitTracker.startingUpdates();
 +        indexCacheRequestTracker.startingUpdates();
 +        dataCacheHitTracker.startingUpdates();
 +        dataCacheRequestTracker.startingUpdates();
 +        
 +        for (TabletServerStatus server : mmi.tServerInfo) {
 +          TableInfo summary = Monitor.summarizeTableStats(server);
 +          totalIngestRate += summary.ingestRate;
 +          totalIngestByteRate += summary.ingestByteRate;
 +          totalQueryRate += summary.queryRate;
 +          totalScanRate += summary.scanRate;
 +          totalQueryByteRate += summary.queryByteRate;
 +          totalEntries += summary.recs;
 +          totalHoldTime += server.holdTime;
 +          totalLookups += server.lookups;
 +          majorCompactions += summary.majors.running;
 +          minorCompactions += summary.minors.running;
 +          lookupRateTracker.updateTabletServer(server.name, server.lastContact, server.lookups);
 +          indexCacheHitTracker.updateTabletServer(server.name, server.lastContact, server.indexCacheHits);
 +          indexCacheRequestTracker.updateTabletServer(server.name, server.lastContact, server.indexCacheRequest);
 +          dataCacheHitTracker.updateTabletServer(server.name, server.lastContact, server.dataCacheHits);
 +          dataCacheRequestTracker.updateTabletServer(server.name, server.lastContact, server.dataCacheRequest);
 +        }
 +        
 +        lookupRateTracker.finishedUpdating();
 +        indexCacheHitTracker.finishedUpdating();
 +        indexCacheRequestTracker.finishedUpdating();
 +        dataCacheHitTracker.finishedUpdating();
 +        dataCacheRequestTracker.finishedUpdating();
 +        
 +        int totalTables = 0;
 +        for (TableInfo tInfo : mmi.tableMap.values()) {
 +          totalTabletCount += tInfo.tablets;
 +          onlineTabletCount += tInfo.onlineTablets;
 +          totalTables++;
 +        }
 +        Monitor.totalIngestRate = totalIngestRate;
 +        Monitor.totalTables = totalTables;
 +        totalIngestByteRate = totalIngestByteRate / 1000000.0;
 +        Monitor.totalIngestByteRate = totalIngestByteRate;
 +        Monitor.totalQueryRate = totalQueryRate;
 +        Monitor.totalScanRate = totalScanRate;
 +        totalQueryByteRate = totalQueryByteRate / 1000000.0;
 +        Monitor.totalQueryByteRate = totalQueryByteRate;
 +        Monitor.totalEntries = totalEntries;
 +        Monitor.totalTabletCount = totalTabletCount;
 +        Monitor.onlineTabletCount = onlineTabletCount;
 +        Monitor.totalHoldTime = totalHoldTime;
 +        Monitor.totalLookups = totalLookups;
 +        
 +        ingestRateOverTime.add(new Pair<Long,Double>(currentTime, totalIngestRate));
 +        ingestByteRateOverTime.add(new Pair<Long,Double>(currentTime, totalIngestByteRate));
 +        
 +        double totalLoad = 0.;
 +        for (TabletServerStatus status : mmi.tServerInfo) {
 +          if (status != null)
 +            totalLoad += status.osLoad;
 +        }
 +        loadOverTime.add(new Pair<Long,Double>(currentTime, totalLoad));
 +        
 +        minorCompactionsOverTime.add(new Pair<Long,Integer>(currentTime, minorCompactions));
 +        majorCompactionsOverTime.add(new Pair<Long,Integer>(currentTime, majorCompactions));
 +        
 +        lookupsOverTime.add(new Pair<Long,Double>(currentTime, lookupRateTracker.calculateRate()));
 +        
 +        queryRateOverTime.add(new Pair<Long,Integer>(currentTime, (int) totalQueryRate));
 +        queryByteRateOverTime.add(new Pair<Long,Double>(currentTime, totalQueryByteRate));
 +        
 +        scanRateOverTime.add(new Pair<Long,Integer>(currentTime, (int) totalScanRate));
 +        
 +        calcCacheHitRate(indexCacheHitRateOverTime, currentTime, indexCacheHitTracker, indexCacheRequestTracker);
 +        calcCacheHitRate(dataCacheHitRateOverTime, currentTime, dataCacheHitTracker, dataCacheRequestTracker);
 +      }
 +      try {
 +        Monitor.problemSummary = ProblemReports.getInstance().summarize();
 +        Monitor.problemException = null;
 +      } catch (Exception e) {
 +        log.info("Failed to obtain problem reports ", e);
 +        Monitor.problemSummary = Collections.emptyMap();
 +        Monitor.problemException = e;
 +      }
 +      
 +    } finally {
 +      synchronized (Monitor.class) {
 +        fetching = false;
 +        lastRecalc = currentTime;
 +      }
 +    }
 +  }
 +  
 +  private static void calcCacheHitRate(List<Pair<Long,Double>> hitRate, long currentTime, EventCounter cacheHits, EventCounter cacheReq) {
 +    long req = cacheReq.calculateCount();
 +    if (req > 0)
 +      hitRate.add(new Pair<Long,Double>(currentTime, cacheHits.calculateCount() / (double) cacheReq.calculateCount()));
 +    else
 +      hitRate.add(new Pair<Long,Double>(currentTime, null));
 +  }
 +  
 +  private static GCStatus fetchGcStatus() {
 +    GCStatus result = null;
 +    InetSocketAddress address = null;
 +    try {
 +      // Read the gc location from its lock
-       Instance instance = HdfsZooInstance.getInstance();
-       String zooKeepers = instance.getZooKeepers();
-       log.debug("connecting to zookeepers " + zooKeepers);
-       ZooKeeper zk = new ZooKeeper(zooKeepers, (int) config.getConfiguration().getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT), new Watcher() {
-         @Override
-         public void process(WatchedEvent event) {}
-       });
++      ZooReaderWriter zk = ZooReaderWriter.getInstance();
 +      try {
 +        String path = ZooUtil.getRoot(HdfsZooInstance.getInstance()) + Constants.ZGC_LOCK;
 +        List<String> locks = zk.getChildren(path, null);
 +        if (locks != null && locks.size() > 0) {
 +          Collections.sort(locks);
-           address = new ServerServices(new String(zk.getData(path + "/" + locks.get(0), null, null))).getAddress(Service.GC_CLIENT);
-           GCMonitorService.Client client = ThriftUtil.getClient(new GCMonitorService.Client.Factory(), address, config.getConfiguration());
++          address = new ServerServices(new String(zk.getData(path + "/" + locks.get(0), null))).getAddress(Service.GC_CLIENT);
++          GCMonitorService.Client client = ThriftUtil.getClient(new GCMonitorService.Client.Factory(), address, ServerConfiguration.getSystemConfiguration(instance));
 +          try {
 +            result = client.getStatus(Tracer.traceInfo(), SecurityConstants.getSystemCredentials());
 +          } finally {
 +            ThriftUtil.returnClient(client);
 +          }
 +        }
 +      } finally {
 +        zk.close();
 +      }
 +    } catch (Exception ex) {
 +      log.warn("Unable to contact the garbage collector at " + address, ex);
 +    }
 +    return result;
 +  }
 +  
 +  public static void main(String[] args) throws Exception {
 +    SecurityUtil.serverLogin();
 +    
 +    FileSystem fs = FileUtil.getFileSystem(CachedConfiguration.getInstance(), ServerConfiguration.getSiteConfiguration());
 +    String hostname = Accumulo.getLocalAddress(args);
 +    instance = HdfsZooInstance.getInstance();
 +    config = new ServerConfiguration(instance);
 +    Accumulo.init(fs, config, "monitor");
 +    Monitor monitor = new Monitor();
 +    Accumulo.enableTracing(hostname, "monitor");
 +    monitor.run(hostname);
 +  }
 +  
 +  private static long START_TIME;
 +  
 +  public void run(String hostname) {
 +    Monitor.START_TIME = System.currentTimeMillis();
 +    int port = config.getConfiguration().getPort(Property.MONITOR_PORT);
 +    try {
 +      log.debug("Creating monitor on port " + port);
 +      server = new EmbeddedWebServer(hostname, port);
 +    } catch (Throwable ex) {
 +      log.error("Unable to start embedded web server", ex);
 +      throw new RuntimeException(ex);
 +    }
 +    
 +    server.addServlet(DefaultServlet.class, "/");
 +    server.addServlet(OperationServlet.class, "/op");
 +    server.addServlet(MasterServlet.class, "/master");
 +    server.addServlet(TablesServlet.class, "/tables");
 +    server.addServlet(TServersServlet.class, "/tservers");
 +    server.addServlet(ProblemServlet.class, "/problems");
 +    server.addServlet(GcStatusServlet.class, "/gc");
 +    server.addServlet(LogServlet.class, "/log");
 +    server.addServlet(XMLServlet.class, "/xml");
 +    server.addServlet(JSONServlet.class, "/json");
 +    server.addServlet(VisServlet.class, "/vis");
 +    server.addServlet(Summary.class, "/trace/summary");
 +    server.addServlet(ListType.class, "/trace/listType");
 +    server.addServlet(ShowTrace.class, "/trace/show");
 +    if (server.isUsingSsl())
 +      server.addServlet(ShellServlet.class, "/shell");
 +    LogService.startLogListener(Monitor.getSystemConfiguration());
 +    server.start();
 +    
 +    new Daemon(new LoggingRunnable(log, new ZooKeeperStatus()), "ZooKeeperStatus").start();
 +    
 +    // need to regularly fetch data so plot data is updated
 +    new Daemon(new LoggingRunnable(log, new Runnable() {
 +      
 +      @Override
 +      public void run() {
 +        while (true) {
 +          try {
 +            Monitor.fetchData();
 +          } catch (Exception e) {
 +            log.warn(e.getMessage(), e);
 +          }
 +          
 +          UtilWaitThread.sleep(333);
 +        }
 +        
 +      }
 +    }), "Data fetcher").start();
 +  }
 +  
 +  public static MasterMonitorInfo getMmi() {
 +    return mmi;
 +  }
 +  
 +  public static int getTotalTables() {
 +    return totalTables;
 +  }
 +  
 +  public static int getTotalTabletCount() {
 +    return totalTabletCount;
 +  }
 +  
 +  public static int getOnlineTabletCount() {
 +    return onlineTabletCount;
 +  }
 +  
 +  public static long getTotalEntries() {
 +    return totalEntries;
 +  }
 +  
 +  public static double getTotalIngestRate() {
 +    return totalIngestRate;
 +  }
 +  
 +  public static double getTotalIngestByteRate() {
 +    return totalIngestByteRate;
 +  }
 +  
 +  public static double getTotalQueryRate() {
 +    return totalQueryRate;
 +  }
 +  
 +  public static double getTotalScanRate() {
 +    return totalScanRate;
 +  }
 +  
 +  public static double getTotalQueryByteRate() {
 +    return totalQueryByteRate;
 +  }
 +  
 +  public static long getTotalHoldTime() {
 +    return totalHoldTime;
 +  }
 +  
 +  public static Exception getProblemException() {
 +    return problemException;
 +  }
 +  
 +  public static Map<String,Map<ProblemType,Integer>> getProblemSummary() {
 +    return problemSummary;
 +  }
 +  
 +  public static GCStatus getGcStatus() {
 +    return gcStatus;
 +  }
 +  
 +  public static long getTotalLookups() {
 +    return totalLookups;
 +  }
 +  
 +  public static long getStartTime() {
 +    return START_TIME;
 +  }
 +  
 +  public static List<Pair<Long,Double>> getLoadOverTime() {
 +    synchronized (loadOverTime) {
 +      return new ArrayList<Pair<Long,Double>>(loadOverTime);
 +    }
 +  }
 +  
 +  public static List<Pair<Long,Double>> getIngestRateOverTime() {
 +    synchronized (ingestRateOverTime) {
 +      return new ArrayList<Pair<Long,Double>>(ingestRateOverTime);
 +    }
 +  }
 +  
 +  public static List<Pair<Long,Double>> getIngestByteRateOverTime() {
 +    synchronized (ingestByteRateOverTime) {
 +      return new ArrayList<Pair<Long,Double>>(ingestByteRateOverTime);
 +    }
 +  }
 +  
 +  public static List<Pair<Long,Integer>> getRecoveriesOverTime() {
 +    synchronized (recoveriesOverTime) {
 +      return new ArrayList<Pair<Long,Integer>>(recoveriesOverTime);
 +    }
 +  }
 +  
 +  public static List<Pair<Long,Integer>> getMinorCompactionsOverTime() {
 +    synchronized (minorCompactionsOverTime) {
 +      return new ArrayList<Pair<Long,Integer>>(minorCompactionsOverTime);
 +    }
 +  }
 +  
 +  public static List<Pair<Long,Integer>> getMajorCompactionsOverTime() {
 +    synchronized (majorCompactionsOverTime) {
 +      return new ArrayList<Pair<Long,Integer>>(majorCompactionsOverTime);
 +    }
 +  }
 +  
 +  public static List<Pair<Long,Double>> getLookupsOverTime() {
 +    synchronized (lookupsOverTime) {
 +      return new ArrayList<Pair<Long,Double>>(lookupsOverTime);
 +    }
 +  }
 +  
 +  public static double getLookupRate() {
 +    return lookupRateTracker.calculateRate();
 +  }
 +  
 +  public static List<Pair<Long,Integer>> getQueryRateOverTime() {
 +    synchronized (queryRateOverTime) {
 +      return new ArrayList<Pair<Long,Integer>>(queryRateOverTime);
 +    }
 +  }
 +  
 +  public static List<Pair<Long,Integer>> getScanRateOverTime() {
 +    synchronized (scanRateOverTime) {
 +      return new ArrayList<Pair<Long,Integer>>(scanRateOverTime);
 +    }
 +  }
 +  
 +  public static List<Pair<Long,Double>> getQueryByteRateOverTime() {
 +    synchronized (queryByteRateOverTime) {
 +      return new ArrayList<Pair<Long,Double>>(queryByteRateOverTime);
 +    }
 +  }
 +  
 +  public static List<Pair<Long,Double>> getIndexCacheHitRateOverTime() {
 +    synchronized (indexCacheHitRateOverTime) {
 +      return new ArrayList<Pair<Long,Double>>(indexCacheHitRateOverTime);
 +    }
 +  }
 +  
 +  public static List<Pair<Long,Double>> getDataCacheHitRateOverTime() {
 +    synchronized (dataCacheHitRateOverTime) {
 +      return new ArrayList<Pair<Long,Double>>(dataCacheHitRateOverTime);
 +    }
 +  }
 +  
 +  public static AccumuloConfiguration getSystemConfiguration() {
 +    return config.getConfiguration();
 +  }
 +  
 +  public static Instance getInstance() {
 +    return instance;
 +  }
 +  
 +  public static boolean isUsingSsl() {
 +    return server.isUsingSsl();
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/041270b7/server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java
----------------------------------------------------------------------
diff --cc server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java
index aa5fe48,0000000..f859523
mode 100644,000000..100644
--- a/server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java
+++ b/server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java
@@@ -1,1257 -1,0 +1,1259 @@@
 +/*
 + * 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.io.IOException;
 +import java.util.ArrayList;
 +import java.util.Arrays;
 +import java.util.Collection;
 +import java.util.Collections;
 +import java.util.Comparator;
 +import java.util.HashMap;
 +import java.util.HashSet;
 +import java.util.Iterator;
 +import java.util.List;
 +import java.util.Map;
 +import java.util.Map.Entry;
 +import java.util.Set;
 +import java.util.SortedMap;
 +import java.util.TreeMap;
 +import java.util.concurrent.TimeUnit;
 +
 +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.BatchWriter;
 +import org.apache.accumulo.core.client.BatchWriterConfig;
 +import org.apache.accumulo.core.client.Connector;
 +import org.apache.accumulo.core.client.Instance;
 +import org.apache.accumulo.core.client.IsolatedScanner;
 +import org.apache.accumulo.core.client.MutationsRejectedException;
 +import org.apache.accumulo.core.client.Scanner;
 +import org.apache.accumulo.core.client.TableNotFoundException;
 +import org.apache.accumulo.core.client.impl.BatchWriterImpl;
 +import org.apache.accumulo.core.client.impl.ScannerImpl;
 +import org.apache.accumulo.core.client.impl.Writer;
 +import org.apache.accumulo.core.data.Key;
 +import org.apache.accumulo.core.data.KeyExtent;
 +import org.apache.accumulo.core.data.Mutation;
 +import org.apache.accumulo.core.data.PartialKey;
 +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.security.CredentialHelper;
 +import org.apache.accumulo.core.security.thrift.TCredentials;
 +import org.apache.accumulo.core.tabletserver.thrift.ConstraintViolationException;
 +import org.apache.accumulo.core.util.CachedConfiguration;
 +import org.apache.accumulo.core.util.ColumnFQ;
 +import org.apache.accumulo.core.util.FastFormat;
 +import org.apache.accumulo.core.util.Pair;
 +import org.apache.accumulo.core.util.StringUtil;
 +import org.apache.accumulo.core.util.UtilWaitThread;
 +import org.apache.accumulo.core.zookeeper.ZooUtil;
 +import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 +import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
 +import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy;
 +import org.apache.accumulo.server.ServerConstants;
 +import org.apache.accumulo.server.client.HdfsZooInstance;
 +import org.apache.accumulo.server.conf.ServerConfiguration;
 +import org.apache.accumulo.server.master.state.TServerInstance;
 +import org.apache.accumulo.server.security.SecurityConstants;
 +import org.apache.accumulo.server.trace.TraceFileSystem;
 +import org.apache.accumulo.server.zookeeper.ZooLock;
 +import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 +import org.apache.hadoop.fs.FileStatus;
 +import org.apache.hadoop.fs.FileSystem;
 +import org.apache.hadoop.fs.Path;
 +import org.apache.hadoop.io.DataInputBuffer;
 +import org.apache.hadoop.io.DataOutputBuffer;
 +import org.apache.hadoop.io.Text;
 +import org.apache.log4j.Logger;
 +import org.apache.zookeeper.KeeperException;
 +
 +/**
 + * provides a reference to the metadata table for updates by tablet servers
 + */
 +public class MetadataTable extends org.apache.accumulo.core.util.MetadataTable {
 +  
 +  private static final Text EMPTY_TEXT = new Text();
 +  private static Map<TCredentials,Writer> metadata_tables = new HashMap<TCredentials,Writer>();
 +  private static final Logger log = Logger.getLogger(MetadataTable.class);
 +  
 +  private static final int SAVE_ROOT_TABLET_RETRIES = 3;
 +  
 +  private MetadataTable() {
 +    
 +  }
 +  
 +  public synchronized static Writer getMetadataTable(TCredentials credentials) {
 +    Writer metadataTable = metadata_tables.get(credentials);
 +    if (metadataTable == null) {
 +      metadataTable = new Writer(HdfsZooInstance.getInstance(), credentials, Constants.METADATA_TABLE_ID);
 +      metadata_tables.put(credentials, metadataTable);
 +    }
 +    return metadataTable;
 +  }
 +  
 +  public static void putLockID(ZooLock zooLock, Mutation m) {
 +    Constants.METADATA_LOCK_COLUMN.put(m, new Value(zooLock.getLockID().serialize(ZooUtil.getRoot(HdfsZooInstance.getInstance()) + "/").getBytes()));
 +  }
 +  
 +  public static void update(TCredentials credentials, Mutation m) {
 +    update(credentials, null, m);
 +  }
 +  
 +  public static void update(TCredentials credentials, ZooLock zooLock, Mutation m) {
 +    Writer t;
 +    t = getMetadataTable(credentials);
 +    if (zooLock != null)
 +      putLockID(zooLock, m);
 +    while (true) {
 +      try {
 +        t.update(m);
 +        return;
 +      } catch (AccumuloException e) {
 +        log.error(e, e);
 +      } catch (AccumuloSecurityException e) {
 +        log.error(e, e);
 +      } catch (ConstraintViolationException e) {
 +        log.error(e, e);
 +      } catch (TableNotFoundException e) {
 +        log.error(e, e);
 +      }
 +      UtilWaitThread.sleep(1000);
 +    }
 +    
 +  }
 +  
 +  /**
 +   * new data file update function adds one data file to a tablet's list
 +   * 
 +   * path should be relative to the table directory
 +   * 
 +   * @param time
 +   * @param filesInUseByScans
 +   * @param zooLock
 +   * @param flushId
 +   * 
 +   */
 +  public static void updateTabletDataFile(KeyExtent extent, String path, String mergeFile, DataFileValue dfv, String time, TCredentials credentials,
 +      Set<String> filesInUseByScans, String address, ZooLock zooLock, Set<String> unusedWalLogs, TServerInstance lastLocation, long flushId) {
 +    if (extent.equals(Constants.ROOT_TABLET_EXTENT)) {
 +      if (unusedWalLogs != null) {
 +        IZooReaderWriter zk = ZooReaderWriter.getInstance();
 +        // unusedWalLogs will contain the location/name of each log in a log set
 +        // the log set is stored under one of the log names, but not both
 +        // find the entry under one of the names and delete it.
 +        String root = getZookeeperLogLocation();
 +        boolean foundEntry = false;
 +        for (String entry : unusedWalLogs) {
 +          String[] parts = entry.split("/");
 +          String zpath = root + "/" + parts[1];
 +          while (true) {
 +            try {
 +              if (zk.exists(zpath)) {
 +                zk.recursiveDelete(zpath, NodeMissingPolicy.SKIP);
 +                foundEntry = true;
 +              }
 +              break;
 +            } catch (KeeperException e) {
 +              log.error(e, e);
 +            } catch (InterruptedException e) {
 +              log.error(e, e);
 +            }
 +            UtilWaitThread.sleep(1000);
 +          }
 +        }
 +        if (unusedWalLogs.size() > 0 && !foundEntry)
 +          log.warn("WALog entry for root tablet did not exist " + unusedWalLogs);
 +      }
 +      return;
 +    }
 +    
 +    Mutation m = new Mutation(extent.getMetadataEntry());
 +    
 +    if (dfv.getNumEntries() > 0) {
 +      m.put(Constants.METADATA_DATAFILE_COLUMN_FAMILY, new Text(path), new Value(dfv.encode()));
 +      Constants.METADATA_TIME_COLUMN.put(m, new Value(time.getBytes()));
 +      // stuff in this location
 +      TServerInstance self = getTServerInstance(address, zooLock);
 +      self.putLastLocation(m);
 +      // erase the old location
 +      if (lastLocation != null && !lastLocation.equals(self))
 +        lastLocation.clearLastLocation(m);
 +    }
 +    if (unusedWalLogs != null) {
 +      for (String entry : unusedWalLogs) {
 +        m.putDelete(Constants.METADATA_LOG_COLUMN_FAMILY, new Text(entry));
 +      }
 +    }
 +    
 +    for (String scanFile : filesInUseByScans)
 +      m.put(Constants.METADATA_SCANFILE_COLUMN_FAMILY, new Text(scanFile), new Value("".getBytes()));
 +    
 +    if (mergeFile != null)
 +      m.putDelete(Constants.METADATA_DATAFILE_COLUMN_FAMILY, new Text(mergeFile));
 +    
 +    Constants.METADATA_FLUSH_COLUMN.put(m, new Value((flushId + "").getBytes()));
 +    
 +    update(credentials, zooLock, m);
 +    
 +  }
 +  
 +  private static TServerInstance getTServerInstance(String address, ZooLock zooLock) {
 +    while (true) {
 +      try {
 +        return new TServerInstance(address, zooLock.getSessionId());
 +      } catch (KeeperException e) {
 +        log.error(e, e);
 +      } catch (InterruptedException e) {
 +        log.error(e, e);
 +      }
 +      UtilWaitThread.sleep(1000);
 +    }
 +  }
 +  
 +  public static void updateTabletFlushID(KeyExtent extent, long flushID, TCredentials credentials, ZooLock zooLock) {
 +    if (!extent.isRootTablet()) {
 +      Mutation m = new Mutation(extent.getMetadataEntry());
 +      Constants.METADATA_FLUSH_COLUMN.put(m, new Value((flushID + "").getBytes()));
 +      update(credentials, zooLock, m);
 +    }
 +  }
 +  
 +  public static void updateTabletCompactID(KeyExtent extent, long compactID, TCredentials credentials, ZooLock zooLock) {
 +    if (!extent.isRootTablet()) {
 +      Mutation m = new Mutation(extent.getMetadataEntry());
 +      Constants.METADATA_COMPACT_COLUMN.put(m, new Value((compactID + "").getBytes()));
 +      update(credentials, zooLock, m);
 +    }
 +  }
 +  
 +  public static void updateTabletDataFile(long tid, KeyExtent extent, Map<String,DataFileValue> estSizes, String time, TCredentials credentials, ZooLock zooLock) {
 +    Mutation m = new Mutation(extent.getMetadataEntry());
 +    byte[] tidBytes = Long.toString(tid).getBytes();
 +    
 +    for (Entry<String,DataFileValue> entry : estSizes.entrySet()) {
 +      Text file = new Text(entry.getKey());
 +      m.put(Constants.METADATA_DATAFILE_COLUMN_FAMILY, file, new Value(entry.getValue().encode()));
 +      m.put(Constants.METADATA_BULKFILE_COLUMN_FAMILY, file, new Value(tidBytes));
 +    }
 +    Constants.METADATA_TIME_COLUMN.put(m, new Value(time.getBytes()));
 +    update(credentials, zooLock, m);
 +  }
 +  
 +  public static void addTablet(KeyExtent extent, String path, TCredentials credentials, char timeType, ZooLock lock) {
 +    Mutation m = extent.getPrevRowUpdateMutation();
 +    
 +    Constants.METADATA_DIRECTORY_COLUMN.put(m, new Value(path.getBytes()));
 +    Constants.METADATA_TIME_COLUMN.put(m, new Value((timeType + "0").getBytes()));
 +    
 +    update(credentials, lock, m);
 +  }
 +  
 +  public static void updateTabletPrevEndRow(KeyExtent extent, TCredentials credentials) {
 +    Mutation m = extent.getPrevRowUpdateMutation(); //
 +    update(credentials, m);
 +  }
 +  
 +  /**
 +   * convenience method for reading entries from the metadata table
 +   */
 +  public static SortedMap<KeyExtent,Text> getMetadataDirectoryEntries(SortedMap<Key,Value> entries) {
 +    Key key;
 +    Value val;
 +    Text datafile = null;
 +    Value prevRow = null;
 +    KeyExtent ke;
 +    
 +    SortedMap<KeyExtent,Text> results = new TreeMap<KeyExtent,Text>();
 +    
 +    Text lastRowFromKey = new Text();
 +    
 +    // text obj below is meant to be reused in loop for efficiency
 +    Text colf = new Text();
 +    Text colq = new Text();
 +    
 +    for (Entry<Key,Value> entry : entries.entrySet()) {
 +      key = entry.getKey();
 +      val = entry.getValue();
 +      
 +      if (key.compareRow(lastRowFromKey) != 0) {
 +        prevRow = null;
 +        datafile = null;
 +        key.getRow(lastRowFromKey);
 +      }
 +      
 +      colf = key.getColumnFamily(colf);
 +      colq = key.getColumnQualifier(colq);
 +      
 +      // interpret the row id as a key extent
 +      if (Constants.METADATA_DIRECTORY_COLUMN.equals(colf, colq))
 +        datafile = new Text(val.toString());
 +      
 +      else if (Constants.METADATA_PREV_ROW_COLUMN.equals(colf, colq))
 +        prevRow = new Value(val);
 +      
 +      if (datafile != null && prevRow != null) {
 +        ke = new KeyExtent(key.getRow(), prevRow);
 +        results.put(ke, datafile);
 +        
 +        datafile = null;
 +        prevRow = null;
 +      }
 +    }
 +    return results;
 +  }
 +  
 +  public static boolean recordRootTabletLocation(String address) {
 +    IZooReaderWriter zoo = ZooReaderWriter.getInstance();
 +    for (int i = 0; i < SAVE_ROOT_TABLET_RETRIES; i++) {
 +      try {
 +        log.info("trying to write root tablet location to ZooKeeper as " + address);
 +        String zRootLocPath = ZooUtil.getRoot(HdfsZooInstance.getInstance()) + Constants.ZROOT_TABLET_LOCATION;
 +        zoo.putPersistentData(zRootLocPath, address.getBytes(), NodeExistsPolicy.OVERWRITE);
 +        return true;
 +      } catch (Exception e) {
 +        log.error("Master: unable to save root tablet location in zookeeper. exception: " + e, e);
 +      }
 +    }
 +    log.error("Giving up after " + SAVE_ROOT_TABLET_RETRIES + " retries");
 +    return false;
 +  }
 +  
 +  public static SortedMap<String,DataFileValue> getDataFileSizes(KeyExtent extent, TCredentials credentials) {
 +    TreeMap<String,DataFileValue> sizes = new TreeMap<String,DataFileValue>();
 +    
 +    Scanner mdScanner = new ScannerImpl(HdfsZooInstance.getInstance(), credentials, Constants.METADATA_TABLE_ID, Constants.NO_AUTHS);
 +    mdScanner.fetchColumnFamily(Constants.METADATA_DATAFILE_COLUMN_FAMILY);
 +    Text row = extent.getMetadataEntry();
 +    
 +    Key endKey = new Key(row, Constants.METADATA_DATAFILE_COLUMN_FAMILY, new Text(""));
 +    endKey = endKey.followingKey(PartialKey.ROW_COLFAM);
 +    
 +    mdScanner.setRange(new Range(new Key(row), endKey));
 +    for (Entry<Key,Value> entry : mdScanner) {
 +      
 +      if (!entry.getKey().getRow().equals(row))
 +        break;
 +      DataFileValue dfv = new DataFileValue(entry.getValue().get());
 +      sizes.put(entry.getKey().getColumnQualifier().toString(), dfv);
 +    }
 +    
 +    return sizes;
 +  }
 +  
 +  public static void addNewTablet(KeyExtent extent, String path, TServerInstance location, Map<String,DataFileValue> datafileSizes,
 +      Map<String,Long> bulkLoadedFiles, TCredentials credentials, String time, long lastFlushID, long lastCompactID, ZooLock zooLock) {
 +    Mutation m = extent.getPrevRowUpdateMutation();
 +    
 +    Constants.METADATA_DIRECTORY_COLUMN.put(m, new Value(path.getBytes()));
 +    Constants.METADATA_TIME_COLUMN.put(m, new Value(time.getBytes()));
 +    if (lastFlushID > 0)
 +      Constants.METADATA_FLUSH_COLUMN.put(m, new Value(("" + lastFlushID).getBytes()));
 +    if (lastCompactID > 0)
 +      Constants.METADATA_COMPACT_COLUMN.put(m, new Value(("" + lastCompactID).getBytes()));
 +    
 +    if (location != null) {
 +      m.put(Constants.METADATA_CURRENT_LOCATION_COLUMN_FAMILY, location.asColumnQualifier(), location.asMutationValue());
 +      m.putDelete(Constants.METADATA_FUTURE_LOCATION_COLUMN_FAMILY, location.asColumnQualifier());
 +    }
 +    
 +    for (Entry<String,DataFileValue> entry : datafileSizes.entrySet()) {
 +      m.put(Constants.METADATA_DATAFILE_COLUMN_FAMILY, new Text(entry.getKey()), new Value(entry.getValue().encode()));
 +    }
 +    
 +    for (Entry<String,Long> entry : bulkLoadedFiles.entrySet()) {
 +      byte[] tidBytes = Long.toString(entry.getValue()).getBytes();
 +      m.put(Constants.METADATA_BULKFILE_COLUMN_FAMILY, new Text(entry.getKey()), new Value(tidBytes));
 +    }
 +    
 +    update(credentials, zooLock, m);
 +  }
 +  
 +  public static void rollBackSplit(Text metadataEntry, Text oldPrevEndRow, TCredentials credentials, ZooLock zooLock) {
 +    KeyExtent ke = new KeyExtent(metadataEntry, oldPrevEndRow);
 +    Mutation m = ke.getPrevRowUpdateMutation();
 +    Constants.METADATA_SPLIT_RATIO_COLUMN.putDelete(m);
 +    Constants.METADATA_OLD_PREV_ROW_COLUMN.putDelete(m);
 +    update(credentials, zooLock, m);
 +  }
 +
 +  public static void splitTablet(KeyExtent extent, Text oldPrevEndRow, double splitRatio, TCredentials credentials, ZooLock zooLock) {
 +    Mutation m = extent.getPrevRowUpdateMutation(); //
 +    
 +    Constants.METADATA_SPLIT_RATIO_COLUMN.put(m, new Value(Double.toString(splitRatio).getBytes()));
 +    
 +    Constants.METADATA_OLD_PREV_ROW_COLUMN.put(m, KeyExtent.encodePrevEndRow(oldPrevEndRow));
 +    Constants.METADATA_CHOPPED_COLUMN.putDelete(m);
 +    update(credentials, zooLock, m);
 +  }
 +  
 +  public static void finishSplit(Text metadataEntry, Map<String,DataFileValue> datafileSizes, List<String> highDatafilesToRemove, TCredentials credentials,
 +      ZooLock zooLock) {
 +    Mutation m = new Mutation(metadataEntry);
 +    Constants.METADATA_SPLIT_RATIO_COLUMN.putDelete(m);
 +    Constants.METADATA_OLD_PREV_ROW_COLUMN.putDelete(m);
 +    Constants.METADATA_CHOPPED_COLUMN.putDelete(m);
 +    
 +    for (Entry<String,DataFileValue> entry : datafileSizes.entrySet()) {
 +      m.put(Constants.METADATA_DATAFILE_COLUMN_FAMILY, new Text(entry.getKey()), new Value(entry.getValue().encode()));
 +    }
 +    
 +    for (String pathToRemove : highDatafilesToRemove) {
 +      m.putDelete(Constants.METADATA_DATAFILE_COLUMN_FAMILY, new Text(pathToRemove));
 +    }
 +    
 +    update(credentials, zooLock, m);
 +  }
 +  
 +  public static void finishSplit(KeyExtent extent, Map<String,DataFileValue> datafileSizes, List<String> highDatafilesToRemove, TCredentials credentials,
 +      ZooLock zooLock) {
 +    finishSplit(extent.getMetadataEntry(), datafileSizes, highDatafilesToRemove, credentials, zooLock);
 +  }
 +  
 +  public static void replaceDatafiles(KeyExtent extent, Set<String> datafilesToDelete, Set<String> scanFiles, String path, Long compactionId,
 +      DataFileValue size, TCredentials credentials, String address, TServerInstance lastLocation, ZooLock zooLock) {
 +    replaceDatafiles(extent, datafilesToDelete, scanFiles, path, compactionId, size, credentials, address, lastLocation, zooLock, true);
 +  }
 +  
 +  public static void replaceDatafiles(KeyExtent extent, Set<String> datafilesToDelete, Set<String> scanFiles, String path, Long compactionId,
 +      DataFileValue size, TCredentials credentials, String address, TServerInstance lastLocation, ZooLock zooLock, boolean insertDeleteFlags) {
 +    
 +    if (insertDeleteFlags) {
 +      // add delete flags for those paths before the data file reference is removed
 +      addDeleteEntries(extent, datafilesToDelete, credentials);
 +    }
 +    
 +    // replace data file references to old mapfiles with the new mapfiles
 +    Mutation m = new Mutation(extent.getMetadataEntry());
 +    
 +    for (String pathToRemove : datafilesToDelete)
 +      m.putDelete(Constants.METADATA_DATAFILE_COLUMN_FAMILY, new Text(pathToRemove));
 +    
 +    for (String scanFile : scanFiles)
 +      m.put(Constants.METADATA_SCANFILE_COLUMN_FAMILY, new Text(scanFile), new Value("".getBytes()));
 +    
 +    if (size.getNumEntries() > 0)
 +      m.put(Constants.METADATA_DATAFILE_COLUMN_FAMILY, new Text(path), new Value(size.encode()));
 +    
 +    if (compactionId != null)
 +      Constants.METADATA_COMPACT_COLUMN.put(m, new Value(("" + compactionId).getBytes()));
 +    
 +    TServerInstance self = getTServerInstance(address, zooLock);
 +    self.putLastLocation(m);
 +    
 +    // remove the old location
 +    if (lastLocation != null && !lastLocation.equals(self))
 +      lastLocation.clearLastLocation(m);
 +    
 +    update(credentials, zooLock, m);
 +  }
 +  
 +  public static void addDeleteEntries(KeyExtent extent, Set<String> datafilesToDelete, TCredentials credentials) {
 +    
 +    String tableId = extent.getTableId().toString();
 +    
 +    // TODO could use batch writer,would need to handle failure and retry like update does - ACCUMULO-1294
 +    for (String pathToRemove : datafilesToDelete)
 +      update(credentials, createDeleteMutation(tableId, pathToRemove));
 +  }
 +  
 +  public static void addDeleteEntry(String tableId, String path) {
 +    update(SecurityConstants.getSystemCredentials(), createDeleteMutation(tableId, path));
 +  }
 +  
 +  public static Mutation createDeleteMutation(String tableId, String pathToRemove) {
 +    Mutation delFlag;
 +    String prefix = Constants.METADATA_DELETE_FLAG_PREFIX;
 +    if (tableId.equals(Constants.METADATA_TABLE_ID))
 +      prefix = Constants.METADATA_DELETE_FLAG_FOR_METADATA_PREFIX;
 +
 +    if (pathToRemove.startsWith("../"))
 +      delFlag = new Mutation(new Text(prefix + pathToRemove.substring(2)));
 +    else
 +      delFlag = new Mutation(new Text(prefix + "/" + tableId + pathToRemove));
 +
 +    delFlag.put(EMPTY_TEXT, EMPTY_TEXT, new Value(new byte[] {}));
 +    return delFlag;
 +  }
 +  
 +  public static void removeScanFiles(KeyExtent extent, Set<String> scanFiles, TCredentials credentials, ZooLock zooLock) {
 +    Mutation m = new Mutation(extent.getMetadataEntry());
 +    
 +    for (String pathToRemove : scanFiles)
 +      m.putDelete(Constants.METADATA_SCANFILE_COLUMN_FAMILY, new Text(pathToRemove));
 +    
 +    update(credentials, zooLock, m);
 +  }
 +  
 +  private static KeyExtent fixSplit(Text table, Text metadataEntry, Text metadataPrevEndRow, Value oper, double splitRatio, TServerInstance tserver,
 +      TCredentials credentials, String time, long initFlushID, long initCompactID, ZooLock lock) throws AccumuloException {
 +    if (metadataPrevEndRow == null)
 +      // something is wrong, this should not happen... if a tablet is split, it will always have a
 +      // prev end row....
 +      throw new AccumuloException("Split tablet does not have prev end row, something is amiss, extent = " + metadataEntry);
 +    
 +    // check to see if prev tablet exist in metadata tablet
 +    Key prevRowKey = new Key(new Text(KeyExtent.getMetadataEntry(table, metadataPrevEndRow)));
 +
 +    ScannerImpl scanner2 = new ScannerImpl(HdfsZooInstance.getInstance(), credentials, Constants.METADATA_TABLE_ID, Constants.NO_AUTHS);
 +    scanner2.setRange(new Range(prevRowKey, prevRowKey.followingKey(PartialKey.ROW)));
 +    
 +    if (!scanner2.iterator().hasNext()) {
 +      log.info("Rolling back incomplete split " + metadataEntry + " " + metadataPrevEndRow);
 +      rollBackSplit(metadataEntry, KeyExtent.decodePrevEndRow(oper), credentials, lock);
 +      return new KeyExtent(metadataEntry, KeyExtent.decodePrevEndRow(oper));
 +    } else {
 +      log.info("Finishing incomplete split " + metadataEntry + " " + metadataPrevEndRow);
 +
 +      List<String> highDatafilesToRemove = new ArrayList<String>();
 +
 +      Scanner scanner3 = new ScannerImpl(HdfsZooInstance.getInstance(), credentials, Constants.METADATA_TABLE_ID, Constants.NO_AUTHS);
 +      Key rowKey = new Key(metadataEntry);
 +      
 +      SortedMap<String,DataFileValue> origDatafileSizes = new TreeMap<String,DataFileValue>();
 +      SortedMap<String,DataFileValue> highDatafileSizes = new TreeMap<String,DataFileValue>();
 +      SortedMap<String,DataFileValue> lowDatafileSizes = new TreeMap<String,DataFileValue>();
 +      scanner3.fetchColumnFamily(Constants.METADATA_DATAFILE_COLUMN_FAMILY);
 +      scanner3.setRange(new Range(rowKey, rowKey.followingKey(PartialKey.ROW)));
 +      
 +      for (Entry<Key,Value> entry : scanner3) {
 +        if (entry.getKey().compareColumnFamily(Constants.METADATA_DATAFILE_COLUMN_FAMILY) == 0) {
 +          origDatafileSizes.put(entry.getKey().getColumnQualifier().toString(), new DataFileValue(entry.getValue().get()));
 +        }
 +      }
 +      
 +      splitDatafiles(table, metadataPrevEndRow, splitRatio, new HashMap<String,FileUtil.FileInfo>(), origDatafileSizes, lowDatafileSizes, highDatafileSizes,
 +          highDatafilesToRemove);
 +    
 +      MetadataTable.finishSplit(metadataEntry, highDatafileSizes, highDatafilesToRemove, credentials, lock);
 +      
 +      return new KeyExtent(metadataEntry, KeyExtent.encodePrevEndRow(metadataPrevEndRow));
 +    }
 +
 +
 +  }
 +  
 +  public static void splitDatafiles(Text table, Text midRow, double splitRatio, Map<String,FileUtil.FileInfo> firstAndLastRows,
 +      SortedMap<String,DataFileValue> datafiles, SortedMap<String,DataFileValue> lowDatafileSizes, SortedMap<String,DataFileValue> highDatafileSizes,
 +      List<String> highDatafilesToRemove) {
 +    
 +    for (Entry<String,DataFileValue> entry : datafiles.entrySet()) {
 +      
 +      Text firstRow = null;
 +      Text lastRow = null;
 +      
 +      boolean rowsKnown = false;
 +      
 +      FileUtil.FileInfo mfi = firstAndLastRows.get(entry.getKey());
 +      
 +      if (mfi != null) {
 +        firstRow = mfi.getFirstRow();
 +        lastRow = mfi.getLastRow();
 +        rowsKnown = true;
 +      }
 +      
 +      if (rowsKnown && firstRow.compareTo(midRow) > 0) {
 +        // only in high
 +        long highSize = entry.getValue().getSize();
 +        long highEntries = entry.getValue().getNumEntries();
 +        highDatafileSizes.put(entry.getKey(), new DataFileValue(highSize, highEntries, entry.getValue().getTime()));
 +      } else if (rowsKnown && lastRow.compareTo(midRow) <= 0) {
 +        // only in low
 +        long lowSize = entry.getValue().getSize();
 +        long lowEntries = entry.getValue().getNumEntries();
 +        lowDatafileSizes.put(entry.getKey(), new DataFileValue(lowSize, lowEntries, entry.getValue().getTime()));
 +        
 +        highDatafilesToRemove.add(entry.getKey());
 +      } else {
 +        long lowSize = (long) Math.floor((entry.getValue().getSize() * splitRatio));
 +        long lowEntries = (long) Math.floor((entry.getValue().getNumEntries() * splitRatio));
 +        lowDatafileSizes.put(entry.getKey(), new DataFileValue(lowSize, lowEntries, entry.getValue().getTime()));
 +        
 +        long highSize = (long) Math.ceil((entry.getValue().getSize() * (1.0 - splitRatio)));
 +        long highEntries = (long) Math.ceil((entry.getValue().getNumEntries() * (1.0 - splitRatio)));
 +        highDatafileSizes.put(entry.getKey(), new DataFileValue(highSize, highEntries, entry.getValue().getTime()));
 +      }
 +    }
 +  }
 +  
 +  public static KeyExtent fixSplit(Text metadataEntry, SortedMap<ColumnFQ,Value> columns, TServerInstance tserver, TCredentials credentials, ZooLock lock)
 +      throws AccumuloException {
 +    log.info("Incomplete split " + metadataEntry + " attempting to fix");
 +    
 +    Value oper = columns.get(Constants.METADATA_OLD_PREV_ROW_COLUMN);
 +    
 +    if (columns.get(Constants.METADATA_SPLIT_RATIO_COLUMN) == null) {
 +      throw new IllegalArgumentException("Metadata entry does not have split ratio (" + metadataEntry + ")");
 +    }
 +    
 +    double splitRatio = Double.parseDouble(new String(columns.get(Constants.METADATA_SPLIT_RATIO_COLUMN).get()));
 +    
 +    Value prevEndRowIBW = columns.get(Constants.METADATA_PREV_ROW_COLUMN);
 +    
 +    if (prevEndRowIBW == null) {
 +      throw new IllegalArgumentException("Metadata entry does not have prev row (" + metadataEntry + ")");
 +    }
 +    
 +    Value time = columns.get(Constants.METADATA_TIME_COLUMN);
 +    
 +    if (time == null) {
 +      throw new IllegalArgumentException("Metadata entry does not have time (" + metadataEntry + ")");
 +    }
 +    
 +    Value flushID = columns.get(Constants.METADATA_FLUSH_COLUMN);
 +    long initFlushID = -1;
 +    if (flushID != null)
 +      initFlushID = Long.parseLong(flushID.toString());
 +    
 +    Value compactID = columns.get(Constants.METADATA_COMPACT_COLUMN);
 +    long initCompactID = -1;
 +    if (compactID != null)
 +      initCompactID = Long.parseLong(compactID.toString());
 +    
 +    Text metadataPrevEndRow = KeyExtent.decodePrevEndRow(prevEndRowIBW);
 +    
 +    Text table = (new KeyExtent(metadataEntry, (Text) null)).getTableId();
 +    
 +    return fixSplit(table, metadataEntry, metadataPrevEndRow, oper, splitRatio, tserver, credentials, time.toString(), initFlushID, initCompactID, lock);
 +  }
 +  
 +  public static void deleteTable(String tableId, boolean insertDeletes, TCredentials credentials, ZooLock lock) throws AccumuloException {
 +    Scanner ms = new ScannerImpl(HdfsZooInstance.getInstance(), credentials, Constants.METADATA_TABLE_ID, Constants.NO_AUTHS);
 +    Text tableIdText = new Text(tableId);
 +    BatchWriter bw = new BatchWriterImpl(HdfsZooInstance.getInstance(), credentials, Constants.METADATA_TABLE_ID, new BatchWriterConfig().setMaxMemory(1000000)
 +        .setMaxLatency(120000l, TimeUnit.MILLISECONDS).setMaxWriteThreads(2));
 +    
 +    // scan metadata for our table and delete everything we find
 +    Mutation m = null;
 +    ms.setRange(new KeyExtent(tableIdText, null, null).toMetadataRange());
 +    
 +    // insert deletes before deleting data from !METADATA... this makes the code fault tolerant
 +    if (insertDeletes) {
 +      
 +      ms.fetchColumnFamily(Constants.METADATA_DATAFILE_COLUMN_FAMILY);
 +      Constants.METADATA_DIRECTORY_COLUMN.fetch(ms);
 +      
 +      for (Entry<Key,Value> cell : ms) {
 +        Key key = cell.getKey();
 +        
 +        if (key.getColumnFamily().equals(Constants.METADATA_DATAFILE_COLUMN_FAMILY)) {
 +          String relPath = key.getColumnQualifier().toString();
 +          // only insert deletes for files owned by this table
 +          if (!relPath.startsWith("../"))
 +            bw.addMutation(createDeleteMutation(tableId, relPath));
 +        }
 +        
 +        if (Constants.METADATA_DIRECTORY_COLUMN.hasColumns(key)) {
 +          bw.addMutation(createDeleteMutation(tableId, cell.getValue().toString()));
 +        }
 +      }
 +      
 +      bw.flush();
 +      
 +      ms.clearColumns();
 +    }
 +    
 +    for (Entry<Key,Value> cell : ms) {
 +      Key key = cell.getKey();
 +      
 +      if (m == null) {
 +        m = new Mutation(key.getRow());
 +        if (lock != null)
 +          putLockID(lock, m);
 +      }
 +      
 +      if (key.getRow().compareTo(m.getRow(), 0, m.getRow().length) != 0) {
 +        bw.addMutation(m);
 +        m = new Mutation(key.getRow());
 +        if (lock != null)
 +          putLockID(lock, m);
 +      }
 +      m.putDelete(key.getColumnFamily(), key.getColumnQualifier());
 +    }
 +    
 +    if (m != null)
 +      bw.addMutation(m);
 +    
 +    bw.close();
 +  }
 +  
 +  public static class LogEntry {
 +    public KeyExtent extent;
 +    public long timestamp;
 +    public String server;
 +    public String filename;
 +    public int tabletId;
 +    public Collection<String> logSet;
 +    
 +    @Override
 +    public String toString() {
 +      return extent.toString() + " " + filename + " (" + tabletId + ")";
 +    }
 +    
 +    public String getName() {
 +      return server + "/" + filename;
 +    }
 +    
 +    public byte[] toBytes() throws IOException {
 +      DataOutputBuffer out = new DataOutputBuffer();
 +      extent.write(out);
 +      out.writeLong(timestamp);
 +      out.writeUTF(server);
 +      out.writeUTF(filename);
 +      out.write(tabletId);
 +      out.write(logSet.size());
 +      for (String s : logSet) {
 +        out.writeUTF(s);
 +      }
 +      return Arrays.copyOf(out.getData(), out.getLength());
 +    }
 +    
 +    public void fromBytes(byte bytes[]) throws IOException {
 +      DataInputBuffer inp = new DataInputBuffer();
 +      inp.reset(bytes, bytes.length);
 +      extent = new KeyExtent();
 +      extent.readFields(inp);
 +      timestamp = inp.readLong();
 +      server = inp.readUTF();
 +      filename = inp.readUTF();
 +      tabletId = inp.read();
 +      int count = inp.read();
 +      ArrayList<String> logSet = new ArrayList<String>(count);
 +      for (int i = 0; i < count; i++)
 +        logSet.add(inp.readUTF());
 +      this.logSet = logSet;
 +    }
 +    
 +  }
 +  
 +  private static String getZookeeperLogLocation() {
 +    return ZooUtil.getRoot(HdfsZooInstance.getInstance()) + Constants.ZROOT_TABLET_WALOGS;
 +  }
 +  
 +  public static void addLogEntry(TCredentials credentials, LogEntry entry, ZooLock zooLock) {
 +    if (entry.extent.isRootTablet()) {
 +      String root = getZookeeperLogLocation();
 +      while (true) {
 +        try {
 +          IZooReaderWriter zoo = ZooReaderWriter.getInstance();
 +          if (zoo.isLockHeld(zooLock.getLockID()))
 +            zoo.putPersistentData(root + "/" + entry.filename, entry.toBytes(), NodeExistsPolicy.OVERWRITE);
 +          break;
 +        } catch (KeeperException e) {
 +          log.error(e, e);
 +        } catch (InterruptedException e) {
 +          log.error(e, e);
 +        } catch (IOException e) {
 +          log.error(e, e);
 +        }
 +        UtilWaitThread.sleep(1000);
 +      }
 +    } else {
 +      String value = StringUtil.join(entry.logSet, ";") + "|" + entry.tabletId;
 +      Mutation m = new Mutation(entry.extent.getMetadataEntry());
 +      m.put(Constants.METADATA_LOG_COLUMN_FAMILY, new Text(entry.server + "/" + entry.filename), new Value(value.getBytes()));
 +      update(credentials, zooLock, m);
 +    }
 +  }
 +  
 +  public static LogEntry entryFromKeyValue(Key key, Value value) {
 +    MetadataTable.LogEntry e = new MetadataTable.LogEntry();
 +    e.extent = new KeyExtent(key.getRow(), EMPTY_TEXT);
 +    String[] parts = key.getColumnQualifier().toString().split("/");
 +    e.server = parts[0];
 +    e.filename = parts[1];
 +    parts = value.toString().split("\\|");
 +    e.tabletId = Integer.parseInt(parts[1]);
 +    e.logSet = Arrays.asList(parts[0].split(";"));
 +    e.timestamp = key.getTimestamp();
 +    return e;
 +  }
 +  
 +  public static Pair<List<LogEntry>,SortedMap<String,DataFileValue>> getFileAndLogEntries(TCredentials credentials, KeyExtent extent) throws KeeperException,
 +      InterruptedException, IOException {
 +    ArrayList<LogEntry> result = new ArrayList<LogEntry>();
 +    TreeMap<String,DataFileValue> sizes = new TreeMap<String,DataFileValue>();
 +    
 +    if (extent.isRootTablet()) {
 +      getRootLogEntries(result);
 +      FileSystem fs = TraceFileSystem.wrap(FileUtil.getFileSystem(CachedConfiguration.getInstance(), ServerConfiguration.getSiteConfiguration()));
 +      FileStatus[] files = fs.listStatus(new Path(ServerConstants.getRootTabletDir()));
 +      
 +      for (FileStatus fileStatus : files) {
 +        if (fileStatus.getPath().toString().endsWith("_tmp")) {
 +          continue;
 +        }
 +        DataFileValue dfv = new DataFileValue(0, 0);
 +        sizes.put(Constants.ZROOT_TABLET + "/" + fileStatus.getPath().getName(), dfv);
 +      }
 +      
 +    } else {
 +      Scanner scanner = new ScannerImpl(HdfsZooInstance.getInstance(), credentials, Constants.METADATA_TABLE_ID, Constants.NO_AUTHS);
 +      scanner.fetchColumnFamily(Constants.METADATA_LOG_COLUMN_FAMILY);
 +      scanner.fetchColumnFamily(Constants.METADATA_DATAFILE_COLUMN_FAMILY);
 +      scanner.setRange(extent.toMetadataRange());
 +      
 +      for (Entry<Key,Value> entry : scanner) {
 +        if (!entry.getKey().getRow().equals(extent.getMetadataEntry())) {
 +          throw new RuntimeException("Unexpected row " + entry.getKey().getRow() + " expected " + extent.getMetadataEntry());
 +        }
 +        
 +        if (entry.getKey().getColumnFamily().equals(Constants.METADATA_LOG_COLUMN_FAMILY)) {
 +          result.add(entryFromKeyValue(entry.getKey(), entry.getValue()));
 +        } else if (entry.getKey().getColumnFamily().equals(Constants.METADATA_DATAFILE_COLUMN_FAMILY)) {
 +          DataFileValue dfv = new DataFileValue(entry.getValue().get());
 +          sizes.put(entry.getKey().getColumnQualifier().toString(), dfv);
 +        } else {
 +          throw new RuntimeException("Unexpected col fam " + entry.getKey().getColumnFamily());
 +        }
 +      }
 +    }
 +    
 +    return new Pair<List<LogEntry>,SortedMap<String,DataFileValue>>(result, sizes);
 +  }
 +  
 +  public static List<LogEntry> getLogEntries(TCredentials credentials, KeyExtent extent) throws IOException, KeeperException, InterruptedException {
 +    log.info("Scanning logging entries for " + extent);
 +    ArrayList<LogEntry> result = new ArrayList<LogEntry>();
 +    if (extent.equals(Constants.ROOT_TABLET_EXTENT)) {
 +      log.info("Getting logs for root tablet from zookeeper");
 +      getRootLogEntries(result);
 +    } else {
 +      log.info("Scanning metadata for logs used for tablet " + extent);
 +      Scanner scanner = getTabletLogScanner(credentials, extent);
 +      Text pattern = extent.getMetadataEntry();
 +      for (Entry<Key,Value> entry : scanner) {
 +        Text row = entry.getKey().getRow();
 +        if (entry.getKey().getColumnFamily().equals(Constants.METADATA_LOG_COLUMN_FAMILY)) {
 +          if (row.equals(pattern)) {
 +            result.add(entryFromKeyValue(entry.getKey(), entry.getValue()));
 +          }
 +        }
 +      }
 +    }
 +    
 +    Collections.sort(result, new Comparator<LogEntry>() {
 +      @Override
 +      public int compare(LogEntry o1, LogEntry o2) {
 +        long diff = o1.timestamp - o2.timestamp;
 +        if (diff < 0)
 +          return -1;
 +        if (diff > 0)
 +          return 1;
 +        return 0;
 +      }
 +    });
 +    log.info("Returning logs " + result + " for extent " + extent);
 +    return result;
 +  }
 +  
 +  private static void getRootLogEntries(ArrayList<LogEntry> result) throws KeeperException, InterruptedException, IOException {
 +    IZooReaderWriter zoo = ZooReaderWriter.getInstance();
 +    String root = getZookeeperLogLocation();
 +    // there's a little race between getting the children and fetching 
 +    // the data.  The log can be removed in between.
 +    while (true) {
 +      result.clear();
 +      for (String child : zoo.getChildren(root)) {
 +        LogEntry e = new LogEntry();
 +        try {
 +          e.fromBytes(zoo.getData(root + "/" + child, null));
 +          result.add(e);
 +        } catch (KeeperException.NoNodeException ex) {
 +          continue;
 +        }
 +      }
 +      break;
 +    }
 +  }
 +  
 +  private static Scanner getTabletLogScanner(TCredentials credentials, KeyExtent extent) {
 +    Scanner scanner = new ScannerImpl(HdfsZooInstance.getInstance(), credentials, Constants.METADATA_TABLE_ID, Constants.NO_AUTHS);
 +    scanner.fetchColumnFamily(Constants.METADATA_LOG_COLUMN_FAMILY);
 +    Text start = extent.getMetadataEntry();
 +    Key endKey = new Key(start, Constants.METADATA_LOG_COLUMN_FAMILY);
 +    endKey = endKey.followingKey(PartialKey.ROW_COLFAM);
 +    scanner.setRange(new Range(new Key(start), endKey));
 +    return scanner;
 +  }
 +  
 +  static class LogEntryIterator implements Iterator<LogEntry> {
 +    
 +    Iterator<LogEntry> rootTabletEntries = null;
 +    Iterator<Entry<Key,Value>> metadataEntries = null;
 +    
 +    LogEntryIterator(TCredentials creds) throws IOException, KeeperException, InterruptedException {
 +      rootTabletEntries = getLogEntries(creds, Constants.ROOT_TABLET_EXTENT).iterator();
 +      try {
 +        Scanner scanner = HdfsZooInstance.getInstance().getConnector(creds.getPrincipal(), CredentialHelper.extractToken(creds))
 +            .createScanner(Constants.METADATA_TABLE_NAME, Constants.NO_AUTHS);
 +        scanner.fetchColumnFamily(Constants.METADATA_LOG_COLUMN_FAMILY);
 +        metadataEntries = scanner.iterator();
 +      } catch (Exception ex) {
 +        throw new IOException(ex);
 +      }
 +    }
 +    
 +    @Override
 +    public boolean hasNext() {
 +      return rootTabletEntries.hasNext() || metadataEntries.hasNext();
 +    }
 +    
 +    @Override
 +    public LogEntry next() {
 +      if (rootTabletEntries.hasNext()) {
 +        return rootTabletEntries.next();
 +      }
 +      Entry<Key,Value> entry = metadataEntries.next();
 +      return entryFromKeyValue(entry.getKey(), entry.getValue());
 +    }
 +    
 +    @Override
 +    public void remove() {
 +      throw new UnsupportedOperationException();
 +    }
 +  }
 +  
 +  public static Iterator<LogEntry> getLogEntries(TCredentials creds) throws IOException, KeeperException, InterruptedException {
 +    return new LogEntryIterator(creds);
 +  }
 +  
 +  public static void removeUnusedWALEntries(KeyExtent extent, List<LogEntry> logEntries, ZooLock zooLock) {
-     for (LogEntry entry : logEntries) {
-       if (entry.extent.isRootTablet()) {
++      if (extent.isRootTablet()) {
++        for (LogEntry entry : logEntries) {
 +        String root = getZookeeperLogLocation();
 +        while (true) {
 +          try {
 +            IZooReaderWriter zoo = ZooReaderWriter.getInstance();
 +            if (zoo.isLockHeld(zooLock.getLockID()))
 +              zoo.recursiveDelete(root + "/" + entry.filename, NodeMissingPolicy.SKIP);
 +            break;
 +          } catch (Exception e) {
 +            log.error(e, e);
 +          }
 +          UtilWaitThread.sleep(1000);
 +        }
++        }
 +      } else {
-         Mutation m = new Mutation(entry.extent.getMetadataEntry());
-         m.putDelete(Constants.METADATA_LOG_COLUMN_FAMILY, new Text(entry.server + "/" + entry.filename));
++        Mutation m = new Mutation(extent.getMetadataEntry());
++        for (LogEntry entry : logEntries) {
++          m.putDelete(Constants.METADATA_LOG_COLUMN_FAMILY, new Text(entry.server + "/" + entry.filename));
++        }
 +        update(SecurityConstants.getSystemCredentials(), zooLock, m);
-       }
 +    }
 +  }
 +  
 +  private static void getFiles(Set<String> files, Map<Key,Value> tablet, String srcTableId) {
 +    for (Entry<Key,Value> entry : tablet.entrySet()) {
 +      if (entry.getKey().getColumnFamily().equals(Constants.METADATA_DATAFILE_COLUMN_FAMILY)) {
 +        String cf = entry.getKey().getColumnQualifier().toString();
 +        if (srcTableId != null && !cf.startsWith("../"))
 +          cf = "../" + srcTableId + entry.getKey().getColumnQualifier();
 +        files.add(cf);
 +      }
 +    }
 +  }
 +  
 +  private static Mutation createCloneMutation(String srcTableId, String tableId, Map<Key,Value> tablet) {
 +    
 +    KeyExtent ke = new KeyExtent(tablet.keySet().iterator().next().getRow(), (Text) null);
 +    Mutation m = new Mutation(KeyExtent.getMetadataEntry(new Text(tableId), ke.getEndRow()));
 +    
 +    for (Entry<Key,Value> entry : tablet.entrySet()) {
 +      if (entry.getKey().getColumnFamily().equals(Constants.METADATA_DATAFILE_COLUMN_FAMILY)) {
 +        String cf = entry.getKey().getColumnQualifier().toString();
 +        if (!cf.startsWith("../"))
 +          cf = "../" + srcTableId + entry.getKey().getColumnQualifier();
 +        m.put(entry.getKey().getColumnFamily(), new Text(cf), entry.getValue());
 +      } else if (entry.getKey().getColumnFamily().equals(Constants.METADATA_CURRENT_LOCATION_COLUMN_FAMILY)) {
 +        m.put(Constants.METADATA_LAST_LOCATION_COLUMN_FAMILY, entry.getKey().getColumnQualifier(), entry.getValue());
 +      } else if (entry.getKey().getColumnFamily().equals(Constants.METADATA_LAST_LOCATION_COLUMN_FAMILY)) {
 +        // skip
 +      } else {
 +        m.put(entry.getKey().getColumnFamily(), entry.getKey().getColumnQualifier(), entry.getValue());
 +      }
 +    }
 +    return m;
 +  }
 +  
 +  private static Scanner createCloneScanner(String tableId, Connector conn) throws TableNotFoundException {
 +    Scanner mscanner = new IsolatedScanner(conn.createScanner(Constants.METADATA_TABLE_NAME, Constants.NO_AUTHS));
 +    mscanner.setRange(new KeyExtent(new Text(tableId), null, null).toMetadataRange());
 +    mscanner.fetchColumnFamily(Constants.METADATA_DATAFILE_COLUMN_FAMILY);
 +    mscanner.fetchColumnFamily(Constants.METADATA_CURRENT_LOCATION_COLUMN_FAMILY);
 +    mscanner.fetchColumnFamily(Constants.METADATA_LAST_LOCATION_COLUMN_FAMILY);
 +    mscanner.fetchColumnFamily(Constants.METADATA_CLONED_COLUMN_FAMILY);
 +    Constants.METADATA_PREV_ROW_COLUMN.fetch(mscanner);
 +    Constants.METADATA_TIME_COLUMN.fetch(mscanner);
 +    return mscanner;
 +  }
 +  
 +  static void initializeClone(String srcTableId, String tableId, Connector conn, BatchWriter bw) throws TableNotFoundException, MutationsRejectedException {
 +    TabletIterator ti = new TabletIterator(createCloneScanner(srcTableId, conn), new KeyExtent(new Text(srcTableId), null, null).toMetadataRange(), true, true);
 +    
 +    if (!ti.hasNext())
 +      throw new RuntimeException(" table deleted during clone?  srcTableId = " + srcTableId);
 +    
 +    while (ti.hasNext())
 +      bw.addMutation(createCloneMutation(srcTableId, tableId, ti.next()));
 +    
 +    bw.flush();
 +  }
 +  
 +  static int compareEndRows(Text endRow1, Text endRow2) {
 +    return new KeyExtent(new Text("0"), endRow1, null).compareTo(new KeyExtent(new Text("0"), endRow2, null));
 +  }
 +  
 +  static int checkClone(String srcTableId, String tableId, Connector conn, BatchWriter bw) throws TableNotFoundException, MutationsRejectedException {
 +    TabletIterator srcIter = new TabletIterator(createCloneScanner(srcTableId, conn), new KeyExtent(new Text(srcTableId), null, null).toMetadataRange(), true,
 +        true);
 +    TabletIterator cloneIter = new TabletIterator(createCloneScanner(tableId, conn), new KeyExtent(new Text(tableId), null, null).toMetadataRange(), true, true);
 +    
 +    if (!cloneIter.hasNext() || !srcIter.hasNext())
 +      throw new RuntimeException(" table deleted during clone?  srcTableId = " + srcTableId + " tableId=" + tableId);
 +    
 +    int rewrites = 0;
 +    
 +    while (cloneIter.hasNext()) {
 +      Map<Key,Value> cloneTablet = cloneIter.next();
 +      Text cloneEndRow = new KeyExtent(cloneTablet.keySet().iterator().next().getRow(), (Text) null).getEndRow();
 +      HashSet<String> cloneFiles = new HashSet<String>();
 +      
 +      boolean cloneSuccessful = false;
 +      for (Entry<Key,Value> entry : cloneTablet.entrySet()) {
 +        if (entry.getKey().getColumnFamily().equals(Constants.METADATA_CLONED_COLUMN_FAMILY)) {
 +          cloneSuccessful = true;
 +          break;
 +        }
 +      }
 +      
 +      if (!cloneSuccessful)
 +        getFiles(cloneFiles, cloneTablet, null);
 +      
 +      List<Map<Key,Value>> srcTablets = new ArrayList<Map<Key,Value>>();
 +      Map<Key,Value> srcTablet = srcIter.next();
 +      srcTablets.add(srcTablet);
 +      
 +      Text srcEndRow = new KeyExtent(srcTablet.keySet().iterator().next().getRow(), (Text) null).getEndRow();
 +      
 +      int cmp = compareEndRows(cloneEndRow, srcEndRow);
 +      if (cmp < 0)
 +        throw new TabletIterator.TabletDeletedException("Tablets deleted from src during clone : " + cloneEndRow + " " + srcEndRow);
 +      
 +      HashSet<String> srcFiles = new HashSet<String>();
 +      if (!cloneSuccessful)
 +        getFiles(srcFiles, srcTablet, srcTableId);
 +      
 +      while (cmp > 0) {
 +        srcTablet = srcIter.next();
 +        srcTablets.add(srcTablet);
 +        srcEndRow = new KeyExtent(srcTablet.keySet().iterator().next().getRow(), (Text) null).getEndRow();
 +        cmp = compareEndRows(cloneEndRow, srcEndRow);
 +        if (cmp < 0)
 +          throw new TabletIterator.TabletDeletedException("Tablets deleted from src during clone : " + cloneEndRow + " " + srcEndRow);
 +        
 +        if (!cloneSuccessful)
 +          getFiles(srcFiles, srcTablet, srcTableId);
 +      }
 +      
 +      if (cloneSuccessful)
 +        continue;
 +      
 +      if (!srcFiles.containsAll(cloneFiles)) {
 +        // delete existing cloned tablet entry
 +        Mutation m = new Mutation(cloneTablet.keySet().iterator().next().getRow());
 +        
 +        for (Entry<Key,Value> entry : cloneTablet.entrySet()) {
 +          Key k = entry.getKey();
 +          m.putDelete(k.getColumnFamily(), k.getColumnQualifier(), k.getTimestamp());
 +        }
 +        
 +        bw.addMutation(m);
 +        
 +        for (Map<Key,Value> st : srcTablets)
 +          bw.addMutation(createCloneMutation(srcTableId, tableId, st));
 +        
 +        rewrites++;
 +      } else {
 +        // write out marker that this tablet was successfully cloned
 +        Mutation m = new Mutation(cloneTablet.keySet().iterator().next().getRow());
 +        m.put(Constants.METADATA_CLONED_COLUMN_FAMILY, new Text(""), new Value("OK".getBytes()));
 +        bw.addMutation(m);
 +      }
 +    }
 +    
 +    bw.flush();
 +    return rewrites;
 +  }
 +  
 +  public static void cloneTable(Instance instance, String srcTableId, String tableId) throws Exception {
 +    
 +    Connector conn = instance.getConnector(SecurityConstants.SYSTEM_PRINCIPAL, SecurityConstants.getSystemToken());
 +    BatchWriter bw = conn.createBatchWriter(Constants.METADATA_TABLE_NAME, new BatchWriterConfig());
 +    
 +    while (true) {
 +      
 +      try {
 +        initializeClone(srcTableId, tableId, conn, bw);
 +        
 +        // the following loop looks changes in the file that occurred during the copy.. if files were dereferenced then they could have been GCed
 +        
 +        while (true) {
 +          int rewrites = checkClone(srcTableId, tableId, conn, bw);
 +          
 +          if (rewrites == 0)
 +            break;
 +        }
 +        
 +        bw.flush();
 +        break;
 +        
 +      } catch (TabletIterator.TabletDeletedException tde) {
 +        // tablets were merged in the src table
 +        bw.flush();
 +        
 +        // delete what we have cloned and try again
 +        deleteTable(tableId, false, SecurityConstants.getSystemCredentials(), null);
 +        
 +        log.debug("Tablets merged in table " + srcTableId + " while attempting to clone, trying again");
 +        
 +        UtilWaitThread.sleep(100);
 +      }
 +    }
 +    
 +    // delete the clone markers and create directory entries
 +    Scanner mscanner = conn.createScanner(Constants.METADATA_TABLE_NAME, Constants.NO_AUTHS);
 +    mscanner.setRange(new KeyExtent(new Text(tableId), null, null).toMetadataRange());
 +    mscanner.fetchColumnFamily(Constants.METADATA_CLONED_COLUMN_FAMILY);
 +    
 +    int dirCount = 0;
 +    
 +    for (Entry<Key,Value> entry : mscanner) {
 +      Key k = entry.getKey();
 +      Mutation m = new Mutation(k.getRow());
 +      m.putDelete(k.getColumnFamily(), k.getColumnQualifier());
 +      Constants.METADATA_DIRECTORY_COLUMN.put(m, new Value(FastFormat.toZeroPaddedString(dirCount++, 8, 16, "/c-".getBytes())));
 +      bw.addMutation(m);
 +    }
 +    
 +    bw.close();
 +    
 +  }
 +  
 +  public static void chopped(KeyExtent extent, ZooLock zooLock) {
 +    Mutation m = new Mutation(extent.getMetadataEntry());
 +    Constants.METADATA_CHOPPED_COLUMN.put(m, new Value("chopped".getBytes()));
 +    update(SecurityConstants.getSystemCredentials(), zooLock, m);
 +  }
 +  
 +  public static void removeBulkLoadEntries(Connector conn, String tableId, long tid) throws Exception {
 +    Scanner mscanner = new IsolatedScanner(conn.createScanner(Constants.METADATA_TABLE_NAME, Constants.NO_AUTHS));
 +    mscanner.setRange(new KeyExtent(new Text(tableId), null, null).toMetadataRange());
 +    mscanner.fetchColumnFamily(Constants.METADATA_BULKFILE_COLUMN_FAMILY);
 +    BatchWriter bw = conn.createBatchWriter(Constants.METADATA_TABLE_NAME, new BatchWriterConfig());
 +    for (Entry<Key,Value> entry : mscanner) {
 +      log.debug("Looking at entry " + entry + " with tid " + tid);
 +      if (Long.parseLong(entry.getValue().toString()) == tid) {
 +        log.debug("deleting entry " + entry);
 +        Mutation m = new Mutation(entry.getKey().getRow());
 +        m.putDelete(entry.getKey().getColumnFamily(), entry.getKey().getColumnQualifier());
 +        bw.addMutation(m);
 +      }
 +    }
 +    bw.close();
 +  }
 +  
 +  public static List<String> getBulkFilesLoaded(Connector conn, KeyExtent extent, long tid) {
 +    List<String> result = new ArrayList<String>();
 +    try {
 +      Scanner mscanner = new IsolatedScanner(conn.createScanner(Constants.METADATA_TABLE_NAME, Constants.NO_AUTHS));
 +      mscanner.setRange(extent.toMetadataRange());
 +      mscanner.fetchColumnFamily(Constants.METADATA_BULKFILE_COLUMN_FAMILY);
 +      for (Entry<Key,Value> entry : mscanner) {
 +        if (Long.parseLong(entry.getValue().toString()) == tid) {
 +          result.add(entry.getKey().getColumnQualifier().toString());
 +        }
 +      }
 +      return result;
 +    } catch (TableNotFoundException ex) {
 +      // unlikely
 +      throw new RuntimeException("Onos! teh metadata table has vanished!!");
 +    }
 +  }
 +  
 +  public static Map<String,Long> getBulkFilesLoaded(TCredentials credentials, KeyExtent extent) {
 +    return getBulkFilesLoaded(credentials, extent.getMetadataEntry());
 +  }
 +  
 +  public static Map<String,Long> getBulkFilesLoaded(TCredentials credentials, Text metadataRow) {
 +    
 +    Map<String,Long> ret = new HashMap<String,Long>();
 +    
 +    Scanner scanner = new ScannerImpl(HdfsZooInstance.getInstance(), credentials, Constants.METADATA_TABLE_ID, Constants.NO_AUTHS);
 +    scanner.setRange(new Range(metadataRow));
 +    scanner.fetchColumnFamily(Constants.METADATA_BULKFILE_COLUMN_FAMILY);
 +    for (Entry<Key,Value> entry : scanner) {
 +      String file = entry.getKey().getColumnQualifier().toString();
 +      Long tid = Long.parseLong(entry.getValue().toString());
 +      
 +      ret.put(file, tid);
 +    }
 +    return ret;
 +  }
 +  
 +  public static void addBulkLoadInProgressFlag(String path) {
 +    
 +    Mutation m = new Mutation(Constants.METADATA_BLIP_FLAG_PREFIX + path);
 +    m.put(EMPTY_TEXT, EMPTY_TEXT, new Value(new byte[] {}));
 +    
 +    update(SecurityConstants.getSystemCredentials(), m);
 +  }
 +  
 +  public static void removeBulkLoadInProgressFlag(String path) {
 +    
 +    Mutation m = new Mutation(Constants.METADATA_BLIP_FLAG_PREFIX + path);
 +    m.putDelete(EMPTY_TEXT, EMPTY_TEXT);
 +    
 +    update(SecurityConstants.getSystemCredentials(), m);
 +  }
 +
 +  public static void moveMetaDeleteMarkers(Instance instance, TCredentials creds) {
 +    // move delete markers from the normal delete keyspace to the root tablet delete keyspace if the files are for the !METADATA table
 +    Scanner scanner = new ScannerImpl(instance, creds, Constants.METADATA_TABLE_ID, Constants.NO_AUTHS);
 +    scanner.setRange(new Range(Constants.METADATA_DELETES_KEYSPACE));
 +    for (Entry<Key,Value> entry : scanner) {
 +      String row = entry.getKey().getRow().toString();
 +      if (row.startsWith(Constants.METADATA_DELETE_FLAG_PREFIX + "/" + Constants.METADATA_TABLE_ID)) {
 +        String filename = row.substring(Constants.METADATA_DELETE_FLAG_PREFIX.length());
 +        // add the new entry first
 +        log.info("Moving " + filename + " marker to the root tablet");
 +        Mutation m = new Mutation(Constants.METADATA_DELETE_FLAG_FOR_METADATA_PREFIX + filename);
 +        m.put(new byte[]{}, new byte[]{}, new byte[]{});
 +        update(creds, m);
 +        // remove the old entry
 +        m = new Mutation(entry.getKey().getRow());
 +        m.putDelete(new byte[]{}, new byte[]{});
 +        update(creds, m);
 +      } else {
 +        break;
 +      }
 +    }
 +    
 +  }
 +}


[6/8] git commit: ACCUMULO-1914 make WALog cleanup after recovery an atomic mutation

Posted by ec...@apache.org.
ACCUMULO-1914 make WALog cleanup after recovery an atomic mutation


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

Branch: refs/heads/master
Commit: fe46a60c40677df0909e400eda29c79e04f0ead9
Parents: efb4246 041270b
Author: Eric Newton <er...@gmail.com>
Authored: Thu Nov 21 12:45:13 2013 -0500
Committer: Eric Newton <er...@gmail.com>
Committed: Thu Nov 21 12:45:13 2013 -0500

----------------------------------------------------------------------
 .../accumulo/server/util/MetadataTableUtil.java | 99 +++-----------------
 1 file changed, 12 insertions(+), 87 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/fe46a60c/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java
----------------------------------------------------------------------
diff --cc server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java
index 76ea3b1,0000000..a3b8011
mode 100644,000000..100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java
@@@ -1,1034 -1,0 +1,959 @@@
 +/*
 + * 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.io.IOException;
 +import java.util.ArrayList;
 +import java.util.Collections;
 +import java.util.Comparator;
 +import java.util.HashMap;
 +import java.util.HashSet;
 +import java.util.Iterator;
 +import java.util.List;
 +import java.util.Map;
 +import java.util.Map.Entry;
 +import java.util.Set;
 +import java.util.SortedMap;
 +import java.util.TreeMap;
 +import java.util.concurrent.TimeUnit;
 +
 +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.BatchWriter;
 +import org.apache.accumulo.core.client.BatchWriterConfig;
 +import org.apache.accumulo.core.client.Connector;
 +import org.apache.accumulo.core.client.Instance;
 +import org.apache.accumulo.core.client.IsolatedScanner;
 +import org.apache.accumulo.core.client.MutationsRejectedException;
 +import org.apache.accumulo.core.client.Scanner;
 +import org.apache.accumulo.core.client.TableNotFoundException;
 +import org.apache.accumulo.core.client.impl.BatchWriterImpl;
 +import org.apache.accumulo.core.client.impl.ScannerImpl;
 +import org.apache.accumulo.core.client.impl.Writer;
 +import org.apache.accumulo.core.data.Key;
 +import org.apache.accumulo.core.data.KeyExtent;
 +import org.apache.accumulo.core.data.Mutation;
 +import org.apache.accumulo.core.data.PartialKey;
 +import org.apache.accumulo.core.data.Range;
 +import org.apache.accumulo.core.data.Value;
- import org.apache.accumulo.core.master.state.tables.TableState;
 +import org.apache.accumulo.core.metadata.MetadataTable;
 +import org.apache.accumulo.core.metadata.RootTable;
 +import org.apache.accumulo.core.metadata.schema.DataFileValue;
 +import org.apache.accumulo.core.metadata.schema.MetadataSchema;
 +import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
 +import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ChoppedColumnFamily;
 +import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ClonedColumnFamily;
 +import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
 +import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LogColumnFamily;
 +import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ScanFileColumnFamily;
 +import org.apache.accumulo.core.security.Authorizations;
 +import org.apache.accumulo.core.security.Credentials;
 +import org.apache.accumulo.core.tabletserver.log.LogEntry;
 +import org.apache.accumulo.core.tabletserver.thrift.ConstraintViolationException;
 +import org.apache.accumulo.core.util.ColumnFQ;
 +import org.apache.accumulo.core.util.FastFormat;
 +import org.apache.accumulo.core.util.Pair;
 +import org.apache.accumulo.core.util.UtilWaitThread;
 +import org.apache.accumulo.core.zookeeper.ZooUtil;
 +import org.apache.accumulo.fate.zookeeper.IZooReaderWriter;
 +import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
 +import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy;
 +import org.apache.accumulo.server.ServerConstants;
 +import org.apache.accumulo.server.client.HdfsZooInstance;
 +import org.apache.accumulo.server.fs.FileRef;
 +import org.apache.accumulo.server.fs.VolumeManager;
 +import org.apache.accumulo.server.fs.VolumeManager.FileType;
 +import org.apache.accumulo.server.fs.VolumeManagerImpl;
 +import org.apache.accumulo.server.security.SystemCredentials;
- import org.apache.accumulo.server.tables.TableManager;
 +import org.apache.accumulo.server.zookeeper.ZooLock;
 +import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 +import org.apache.hadoop.fs.FileStatus;
 +import org.apache.hadoop.fs.Path;
 +import org.apache.hadoop.io.Text;
 +import org.apache.log4j.Logger;
 +import org.apache.zookeeper.KeeperException;
 +
 +/**
 + * provides a reference to the metadata table for updates by tablet servers
 + */
 +public class MetadataTableUtil {
 +
 +  private static final Text EMPTY_TEXT = new Text();
 +  private static Map<Credentials,Writer> root_tables = new HashMap<Credentials,Writer>();
 +  private static Map<Credentials,Writer> metadata_tables = new HashMap<Credentials,Writer>();
 +  private static final Logger log = Logger.getLogger(MetadataTableUtil.class);
 +
-   private static final int SAVE_ROOT_TABLET_RETRIES = 3;
- 
 +  private MetadataTableUtil() {}
 +
 +  public synchronized static Writer getMetadataTable(Credentials credentials) {
 +    Writer metadataTable = metadata_tables.get(credentials);
 +    if (metadataTable == null) {
 +      metadataTable = new Writer(HdfsZooInstance.getInstance(), credentials, MetadataTable.ID);
 +      metadata_tables.put(credentials, metadataTable);
 +    }
 +    return metadataTable;
 +  }
 +
-   public synchronized static Writer getRootTable(Credentials credentials) {
++  private synchronized static Writer getRootTable(Credentials credentials) {
 +    Writer rootTable = root_tables.get(credentials);
 +    if (rootTable == null) {
 +      rootTable = new Writer(HdfsZooInstance.getInstance(), credentials, RootTable.ID);
 +      root_tables.put(credentials, rootTable);
 +    }
 +    return rootTable;
 +  }
 +
-   public static void putLockID(ZooLock zooLock, Mutation m) {
++  private static void putLockID(ZooLock zooLock, Mutation m) {
 +    TabletsSection.ServerColumnFamily.LOCK_COLUMN.put(m, new Value(zooLock.getLockID().serialize(ZooUtil.getRoot(HdfsZooInstance.getInstance()) + "/")
 +        .getBytes()));
 +  }
 +
-   public static void update(Credentials credentials, Mutation m, KeyExtent extent) {
++  private static void update(Credentials credentials, Mutation m, KeyExtent extent) {
 +    update(credentials, null, m, extent);
 +  }
 +
 +  public static void update(Credentials credentials, ZooLock zooLock, Mutation m, KeyExtent extent) {
 +    Writer t = extent.isMeta() ? getRootTable(credentials) : getMetadataTable(credentials);
 +    if (zooLock != null)
 +      putLockID(zooLock, m);
 +    while (true) {
 +      try {
 +        t.update(m);
 +        return;
 +      } catch (AccumuloException e) {
 +        log.error(e, e);
 +      } catch (AccumuloSecurityException e) {
 +        log.error(e, e);
 +      } catch (ConstraintViolationException e) {
 +        log.error(e, e);
 +      } catch (TableNotFoundException e) {
 +        log.error(e, e);
 +      }
 +      UtilWaitThread.sleep(1000);
 +    }
 +
 +  }
 +
 +  public static void updateTabletFlushID(KeyExtent extent, long flushID, Credentials credentials, ZooLock zooLock) {
 +    if (!extent.isRootTablet()) {
 +      Mutation m = new Mutation(extent.getMetadataEntry());
 +      TabletsSection.ServerColumnFamily.FLUSH_COLUMN.put(m, new Value((flushID + "").getBytes()));
 +      update(credentials, zooLock, m, extent);
 +    }
 +  }
 +
 +  public static void updateTabletCompactID(KeyExtent extent, long compactID, Credentials credentials, ZooLock zooLock) {
 +    if (!extent.isRootTablet()) {
 +      Mutation m = new Mutation(extent.getMetadataEntry());
 +      TabletsSection.ServerColumnFamily.COMPACT_COLUMN.put(m, new Value((compactID + "").getBytes()));
 +      update(credentials, zooLock, m, extent);
 +    }
 +  }
 +
 +  public static void updateTabletDataFile(long tid, KeyExtent extent, Map<FileRef,DataFileValue> estSizes, String time, Credentials credentials, ZooLock zooLock) {
 +    Mutation m = new Mutation(extent.getMetadataEntry());
 +    byte[] tidBytes = Long.toString(tid).getBytes();
 +
 +    for (Entry<FileRef,DataFileValue> entry : estSizes.entrySet()) {
 +      Text file = entry.getKey().meta();
 +      m.put(DataFileColumnFamily.NAME, file, new Value(entry.getValue().encode()));
 +      m.put(TabletsSection.BulkFileColumnFamily.NAME, file, new Value(tidBytes));
 +    }
 +    TabletsSection.ServerColumnFamily.TIME_COLUMN.put(m, new Value(time.getBytes()));
 +    update(credentials, zooLock, m, extent);
 +  }
 +
 +  public static void addTablet(KeyExtent extent, String path, Credentials credentials, char timeType, ZooLock lock) {
 +    Mutation m = extent.getPrevRowUpdateMutation();
 +
 +    TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN.put(m, new Value(path.getBytes()));
 +    TabletsSection.ServerColumnFamily.TIME_COLUMN.put(m, new Value((timeType + "0").getBytes()));
 +
 +    update(credentials, lock, m, extent);
 +  }
 +
 +  public static void updateTabletPrevEndRow(KeyExtent extent, Credentials credentials) {
 +    Mutation m = extent.getPrevRowUpdateMutation(); //
 +    update(credentials, m, extent);
 +  }
 +
-   /**
-    * convenience method for reading entries from the metadata table
-    */
-   public static SortedMap<KeyExtent,Text> getMetadataDirectoryEntries(SortedMap<Key,Value> entries) {
-     Key key;
-     Value val;
-     Text datafile = null;
-     Value prevRow = null;
-     KeyExtent ke;
- 
-     SortedMap<KeyExtent,Text> results = new TreeMap<KeyExtent,Text>();
- 
-     Text lastRowFromKey = new Text();
- 
-     // text obj below is meant to be reused in loop for efficiency
-     Text colf = new Text();
-     Text colq = new Text();
- 
-     for (Entry<Key,Value> entry : entries.entrySet()) {
-       key = entry.getKey();
-       val = entry.getValue();
- 
-       if (key.compareRow(lastRowFromKey) != 0) {
-         prevRow = null;
-         datafile = null;
-         key.getRow(lastRowFromKey);
-       }
- 
-       colf = key.getColumnFamily(colf);
-       colq = key.getColumnQualifier(colq);
- 
-       // interpret the row id as a key extent
-       if (TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN.equals(colf, colq))
-         datafile = new Text(val.toString());
- 
-       else if (TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN.equals(colf, colq))
-         prevRow = new Value(val);
- 
-       if (datafile != null && prevRow != null) {
-         ke = new KeyExtent(key.getRow(), prevRow);
-         results.put(ke, datafile);
- 
-         datafile = null;
-         prevRow = null;
-       }
-     }
-     return results;
-   }
- 
-   public static boolean recordRootTabletLocation(String address) {
-     IZooReaderWriter zoo = ZooReaderWriter.getInstance();
-     for (int i = 0; i < SAVE_ROOT_TABLET_RETRIES; i++) {
-       try {
-         log.info("trying to write root tablet location to ZooKeeper as " + address);
-         String zRootLocPath = ZooUtil.getRoot(HdfsZooInstance.getInstance()) + RootTable.ZROOT_TABLET_LOCATION;
-         zoo.putPersistentData(zRootLocPath, address.getBytes(), NodeExistsPolicy.OVERWRITE);
-         return true;
-       } catch (Exception e) {
-         log.error("Master: unable to save root tablet location in zookeeper. exception: " + e, e);
-       }
-     }
-     log.error("Giving up after " + SAVE_ROOT_TABLET_RETRIES + " retries");
-     return false;
-   }
- 
 +  public static SortedMap<FileRef,DataFileValue> getDataFileSizes(KeyExtent extent, Credentials credentials) throws IOException {
 +    TreeMap<FileRef,DataFileValue> sizes = new TreeMap<FileRef,DataFileValue>();
 +
 +    Scanner mdScanner = new ScannerImpl(HdfsZooInstance.getInstance(), credentials, MetadataTable.ID, Authorizations.EMPTY);
 +    mdScanner.fetchColumnFamily(DataFileColumnFamily.NAME);
 +    Text row = extent.getMetadataEntry();
 +    VolumeManager fs = VolumeManagerImpl.get();
 +
 +    Key endKey = new Key(row, DataFileColumnFamily.NAME, new Text(""));
 +    endKey = endKey.followingKey(PartialKey.ROW_COLFAM);
 +
 +    mdScanner.setRange(new Range(new Key(row), endKey));
 +    for (Entry<Key,Value> entry : mdScanner) {
 +
 +      if (!entry.getKey().getRow().equals(row))
 +        break;
 +      DataFileValue dfv = new DataFileValue(entry.getValue().get());
 +      sizes.put(new FileRef(fs, entry.getKey()), dfv);
 +    }
 +
 +    return sizes;
 +  }
 +
 +  public static void rollBackSplit(Text metadataEntry, Text oldPrevEndRow, Credentials credentials, ZooLock zooLock) {
 +    KeyExtent ke = new KeyExtent(metadataEntry, oldPrevEndRow);
 +    Mutation m = ke.getPrevRowUpdateMutation();
 +    TabletsSection.TabletColumnFamily.SPLIT_RATIO_COLUMN.putDelete(m);
 +    TabletsSection.TabletColumnFamily.OLD_PREV_ROW_COLUMN.putDelete(m);
 +    update(credentials, zooLock, m, new KeyExtent(metadataEntry, (Text) null));
 +  }
 +
 +  public static void splitTablet(KeyExtent extent, Text oldPrevEndRow, double splitRatio, Credentials credentials, ZooLock zooLock) {
 +    Mutation m = extent.getPrevRowUpdateMutation(); //
 +
 +    TabletsSection.TabletColumnFamily.SPLIT_RATIO_COLUMN.put(m, new Value(Double.toString(splitRatio).getBytes()));
 +
 +    TabletsSection.TabletColumnFamily.OLD_PREV_ROW_COLUMN.put(m, KeyExtent.encodePrevEndRow(oldPrevEndRow));
 +    ChoppedColumnFamily.CHOPPED_COLUMN.putDelete(m);
 +    update(credentials, zooLock, m, extent);
 +  }
 +
 +  public static void finishSplit(Text metadataEntry, Map<FileRef,DataFileValue> datafileSizes, List<FileRef> highDatafilesToRemove, Credentials credentials,
 +      ZooLock zooLock) {
 +    Mutation m = new Mutation(metadataEntry);
 +    TabletsSection.TabletColumnFamily.SPLIT_RATIO_COLUMN.putDelete(m);
 +    TabletsSection.TabletColumnFamily.OLD_PREV_ROW_COLUMN.putDelete(m);
 +    ChoppedColumnFamily.CHOPPED_COLUMN.putDelete(m);
 +
 +    for (Entry<FileRef,DataFileValue> entry : datafileSizes.entrySet()) {
 +      m.put(DataFileColumnFamily.NAME, entry.getKey().meta(), new Value(entry.getValue().encode()));
 +    }
 +
 +    for (FileRef pathToRemove : highDatafilesToRemove) {
 +      m.putDelete(DataFileColumnFamily.NAME, pathToRemove.meta());
 +    }
 +
 +    update(credentials, zooLock, m, new KeyExtent(metadataEntry, (Text) null));
 +  }
 +
 +  public static void finishSplit(KeyExtent extent, Map<FileRef,DataFileValue> datafileSizes, List<FileRef> highDatafilesToRemove, Credentials credentials,
 +      ZooLock zooLock) {
 +    finishSplit(extent.getMetadataEntry(), datafileSizes, highDatafilesToRemove, credentials, zooLock);
 +  }
 +
 +  public static void addDeleteEntries(KeyExtent extent, Set<FileRef> datafilesToDelete, Credentials credentials) throws IOException {
 +
 +    String tableId = extent.getTableId().toString();
 +
 +    // TODO could use batch writer,would need to handle failure and retry like update does - ACCUMULO-1294
 +    for (FileRef pathToRemove : datafilesToDelete) {
 +      update(credentials, createDeleteMutation(tableId, pathToRemove.path().toString()), extent);
 +    }
 +  }
 +
 +  public static void addDeleteEntry(String tableId, String path) throws IOException {
 +    update(SystemCredentials.get(), createDeleteMutation(tableId, path), new KeyExtent(new Text(tableId), null, null));
 +  }
 +
 +  public static Mutation createDeleteMutation(String tableId, String pathToRemove) throws IOException {
 +    if (!pathToRemove.contains(":")) {
 +      if (pathToRemove.startsWith("../"))
 +        pathToRemove = pathToRemove.substring(2);
 +      else
 +        pathToRemove = "/" + tableId + pathToRemove;
 +    }
 +
 +    Path path = VolumeManagerImpl.get().getFullPath(FileType.TABLE, pathToRemove);
 +    Mutation delFlag = new Mutation(new Text(MetadataSchema.DeletesSection.getRowPrefix() + path.toString()));
 +    delFlag.put(EMPTY_TEXT, EMPTY_TEXT, new Value(new byte[] {}));
 +    return delFlag;
 +  }
 +
 +  public static void removeScanFiles(KeyExtent extent, Set<FileRef> scanFiles, Credentials credentials, ZooLock zooLock) {
 +    Mutation m = new Mutation(extent.getMetadataEntry());
 +
 +    for (FileRef pathToRemove : scanFiles)
 +      m.putDelete(ScanFileColumnFamily.NAME, pathToRemove.meta());
 +
 +    update(credentials, zooLock, m, extent);
 +  }
 +
 +  public static void splitDatafiles(Text table, Text midRow, double splitRatio, Map<FileRef,FileUtil.FileInfo> firstAndLastRows,
 +      SortedMap<FileRef,DataFileValue> datafiles, SortedMap<FileRef,DataFileValue> lowDatafileSizes, SortedMap<FileRef,DataFileValue> highDatafileSizes,
 +      List<FileRef> highDatafilesToRemove) {
 +
 +    for (Entry<FileRef,DataFileValue> entry : datafiles.entrySet()) {
 +
 +      Text firstRow = null;
 +      Text lastRow = null;
 +
 +      boolean rowsKnown = false;
 +
 +      FileUtil.FileInfo mfi = firstAndLastRows.get(entry.getKey());
 +
 +      if (mfi != null) {
 +        firstRow = mfi.getFirstRow();
 +        lastRow = mfi.getLastRow();
 +        rowsKnown = true;
 +      }
 +
 +      if (rowsKnown && firstRow.compareTo(midRow) > 0) {
 +        // only in high
 +        long highSize = entry.getValue().getSize();
 +        long highEntries = entry.getValue().getNumEntries();
 +        highDatafileSizes.put(entry.getKey(), new DataFileValue(highSize, highEntries, entry.getValue().getTime()));
 +      } else if (rowsKnown && lastRow.compareTo(midRow) <= 0) {
 +        // only in low
 +        long lowSize = entry.getValue().getSize();
 +        long lowEntries = entry.getValue().getNumEntries();
 +        lowDatafileSizes.put(entry.getKey(), new DataFileValue(lowSize, lowEntries, entry.getValue().getTime()));
 +
 +        highDatafilesToRemove.add(entry.getKey());
 +      } else {
 +        long lowSize = (long) Math.floor((entry.getValue().getSize() * splitRatio));
 +        long lowEntries = (long) Math.floor((entry.getValue().getNumEntries() * splitRatio));
 +        lowDatafileSizes.put(entry.getKey(), new DataFileValue(lowSize, lowEntries, entry.getValue().getTime()));
 +
 +        long highSize = (long) Math.ceil((entry.getValue().getSize() * (1.0 - splitRatio)));
 +        long highEntries = (long) Math.ceil((entry.getValue().getNumEntries() * (1.0 - splitRatio)));
 +        highDatafileSizes.put(entry.getKey(), new DataFileValue(highSize, highEntries, entry.getValue().getTime()));
 +      }
 +    }
 +  }
 +
 +  public static void deleteTable(String tableId, boolean insertDeletes, Credentials credentials, ZooLock lock) throws AccumuloException, IOException {
 +    Scanner ms = new ScannerImpl(HdfsZooInstance.getInstance(), credentials, MetadataTable.ID, Authorizations.EMPTY);
 +    Text tableIdText = new Text(tableId);
 +    BatchWriter bw = new BatchWriterImpl(HdfsZooInstance.getInstance(), credentials, MetadataTable.ID, new BatchWriterConfig().setMaxMemory(1000000)
 +        .setMaxLatency(120000l, TimeUnit.MILLISECONDS).setMaxWriteThreads(2));
 +
 +    // scan metadata for our table and delete everything we find
 +    Mutation m = null;
 +    ms.setRange(new KeyExtent(tableIdText, null, null).toMetadataRange());
 +
 +    // insert deletes before deleting data from !METADATA... this makes the code fault tolerant
 +    if (insertDeletes) {
 +
 +      ms.fetchColumnFamily(DataFileColumnFamily.NAME);
 +      TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN.fetch(ms);
 +
 +      for (Entry<Key,Value> cell : ms) {
 +        Key key = cell.getKey();
 +
 +        if (key.getColumnFamily().equals(DataFileColumnFamily.NAME)) {
 +          FileRef ref = new FileRef(VolumeManagerImpl.get(), key);
 +          bw.addMutation(createDeleteMutation(tableId, ref.meta().toString()));
 +        }
 +
 +        if (TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN.hasColumns(key)) {
 +          bw.addMutation(createDeleteMutation(tableId, cell.getValue().toString()));
 +        }
 +      }
 +
 +      bw.flush();
 +
 +      ms.clearColumns();
 +    }
 +
 +    for (Entry<Key,Value> cell : ms) {
 +      Key key = cell.getKey();
 +
 +      if (m == null) {
 +        m = new Mutation(key.getRow());
 +        if (lock != null)
 +          putLockID(lock, m);
 +      }
 +
 +      if (key.getRow().compareTo(m.getRow(), 0, m.getRow().length) != 0) {
 +        bw.addMutation(m);
 +        m = new Mutation(key.getRow());
 +        if (lock != null)
 +          putLockID(lock, m);
 +      }
 +      m.putDelete(key.getColumnFamily(), key.getColumnQualifier());
 +    }
 +
 +    if (m != null)
 +      bw.addMutation(m);
 +
 +    bw.close();
 +  }
 +
 +  static String getZookeeperLogLocation() {
 +    return ZooUtil.getRoot(HdfsZooInstance.getInstance()) + RootTable.ZROOT_TABLET_WALOGS;
 +  }
 +
 +  public static void addLogEntry(Credentials credentials, LogEntry entry, ZooLock zooLock) {
 +    if (entry.extent.isRootTablet()) {
 +      String root = getZookeeperLogLocation();
 +      while (true) {
 +        try {
 +          IZooReaderWriter zoo = ZooReaderWriter.getInstance();
 +          if (zoo.isLockHeld(zooLock.getLockID())) {
 +            String[] parts = entry.filename.split("/");
 +            String uniqueId = parts[parts.length - 1];
 +            zoo.putPersistentData(root + "/" + uniqueId, entry.toBytes(), NodeExistsPolicy.OVERWRITE);
 +          }
 +          break;
 +        } catch (KeeperException e) {
 +          log.error(e, e);
 +        } catch (InterruptedException e) {
 +          log.error(e, e);
 +        } catch (IOException e) {
 +          log.error(e, e);
 +        }
 +        UtilWaitThread.sleep(1000);
 +      }
 +    } else {
 +      Mutation m = new Mutation(entry.getRow());
 +      m.put(entry.getColumnFamily(), entry.getColumnQualifier(), entry.getValue());
 +      update(credentials, zooLock, m, entry.extent);
 +    }
 +  }
 +
 +  public static String getRootTabletDir() throws IOException {
 +    IZooReaderWriter zoo = ZooReaderWriter.getInstance();
 +    String zpath = ZooUtil.getRoot(HdfsZooInstance.getInstance()) + RootTable.ZROOT_TABLET_PATH;
 +    try {
 +      return new String(zoo.getData(zpath, null), Constants.UTF8);
 +    } catch (KeeperException e) {
 +      throw new IOException(e);
 +    } catch (InterruptedException e) {
 +      throw new IOException(e);
 +    }
 +  }
 +
 +  public static Pair<List<LogEntry>,SortedMap<FileRef,DataFileValue>> getFileAndLogEntries(Credentials credentials, KeyExtent extent) throws KeeperException,
 +      InterruptedException, IOException {
 +    ArrayList<LogEntry> result = new ArrayList<LogEntry>();
 +    TreeMap<FileRef,DataFileValue> sizes = new TreeMap<FileRef,DataFileValue>();
 +
 +    VolumeManager fs = VolumeManagerImpl.get();
 +    if (extent.isRootTablet()) {
 +      getRootLogEntries(result);
 +      Path rootDir = new Path(getRootTabletDir());
 +      FileStatus[] files = fs.listStatus(rootDir);
 +      for (FileStatus fileStatus : files) {
 +        if (fileStatus.getPath().toString().endsWith("_tmp")) {
 +          continue;
 +        }
 +        DataFileValue dfv = new DataFileValue(0, 0);
 +        sizes.put(new FileRef(fileStatus.getPath().toString(), fileStatus.getPath()), dfv);
 +      }
 +
 +    } else {
 +      String systemTableToCheck = extent.isMeta() ? RootTable.ID : MetadataTable.ID;
 +      Scanner scanner = new ScannerImpl(HdfsZooInstance.getInstance(), credentials, systemTableToCheck, Authorizations.EMPTY);
 +      scanner.fetchColumnFamily(LogColumnFamily.NAME);
 +      scanner.fetchColumnFamily(DataFileColumnFamily.NAME);
 +      scanner.setRange(extent.toMetadataRange());
 +
 +      for (Entry<Key,Value> entry : scanner) {
 +        if (!entry.getKey().getRow().equals(extent.getMetadataEntry())) {
 +          throw new RuntimeException("Unexpected row " + entry.getKey().getRow() + " expected " + extent.getMetadataEntry());
 +        }
 +
 +        if (entry.getKey().getColumnFamily().equals(LogColumnFamily.NAME)) {
 +          result.add(LogEntry.fromKeyValue(entry.getKey(), entry.getValue()));
 +        } else if (entry.getKey().getColumnFamily().equals(DataFileColumnFamily.NAME)) {
 +          DataFileValue dfv = new DataFileValue(entry.getValue().get());
 +          sizes.put(new FileRef(fs, entry.getKey()), dfv);
 +        } else {
 +          throw new RuntimeException("Unexpected col fam " + entry.getKey().getColumnFamily());
 +        }
 +      }
 +    }
 +
 +    return new Pair<List<LogEntry>,SortedMap<FileRef,DataFileValue>>(result, sizes);
 +  }
 +
 +  public static List<LogEntry> getLogEntries(Credentials credentials, KeyExtent extent) throws IOException, KeeperException, InterruptedException {
 +    log.info("Scanning logging entries for " + extent);
 +    ArrayList<LogEntry> result = new ArrayList<LogEntry>();
 +    if (extent.equals(RootTable.EXTENT)) {
 +      log.info("Getting logs for root tablet from zookeeper");
 +      getRootLogEntries(result);
 +    } else {
 +      log.info("Scanning metadata for logs used for tablet " + extent);
 +      Scanner scanner = getTabletLogScanner(credentials, extent);
 +      Text pattern = extent.getMetadataEntry();
 +      for (Entry<Key,Value> entry : scanner) {
 +        Text row = entry.getKey().getRow();
 +        if (entry.getKey().getColumnFamily().equals(LogColumnFamily.NAME)) {
 +          if (row.equals(pattern)) {
 +            result.add(LogEntry.fromKeyValue(entry.getKey(), entry.getValue()));
 +          }
 +        }
 +      }
 +    }
 +
 +    Collections.sort(result, new Comparator<LogEntry>() {
 +      @Override
 +      public int compare(LogEntry o1, LogEntry o2) {
 +        long diff = o1.timestamp - o2.timestamp;
 +        if (diff < 0)
 +          return -1;
 +        if (diff > 0)
 +          return 1;
 +        return 0;
 +      }
 +    });
 +    log.info("Returning logs " + result + " for extent " + extent);
 +    return result;
 +  }
 +
 +  private static void getRootLogEntries(ArrayList<LogEntry> result) throws KeeperException, InterruptedException, IOException {
 +    IZooReaderWriter zoo = ZooReaderWriter.getInstance();
 +    String root = getZookeeperLogLocation();
 +    // there's a little race between getting the children and fetching
 +    // the data. The log can be removed in between.
 +    while (true) {
 +      result.clear();
 +      for (String child : zoo.getChildren(root)) {
 +        LogEntry e = new LogEntry();
 +        try {
 +          e.fromBytes(zoo.getData(root + "/" + child, null));
 +          // upgrade from !0;!0<< -> !!R<<
 +          e.extent = RootTable.EXTENT;
 +          result.add(e);
 +        } catch (KeeperException.NoNodeException ex) {
 +          continue;
 +        }
 +      }
 +      break;
 +    }
 +  }
 +
 +  private static Scanner getTabletLogScanner(Credentials credentials, KeyExtent extent) {
 +    String tableId = MetadataTable.ID;
 +    if (extent.isMeta())
 +      tableId = RootTable.ID;
 +    Scanner scanner = new ScannerImpl(HdfsZooInstance.getInstance(), credentials, tableId, Authorizations.EMPTY);
 +    scanner.fetchColumnFamily(LogColumnFamily.NAME);
 +    Text start = extent.getMetadataEntry();
 +    Key endKey = new Key(start, LogColumnFamily.NAME);
 +    endKey = endKey.followingKey(PartialKey.ROW_COLFAM);
 +    scanner.setRange(new Range(new Key(start), endKey));
 +    return scanner;
 +  }
 +
-   static class LogEntryIterator implements Iterator<LogEntry> {
++  private static class LogEntryIterator implements Iterator<LogEntry> {
 +
 +    Iterator<LogEntry> zookeeperEntries = null;
 +    Iterator<LogEntry> rootTableEntries = null;
 +    Iterator<Entry<Key,Value>> metadataEntries = null;
 +
 +    LogEntryIterator(Credentials creds) throws IOException, KeeperException, InterruptedException {
 +      zookeeperEntries = getLogEntries(creds, RootTable.EXTENT).iterator();
 +      rootTableEntries = getLogEntries(creds, new KeyExtent(new Text(MetadataTable.ID), null, null)).iterator();
 +      try {
 +        Scanner scanner = HdfsZooInstance.getInstance().getConnector(creds.getPrincipal(), creds.getToken())
 +            .createScanner(MetadataTable.NAME, Authorizations.EMPTY);
 +        log.info("Setting range to " + MetadataSchema.TabletsSection.getRange());
 +        scanner.setRange(MetadataSchema.TabletsSection.getRange());
 +        scanner.fetchColumnFamily(LogColumnFamily.NAME);
 +        metadataEntries = scanner.iterator();
 +      } catch (Exception ex) {
 +        throw new IOException(ex);
 +      }
 +    }
 +
 +    @Override
 +    public boolean hasNext() {
 +      return zookeeperEntries.hasNext() || rootTableEntries.hasNext() || metadataEntries.hasNext();
 +    }
 +
 +    @Override
 +    public LogEntry next() {
 +      if (zookeeperEntries.hasNext()) {
 +        return zookeeperEntries.next();
 +      }
 +      if (rootTableEntries.hasNext()) {
 +        return rootTableEntries.next();
 +      }
 +      Entry<Key,Value> entry = metadataEntries.next();
 +      return LogEntry.fromKeyValue(entry.getKey(), entry.getValue());
 +    }
 +
 +    @Override
 +    public void remove() {
 +      throw new UnsupportedOperationException();
 +    }
 +  }
 +
 +  public static Iterator<LogEntry> getLogEntries(Credentials creds) throws IOException, KeeperException, InterruptedException {
 +    return new LogEntryIterator(creds);
 +  }
 +
 +  public static void removeUnusedWALEntries(KeyExtent extent, List<LogEntry> logEntries, ZooLock zooLock) {
-     for (LogEntry entry : logEntries) {
-       if (entry.extent.isRootTablet()) {
++    if (extent.isRootTablet()) {
++      for (LogEntry entry : logEntries) {
 +        String root = getZookeeperLogLocation();
 +        while (true) {
 +          try {
 +            IZooReaderWriter zoo = ZooReaderWriter.getInstance();
 +            if (zoo.isLockHeld(zooLock.getLockID()))
 +              zoo.recursiveDelete(root + "/" + entry.filename, NodeMissingPolicy.SKIP);
 +            break;
 +          } catch (Exception e) {
 +            log.error(e, e);
 +          }
 +          UtilWaitThread.sleep(1000);
 +        }
-       } else {
-         Mutation m = new Mutation(entry.extent.getMetadataEntry());
++      }
++    } else {
++      Mutation m = new Mutation(extent.getMetadataEntry());
++      for (LogEntry entry : logEntries) {
 +        m.putDelete(LogColumnFamily.NAME, new Text(entry.toString()));
-         update(SystemCredentials.get(), zooLock, m, entry.extent);
 +      }
++      update(SystemCredentials.get(), zooLock, m, extent);
 +    }
 +  }
 +
 +  private static void getFiles(Set<String> files, Map<Key,Value> tablet, String srcTableId) {
 +    for (Entry<Key,Value> entry : tablet.entrySet()) {
 +      if (entry.getKey().getColumnFamily().equals(DataFileColumnFamily.NAME)) {
 +        String cf = entry.getKey().getColumnQualifier().toString();
 +        if (srcTableId != null && !cf.startsWith("../") && !cf.contains(":")) {
 +          cf = "../" + srcTableId + entry.getKey().getColumnQualifier();
 +        }
 +        files.add(cf);
 +      }
 +    }
 +  }
 +
 +  private static Mutation createCloneMutation(String srcTableId, String tableId, Map<Key,Value> tablet) {
 +
 +    KeyExtent ke = new KeyExtent(tablet.keySet().iterator().next().getRow(), (Text) null);
 +    Mutation m = new Mutation(KeyExtent.getMetadataEntry(new Text(tableId), ke.getEndRow()));
 +
 +    for (Entry<Key,Value> entry : tablet.entrySet()) {
 +      if (entry.getKey().getColumnFamily().equals(DataFileColumnFamily.NAME)) {
 +        String cf = entry.getKey().getColumnQualifier().toString();
 +        if (!cf.startsWith("../") && !cf.contains(":"))
 +          cf = "../" + srcTableId + entry.getKey().getColumnQualifier();
 +        m.put(entry.getKey().getColumnFamily(), new Text(cf), entry.getValue());
 +      } else if (entry.getKey().getColumnFamily().equals(TabletsSection.CurrentLocationColumnFamily.NAME)) {
 +        m.put(TabletsSection.LastLocationColumnFamily.NAME, entry.getKey().getColumnQualifier(), entry.getValue());
 +      } else if (entry.getKey().getColumnFamily().equals(TabletsSection.LastLocationColumnFamily.NAME)) {
 +        // skip
 +      } else {
 +        m.put(entry.getKey().getColumnFamily(), entry.getKey().getColumnQualifier(), entry.getValue());
 +      }
 +    }
 +    return m;
 +  }
 +
 +  private static Scanner createCloneScanner(String tableId, Connector conn) throws TableNotFoundException {
 +    Scanner mscanner = new IsolatedScanner(conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY));
 +    mscanner.setRange(new KeyExtent(new Text(tableId), null, null).toMetadataRange());
 +    mscanner.fetchColumnFamily(DataFileColumnFamily.NAME);
 +    mscanner.fetchColumnFamily(TabletsSection.CurrentLocationColumnFamily.NAME);
 +    mscanner.fetchColumnFamily(TabletsSection.LastLocationColumnFamily.NAME);
 +    mscanner.fetchColumnFamily(ClonedColumnFamily.NAME);
 +    TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN.fetch(mscanner);
 +    TabletsSection.ServerColumnFamily.TIME_COLUMN.fetch(mscanner);
 +    return mscanner;
 +  }
 +
 +  static void initializeClone(String srcTableId, String tableId, Connector conn, BatchWriter bw) throws TableNotFoundException, MutationsRejectedException {
 +    TabletIterator ti = new TabletIterator(createCloneScanner(srcTableId, conn), new KeyExtent(new Text(srcTableId), null, null).toMetadataRange(), true, true);
 +
 +    if (!ti.hasNext())
 +      throw new RuntimeException(" table deleted during clone?  srcTableId = " + srcTableId);
 +
 +    while (ti.hasNext())
 +      bw.addMutation(createCloneMutation(srcTableId, tableId, ti.next()));
 +
 +    bw.flush();
 +  }
 +
-   static int compareEndRows(Text endRow1, Text endRow2) {
++  private static int compareEndRows(Text endRow1, Text endRow2) {
 +    return new KeyExtent(new Text("0"), endRow1, null).compareTo(new KeyExtent(new Text("0"), endRow2, null));
 +  }
 +
 +  static int checkClone(String srcTableId, String tableId, Connector conn, BatchWriter bw) throws TableNotFoundException, MutationsRejectedException {
 +    TabletIterator srcIter = new TabletIterator(createCloneScanner(srcTableId, conn), new KeyExtent(new Text(srcTableId), null, null).toMetadataRange(), true,
 +        true);
 +    TabletIterator cloneIter = new TabletIterator(createCloneScanner(tableId, conn), new KeyExtent(new Text(tableId), null, null).toMetadataRange(), true, true);
 +
 +    if (!cloneIter.hasNext() || !srcIter.hasNext())
 +      throw new RuntimeException(" table deleted during clone?  srcTableId = " + srcTableId + " tableId=" + tableId);
 +
 +    int rewrites = 0;
 +
 +    while (cloneIter.hasNext()) {
 +      Map<Key,Value> cloneTablet = cloneIter.next();
 +      Text cloneEndRow = new KeyExtent(cloneTablet.keySet().iterator().next().getRow(), (Text) null).getEndRow();
 +      HashSet<String> cloneFiles = new HashSet<String>();
 +
 +      boolean cloneSuccessful = false;
 +      for (Entry<Key,Value> entry : cloneTablet.entrySet()) {
 +        if (entry.getKey().getColumnFamily().equals(ClonedColumnFamily.NAME)) {
 +          cloneSuccessful = true;
 +          break;
 +        }
 +      }
 +
 +      if (!cloneSuccessful)
 +        getFiles(cloneFiles, cloneTablet, null);
 +
 +      List<Map<Key,Value>> srcTablets = new ArrayList<Map<Key,Value>>();
 +      Map<Key,Value> srcTablet = srcIter.next();
 +      srcTablets.add(srcTablet);
 +
 +      Text srcEndRow = new KeyExtent(srcTablet.keySet().iterator().next().getRow(), (Text) null).getEndRow();
 +
 +      int cmp = compareEndRows(cloneEndRow, srcEndRow);
 +      if (cmp < 0)
 +        throw new TabletIterator.TabletDeletedException("Tablets deleted from src during clone : " + cloneEndRow + " " + srcEndRow);
 +
 +      HashSet<String> srcFiles = new HashSet<String>();
 +      if (!cloneSuccessful)
 +        getFiles(srcFiles, srcTablet, srcTableId);
 +
 +      while (cmp > 0) {
 +        srcTablet = srcIter.next();
 +        srcTablets.add(srcTablet);
 +        srcEndRow = new KeyExtent(srcTablet.keySet().iterator().next().getRow(), (Text) null).getEndRow();
 +        cmp = compareEndRows(cloneEndRow, srcEndRow);
 +        if (cmp < 0)
 +          throw new TabletIterator.TabletDeletedException("Tablets deleted from src during clone : " + cloneEndRow + " " + srcEndRow);
 +
 +        if (!cloneSuccessful)
 +          getFiles(srcFiles, srcTablet, srcTableId);
 +      }
 +
 +      if (cloneSuccessful)
 +        continue;
 +
 +      if (!srcFiles.containsAll(cloneFiles)) {
 +        // delete existing cloned tablet entry
 +        Mutation m = new Mutation(cloneTablet.keySet().iterator().next().getRow());
 +
 +        for (Entry<Key,Value> entry : cloneTablet.entrySet()) {
 +          Key k = entry.getKey();
 +          m.putDelete(k.getColumnFamily(), k.getColumnQualifier(), k.getTimestamp());
 +        }
 +
 +        bw.addMutation(m);
 +
 +        for (Map<Key,Value> st : srcTablets)
 +          bw.addMutation(createCloneMutation(srcTableId, tableId, st));
 +
 +        rewrites++;
 +      } else {
 +        // write out marker that this tablet was successfully cloned
 +        Mutation m = new Mutation(cloneTablet.keySet().iterator().next().getRow());
 +        m.put(ClonedColumnFamily.NAME, new Text(""), new Value("OK".getBytes()));
 +        bw.addMutation(m);
 +      }
 +    }
 +
 +    bw.flush();
 +    return rewrites;
 +  }
 +
 +  public static void cloneTable(Instance instance, String srcTableId, String tableId, VolumeManager volumeManager) throws Exception {
 +
 +    Connector conn = instance.getConnector(SystemCredentials.get().getPrincipal(), SystemCredentials.get().getToken());
 +    BatchWriter bw = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
 +
 +    while (true) {
 +
 +      try {
 +        initializeClone(srcTableId, tableId, conn, bw);
 +
 +        // the following loop looks changes in the file that occurred during the copy.. if files were dereferenced then they could have been GCed
 +
 +        while (true) {
 +          int rewrites = checkClone(srcTableId, tableId, conn, bw);
 +
 +          if (rewrites == 0)
 +            break;
 +        }
 +
 +        bw.flush();
 +        break;
 +
 +      } catch (TabletIterator.TabletDeletedException tde) {
 +        // tablets were merged in the src table
 +        bw.flush();
 +
 +        // delete what we have cloned and try again
 +        deleteTable(tableId, false, SystemCredentials.get(), null);
 +
 +        log.debug("Tablets merged in table " + srcTableId + " while attempting to clone, trying again");
 +
 +        UtilWaitThread.sleep(100);
 +      }
 +    }
 +
 +    // delete the clone markers and create directory entries
 +    Scanner mscanner = conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
 +    mscanner.setRange(new KeyExtent(new Text(tableId), null, null).toMetadataRange());
 +    mscanner.fetchColumnFamily(ClonedColumnFamily.NAME);
 +
 +    int dirCount = 0;
 +
 +    for (Entry<Key,Value> entry : mscanner) {
 +      Key k = entry.getKey();
 +      Mutation m = new Mutation(k.getRow());
 +      m.putDelete(k.getColumnFamily(), k.getColumnQualifier());
 +      String dir = volumeManager.choose(ServerConstants.getTablesDirs()) + "/" + tableId
 +          + new String(FastFormat.toZeroPaddedString(dirCount++, 8, 16, "/c-".getBytes()));
 +      TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN.put(m, new Value(dir.getBytes()));
 +      bw.addMutation(m);
 +    }
 +
 +    bw.close();
 +
 +  }
 +
 +  public static void chopped(KeyExtent extent, ZooLock zooLock) {
 +    Mutation m = new Mutation(extent.getMetadataEntry());
 +    ChoppedColumnFamily.CHOPPED_COLUMN.put(m, new Value("chopped".getBytes()));
 +    update(SystemCredentials.get(), zooLock, m, extent);
 +  }
 +
 +  public static void removeBulkLoadEntries(Connector conn, String tableId, long tid) throws Exception {
 +    Scanner mscanner = new IsolatedScanner(conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY));
 +    mscanner.setRange(new KeyExtent(new Text(tableId), null, null).toMetadataRange());
 +    mscanner.fetchColumnFamily(TabletsSection.BulkFileColumnFamily.NAME);
 +    BatchWriter bw = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
 +    for (Entry<Key,Value> entry : mscanner) {
 +      log.debug("Looking at entry " + entry + " with tid " + tid);
 +      if (Long.parseLong(entry.getValue().toString()) == tid) {
 +        log.debug("deleting entry " + entry);
 +        Mutation m = new Mutation(entry.getKey().getRow());
 +        m.putDelete(entry.getKey().getColumnFamily(), entry.getKey().getColumnQualifier());
 +        bw.addMutation(m);
 +      }
 +    }
 +    bw.close();
 +  }
 +
 +  public static List<FileRef> getBulkFilesLoaded(Connector conn, KeyExtent extent, long tid) throws IOException {
 +    List<FileRef> result = new ArrayList<FileRef>();
 +    try {
 +      VolumeManager fs = VolumeManagerImpl.get();
 +      Scanner mscanner = new IsolatedScanner(conn.createScanner(extent.isMeta() ? RootTable.NAME : MetadataTable.NAME, Authorizations.EMPTY));
 +      mscanner.setRange(extent.toMetadataRange());
 +      mscanner.fetchColumnFamily(TabletsSection.BulkFileColumnFamily.NAME);
 +      for (Entry<Key,Value> entry : mscanner) {
 +        if (Long.parseLong(entry.getValue().toString()) == tid) {
 +          result.add(new FileRef(fs, entry.getKey()));
 +        }
 +      }
 +      return result;
 +    } catch (TableNotFoundException ex) {
 +      // unlikely
 +      throw new RuntimeException("Onos! teh metadata table has vanished!!");
 +    }
 +  }
 +
 +  public static Map<FileRef,Long> getBulkFilesLoaded(Credentials credentials, KeyExtent extent) throws IOException {
 +    Text metadataRow = extent.getMetadataEntry();
 +    Map<FileRef,Long> ret = new HashMap<FileRef,Long>();
 +
 +    VolumeManager fs = VolumeManagerImpl.get();
 +    Scanner scanner = new ScannerImpl(HdfsZooInstance.getInstance(), credentials, extent.isMeta() ? RootTable.ID : MetadataTable.ID, Authorizations.EMPTY);
 +    scanner.setRange(new Range(metadataRow));
 +    scanner.fetchColumnFamily(TabletsSection.BulkFileColumnFamily.NAME);
 +    for (Entry<Key,Value> entry : scanner) {
 +      Long tid = Long.parseLong(entry.getValue().toString());
 +      ret.put(new FileRef(fs, entry.getKey()), tid);
 +    }
 +    return ret;
 +  }
 +
 +  public static void addBulkLoadInProgressFlag(String path) {
 +
 +    Mutation m = new Mutation(MetadataSchema.BlipSection.getRowPrefix() + path);
 +    m.put(EMPTY_TEXT, EMPTY_TEXT, new Value(new byte[] {}));
 +
 +    // new KeyExtent is only added to force update to write to the metadata table, not the root table
 +    // because bulk loads aren't supported to the metadata table
 +    update(SystemCredentials.get(), m, new KeyExtent(new Text("anythingNotMetadata"), null, null));
 +  }
 +
 +  public static void removeBulkLoadInProgressFlag(String path) {
 +
 +    Mutation m = new Mutation(MetadataSchema.BlipSection.getRowPrefix() + path);
 +    m.putDelete(EMPTY_TEXT, EMPTY_TEXT);
 +
 +    // new KeyExtent is only added to force update to write to the metadata table, not the root table
 +    // because bulk loads aren't supported to the metadata table
 +    update(SystemCredentials.get(), m, new KeyExtent(new Text("anythingNotMetadata"), null, null));
 +  }
 +
 +  public static void moveMetaDeleteMarkers(Instance instance, Credentials creds) {
 +    // move old delete markers to new location, to standardize table schema between all metadata tables
 +    byte[] EMPTY_BYTES = new byte[0];
 +    Scanner scanner = new ScannerImpl(instance, creds, RootTable.ID, Authorizations.EMPTY);
 +    String oldDeletesPrefix = "!!~del";
 +    Range oldDeletesRange = new Range(oldDeletesPrefix, true, "!!~dem", false);
 +    scanner.setRange(oldDeletesRange);
 +    for (Entry<Key,Value> entry : scanner) {
 +      String row = entry.getKey().getRow().toString();
 +      if (row.startsWith(oldDeletesPrefix)) {
 +        String filename = row.substring(oldDeletesPrefix.length());
 +        // add the new entry first
 +        log.info("Moving " + filename + " marker in " + RootTable.NAME);
 +        Mutation m = new Mutation(MetadataSchema.DeletesSection.getRowPrefix() + filename);
 +        m.put(EMPTY_BYTES, EMPTY_BYTES, EMPTY_BYTES);
 +        update(creds, m, RootTable.EXTENT);
 +        // remove the old entry
 +        m = new Mutation(entry.getKey().getRow());
 +        m.putDelete(EMPTY_BYTES, EMPTY_BYTES);
 +        update(creds, m, RootTable.OLD_EXTENT);
 +      } else {
 +        break;
 +      }
 +    }
 +
 +  }
 +
 +  public static SortedMap<Text,SortedMap<ColumnFQ,Value>> getTabletEntries(SortedMap<Key,Value> tabletKeyValues, List<ColumnFQ> columns) {
 +    TreeMap<Text,SortedMap<ColumnFQ,Value>> tabletEntries = new TreeMap<Text,SortedMap<ColumnFQ,Value>>();
 +
 +    HashSet<ColumnFQ> colSet = null;
 +    if (columns != null) {
 +      colSet = new HashSet<ColumnFQ>(columns);
 +    }
 +
 +    for (Entry<Key,Value> entry : tabletKeyValues.entrySet()) {
 +
 +      if (columns != null && !colSet.contains(new ColumnFQ(entry.getKey()))) {
 +        continue;
 +      }
 +
 +      Text row = entry.getKey().getRow();
 +
 +      SortedMap<ColumnFQ,Value> colVals = tabletEntries.get(row);
 +      if (colVals == null) {
 +        colVals = new TreeMap<ColumnFQ,Value>();
 +        tabletEntries.put(row, colVals);
 +      }
 +
 +      colVals.put(new ColumnFQ(entry.getKey()), entry.getValue());
 +    }
 +
 +    return tabletEntries;
 +  }
- 
-   public static void convertRootTabletToRootTable(Instance instance, SystemCredentials systemCredentials) throws KeeperException, InterruptedException {
-     ZooReaderWriter zoo = ZooReaderWriter.getInstance();
-     if (zoo.exists(ZooUtil.getRoot(instance) + "/tables/" + RootTable.ID))
-       return;
-     TableManager.prepareNewTableState(instance.getInstanceID(), RootTable.ID, RootTable.NAME, TableState.ONLINE, NodeExistsPolicy.FAIL);
-   }
- 
 +}


[2/8] git commit: ACCUMULO-1914 make WALog cleanup after recovery an atomic mutation

Posted by ec...@apache.org.
ACCUMULO-1914 make WALog cleanup after recovery an atomic mutation


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

Branch: refs/heads/master
Commit: 8bd6e0ebacb8fafeef8627807f610f10b036be6e
Parents: 32b6b65
Author: Eric Newton <er...@gmail.com>
Authored: Thu Nov 21 12:19:01 2013 -0500
Committer: Eric Newton <er...@gmail.com>
Committed: Thu Nov 21 12:19:01 2013 -0500

----------------------------------------------------------------------
 .../org/apache/accumulo/server/util/MetadataTable.java  | 12 +++++++-----
 1 file changed, 7 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/8bd6e0eb/src/server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java
----------------------------------------------------------------------
diff --git a/src/server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java b/src/server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java
index 9c4eebf..a9b72e0 100644
--- a/src/server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java
+++ b/src/server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java
@@ -991,8 +991,8 @@ public class MetadataTable extends org.apache.accumulo.core.util.MetadataTable {
   }
   
   public static void removeUnusedWALEntries(KeyExtent extent, List<LogEntry> logEntries, ZooLock zooLock) {
-    for (LogEntry entry : logEntries) {
-      if (entry.extent.equals(Constants.ROOT_TABLET_EXTENT)) {
+      if (extent.equals(Constants.ROOT_TABLET_EXTENT)) {
+        for (LogEntry entry : logEntries) {
         String root = getZookeeperLogLocation();
         while (true) {
           try {
@@ -1005,11 +1005,13 @@ public class MetadataTable extends org.apache.accumulo.core.util.MetadataTable {
           }
           UtilWaitThread.sleep(1000);
         }
+        }
       } else {
-        Mutation m = new Mutation(entry.extent.getMetadataEntry());
-        m.putDelete(Constants.METADATA_LOG_COLUMN_FAMILY, new Text(entry.server + "/" + entry.filename));
+        Mutation m = new Mutation(extent.getMetadataEntry());
+        for (LogEntry entry : logEntries) {
+          m.putDelete(Constants.METADATA_LOG_COLUMN_FAMILY, new Text(entry.server + "/" + entry.filename));
+        }
         update(SecurityConstants.getSystemCredentials(), zooLock, m);
-      }
     }
   }