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 2015/04/25 01:20:47 UTC

[01/34] accumulo git commit: ACCUMULO-3625 use log markers against tservers, not tablets

Repository: accumulo
Updated Branches:
  refs/heads/master 6df71693e -> 51f39d292


http://git-wip-us.apache.org/repos/asf/accumulo/blob/b2539fb1/test/src/test/java/org/apache/accumulo/test/replication/ReplicationIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/replication/ReplicationIT.java b/test/src/test/java/org/apache/accumulo/test/replication/ReplicationIT.java
index 185a33a..1e508e8 100644
--- a/test/src/test/java/org/apache/accumulo/test/replication/ReplicationIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/replication/ReplicationIT.java
@@ -16,11 +16,12 @@
  */
 package org.apache.accumulo.test.replication;
 
+import static java.nio.charset.StandardCharsets.UTF_8;
+
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collection;
 import java.util.EnumSet;
 import java.util.HashSet;
 import java.util.Iterator;
@@ -46,6 +47,7 @@ import org.apache.accumulo.core.client.admin.TableOperations;
 import org.apache.accumulo.core.client.replication.ReplicaSystemFactory;
 import org.apache.accumulo.core.conf.Property;
 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.Range;
 import org.apache.accumulo.core.data.Value;
@@ -67,6 +69,7 @@ import org.apache.accumulo.core.replication.proto.Replication.Status;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.TablePermission;
 import org.apache.accumulo.core.tabletserver.log.LogEntry;
+import org.apache.accumulo.core.util.AddressUtil;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
@@ -75,11 +78,10 @@ import org.apache.accumulo.fate.zookeeper.ZooLock;
 import org.apache.accumulo.gc.SimpleGarbageCollector;
 import org.apache.accumulo.minicluster.ServerType;
 import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
-import org.apache.accumulo.minicluster.impl.ProcessReference;
+import org.apache.accumulo.server.master.state.TServerInstance;
 import org.apache.accumulo.server.replication.StatusCombiner;
 import org.apache.accumulo.server.util.ReplicationTableUtil;
 import org.apache.accumulo.test.functional.ConfigurableMacIT;
-import org.apache.accumulo.tserver.TabletServer;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -123,25 +125,38 @@ public class ReplicationIT extends ConfigurableMacIT {
     cfg.setProperty(Property.REPLICATION_WORK_PROCESSOR_DELAY, "1s");
     cfg.setProperty(Property.REPLICATION_WORK_PROCESSOR_PERIOD, "1s");
     cfg.setProperty(Property.TSERV_TOTAL_MUTATION_QUEUE_MAX, "1M");
+    cfg.setProperty(Property.INSTANCE_ZK_TIMEOUT, "5s");
     cfg.setNumTservers(1);
     hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
   }
 
   private Multimap<String,String> getLogs(Connector conn) throws TableNotFoundException {
-    Multimap<String,String> logs = HashMultimap.create();
+    // Map of server to tableId
+    Multimap<TServerInstance, String> serverToTableID = HashMultimap.create();
     Scanner scanner = conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
-    scanner.fetchColumnFamily(LogColumnFamily.NAME);
-    scanner.setRange(new Range());
+    scanner.setRange(MetadataSchema.TabletsSection.getRange());
+    scanner.fetchColumnFamily(MetadataSchema.TabletsSection.CurrentLocationColumnFamily.NAME);
+    for (Entry<Key,Value> entry : scanner) {
+      TServerInstance key = new TServerInstance(entry.getValue(), entry.getKey().getColumnQualifier());
+      byte[] tableId = KeyExtent.tableOfMetadataRow(entry.getKey().getRow());
+      serverToTableID.put(key, new String(tableId, UTF_8));
+    }
+    // Map of logs to tableId
+    Multimap<String,String> logs = HashMultimap.create();
+    scanner = conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
+    scanner.setRange(MetadataSchema.CurrentLogsSection.getRange());
     for (Entry<Key,Value> entry : scanner) {
       if (Thread.interrupted()) {
         return logs;
       }
-
-      LogEntry logEntry = LogEntry.fromKeyValue(entry.getKey(), entry.getValue());
-
-      for (String log : logEntry.logSet) {
-        // Need to normalize the log file from LogEntry
-        logs.put(new Path(log).toString(), logEntry.extent.getTableId().toString());
+      Text path = new Text();
+      MetadataSchema.CurrentLogsSection.getPath(entry.getKey(), path);
+      Text session = new Text();
+      Text hostPort = new Text();
+      MetadataSchema.CurrentLogsSection.getTabletServer(entry.getKey(), hostPort , session);
+      TServerInstance server = new TServerInstance(AddressUtil.parseAddress(hostPort.toString()), session.toString());
+      for (String tableId : serverToTableID.get(server)) {
+        logs.put(new Path(path.toString()).toString(), tableId);
       }
     }
     return logs;
@@ -296,10 +311,12 @@ public class ReplicationIT extends ConfigurableMacIT {
     attempts = 5;
     while (wals.isEmpty() && attempts > 0) {
       s = conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
-      s.fetchColumnFamily(MetadataSchema.TabletsSection.LogColumnFamily.NAME);
+      s.setRange(MetadataSchema.CurrentLogsSection.getRange());
+      s.fetchColumnFamily(MetadataSchema.CurrentLogsSection.COLF);
       for (Entry<Key,Value> entry : s) {
-        LogEntry logEntry = LogEntry.fromKeyValue(entry.getKey(), entry.getValue());
-        wals.add(new Path(logEntry.filename).toString());
+        Text path = new Text();
+        MetadataSchema.CurrentLogsSection.getPath(entry.getKey(), path);
+        wals.add(new Path(path.toString()).toString());
       }
       attempts--;
     }
@@ -330,18 +347,7 @@ public class ReplicationIT extends ConfigurableMacIT {
     Assert.assertFalse(ReplicationTable.isOnline(conn));
 
     for (String table : tables) {
-      BatchWriter bw = conn.createBatchWriter(table, new BatchWriterConfig());
-
-      for (int j = 0; j < 5; j++) {
-        Mutation m = new Mutation(Integer.toString(j));
-        for (int k = 0; k < 5; k++) {
-          String value = Integer.toString(k);
-          m.put(value, "", value);
-        }
-        bw.addMutation(m);
-      }
-
-      bw.close();
+      writeSomeData(conn, table, 5, 5);
     }
 
     // After writing data, still no replication table
@@ -381,18 +387,7 @@ public class ReplicationIT extends ConfigurableMacIT {
     Assert.assertFalse(ReplicationTable.isOnline(conn));
 
     // Write some data to table1
-    BatchWriter bw = conn.createBatchWriter(table1, new BatchWriterConfig());
-
-    for (int rows = 0; rows < 50; rows++) {
-      Mutation m = new Mutation(Integer.toString(rows));
-      for (int cols = 0; cols < 50; cols++) {
-        String value = Integer.toString(cols);
-        m.put(value, "", value);
-      }
-      bw.addMutation(m);
-    }
-
-    bw.close();
+    writeSomeData(conn, table1, 50, 50);
 
     // After the commit for these mutations finishes, we'll get a replication entry in accumulo.metadata for table1
     // Don't want to compact table1 as it ultimately cause the entry in accumulo.metadata to be removed before we can verify it's there
@@ -439,18 +434,7 @@ public class ReplicationIT extends ConfigurableMacIT {
     conn.tableOperations().setProperty(table2, Property.TABLE_REPLICATION.getKey(), "true");
 
     // Write some data to table2
-    bw = conn.createBatchWriter(table2, new BatchWriterConfig());
-
-    for (int rows = 0; rows < 50; rows++) {
-      Mutation m = new Mutation(Integer.toString(rows));
-      for (int cols = 0; cols < 50; cols++) {
-        String value = Integer.toString(cols);
-        m.put(value, "", value);
-      }
-      bw.addMutation(m);
-    }
-
-    bw.close();
+    writeSomeData(conn, table2, 50, 50);
 
     // After the commit on these mutations, we'll get a replication entry in accumulo.metadata for table2
     // Don't want to compact table2 as it ultimately cause the entry in accumulo.metadata to be removed before we can verify it's there
@@ -498,6 +482,19 @@ public class ReplicationIT extends ConfigurableMacIT {
     Assert.assertFalse("Expected to only find two elements in replication table", iter.hasNext());
   }
 
+  private void writeSomeData(Connector conn, String table, int rows, int cols) throws Exception {
+    BatchWriter bw = conn.createBatchWriter(table, new BatchWriterConfig());
+    for (int row = 0; row < rows; row++) {
+      Mutation m = new Mutation(Integer.toString(row));
+      for (int col = 0; col < cols; col++) {
+        String value = Integer.toString(col);
+        m.put(value, "", value);
+      }
+      bw.addMutation(m);
+    }
+    bw.close();
+  }
+
   @Test
   public void replicationEntriesPrecludeWalDeletion() throws Exception {
     final Connector conn = getConnector();
@@ -529,53 +526,21 @@ public class ReplicationIT extends ConfigurableMacIT {
     Thread.sleep(2000);
 
     // Write some data to table1
-    BatchWriter bw = conn.createBatchWriter(table1, new BatchWriterConfig());
-    for (int rows = 0; rows < 200; rows++) {
-      Mutation m = new Mutation(Integer.toString(rows));
-      for (int cols = 0; cols < 500; cols++) {
-        String value = Integer.toString(cols);
-        m.put(value, "", value);
-      }
-      bw.addMutation(m);
-    }
-
-    bw.close();
+    writeSomeData(conn, table1, 200, 500);
 
     conn.tableOperations().create(table2);
     conn.tableOperations().setProperty(table2, Property.TABLE_REPLICATION.getKey(), "true");
     conn.tableOperations().setProperty(table2, Property.TABLE_REPLICATION_TARGET.getKey() + "cluster1", "1");
     Thread.sleep(2000);
 
-    // Write some data to table2
-    bw = conn.createBatchWriter(table2, new BatchWriterConfig());
-    for (int rows = 0; rows < 200; rows++) {
-      Mutation m = new Mutation(Integer.toString(rows));
-      for (int cols = 0; cols < 500; cols++) {
-        String value = Integer.toString(cols);
-        m.put(value, "", value);
-      }
-      bw.addMutation(m);
-    }
-
-    bw.close();
+    writeSomeData(conn, table2, 200, 500);
 
     conn.tableOperations().create(table3);
     conn.tableOperations().setProperty(table3, Property.TABLE_REPLICATION.getKey(), "true");
     conn.tableOperations().setProperty(table3, Property.TABLE_REPLICATION_TARGET.getKey() + "cluster1", "1");
     Thread.sleep(2000);
 
-    // Write some data to table3
-    bw = conn.createBatchWriter(table3, new BatchWriterConfig());
-    for (int rows = 0; rows < 200; rows++) {
-      Mutation m = new Mutation(Integer.toString(rows));
-      for (int cols = 0; cols < 500; cols++) {
-        String value = Integer.toString(cols);
-        m.put(value, "", value);
-      }
-      bw.addMutation(m);
-    }
-
-    bw.close();
+    writeSomeData(conn, table3, 200, 500);
 
     // Force a write to metadata for the data written
     for (String table : Arrays.asList(table1, table2, table3)) {
@@ -609,7 +574,8 @@ public class ReplicationIT extends ConfigurableMacIT {
 
     // We should have *some* reference to each log that was seen in the metadata table
     // They might not yet all be closed though (might be newfile)
-    Assert.assertEquals("Metadata log distribution: " + logs, logs.keySet(), replFiles);
+    Assert.assertTrue("Metadata log distribution: " + logs + "replFiles " + replFiles, logs.keySet().containsAll(replFiles));
+    Assert.assertTrue("Difference between replication entries and current logs is bigger than one", logs.keySet().size() - replFiles.size() <= 1);
 
     for (String replFile : replFiles) {
       Path p = new Path(replFile);
@@ -697,44 +663,11 @@ public class ReplicationIT extends ConfigurableMacIT {
       conn.tableOperations().setProperty(table3, Property.TABLE_REPLICATION.getKey(), "true");
       conn.tableOperations().setProperty(table3, Property.TABLE_REPLICATION_TARGET.getKey() + "cluster1", "1");
 
-      // Write some data to table1
-      BatchWriter bw = conn.createBatchWriter(table1, new BatchWriterConfig());
-      for (int rows = 0; rows < 200; rows++) {
-        Mutation m = new Mutation(Integer.toString(rows));
-        for (int cols = 0; cols < 500; cols++) {
-          String value = Integer.toString(cols);
-          m.put(value, "", value);
-        }
-        bw.addMutation(m);
-      }
+      writeSomeData(conn, table1, 200, 500);
 
-      bw.close();
+      writeSomeData(conn, table2, 200, 500);
 
-      // Write some data to table2
-      bw = conn.createBatchWriter(table2, new BatchWriterConfig());
-      for (int rows = 0; rows < 200; rows++) {
-        Mutation m = new Mutation(Integer.toString(rows));
-        for (int cols = 0; cols < 500; cols++) {
-          String value = Integer.toString(cols);
-          m.put(value, "", value);
-        }
-        bw.addMutation(m);
-      }
-
-      bw.close();
-
-      // Write some data to table3
-      bw = conn.createBatchWriter(table3, new BatchWriterConfig());
-      for (int rows = 0; rows < 200; rows++) {
-        Mutation m = new Mutation(Integer.toString(rows));
-        for (int cols = 0; cols < 500; cols++) {
-          String value = Integer.toString(cols);
-          m.put(value, "", value);
-        }
-        bw.addMutation(m);
-      }
-
-      bw.close();
+      writeSomeData(conn, table3, 200, 500);
 
       // Flush everything to try to make the replication records
       for (String table : Arrays.asList(table1, table2, table3)) {
@@ -789,10 +722,7 @@ public class ReplicationIT extends ConfigurableMacIT {
     Set<String> wals = new HashSet<>();
     for (Entry<Key,Value> entry : s) {
       LogEntry logEntry = LogEntry.fromKeyValue(entry.getKey(), entry.getValue());
-      for (String file : logEntry.logSet) {
-        Path p = new Path(file);
-        wals.add(p.toString());
-      }
+      wals.add(new Path(logEntry.filename).toString());
     }
 
     log.warn("Found wals {}", wals);
@@ -869,9 +799,7 @@ public class ReplicationIT extends ConfigurableMacIT {
   public void singleTableWithSingleTarget() throws Exception {
     // We want to kill the GC so it doesn't come along and close Status records and mess up the comparisons
     // against expected Status messages.
-    for (ProcessReference proc : cluster.getProcesses().get(ServerType.GARBAGE_COLLECTOR)) {
-      cluster.killProcess(ServerType.GARBAGE_COLLECTOR, proc);
-    }
+    getCluster().getClusterControl().stop(ServerType.GARBAGE_COLLECTOR);
 
     Connector conn = getConnector();
     String table1 = "table1";
@@ -905,17 +833,7 @@ public class ReplicationIT extends ConfigurableMacIT {
     }
 
     // Write some data to table1
-    BatchWriter bw = conn.createBatchWriter(table1, new BatchWriterConfig());
-    for (int rows = 0; rows < 2000; rows++) {
-      Mutation m = new Mutation(Integer.toString(rows));
-      for (int cols = 0; cols < 50; cols++) {
-        String value = Integer.toString(cols);
-        m.put(value, "", value);
-      }
-      bw.addMutation(m);
-    }
-
-    bw.close();
+    writeSomeData(conn, table1, 2000, 50);
 
     // Make sure the replication table is online at this point
     boolean online = ReplicationTable.isOnline(conn);
@@ -1002,17 +920,7 @@ public class ReplicationIT extends ConfigurableMacIT {
     }
 
     // Write some more data so that we over-run the single WAL
-    bw = conn.createBatchWriter(table1, new BatchWriterConfig());
-    for (int rows = 0; rows < 3000; rows++) {
-      Mutation m = new Mutation(Integer.toString(rows));
-      for (int cols = 0; cols < 50; cols++) {
-        String value = Integer.toString(cols);
-        m.put(value, "", value);
-      }
-      bw.addMutation(m);
-    }
-
-    bw.close();
+    writeSomeData(conn, table1, 3000, 50);
 
     log.info("Issued compaction for table");
     conn.tableOperations().compact(table1, null, null, true, true);
@@ -1085,17 +993,7 @@ public class ReplicationIT extends ConfigurableMacIT {
     }
 
     // Write some data to table1
-    BatchWriter bw = conn.createBatchWriter(table1, new BatchWriterConfig());
-    for (int rows = 0; rows < 2000; rows++) {
-      Mutation m = new Mutation(Integer.toString(rows));
-      for (int cols = 0; cols < 50; cols++) {
-        String value = Integer.toString(cols);
-        m.put(value, "", value);
-      }
-      bw.addMutation(m);
-    }
-
-    bw.close();
+    writeSomeData(conn, table1, 2000, 50);
 
     String tableId = conn.tableOperations().tableIdMap().get(table1);
     Assert.assertNotNull("Table ID was null", tableId);
@@ -1150,10 +1048,7 @@ public class ReplicationIT extends ConfigurableMacIT {
 
   @Test
   public void replicationRecordsAreClosedAfterGarbageCollection() throws Exception {
-    Collection<ProcessReference> gcProcs = cluster.getProcesses().get(ServerType.GARBAGE_COLLECTOR);
-    for (ProcessReference ref : gcProcs) {
-      cluster.killProcess(ServerType.GARBAGE_COLLECTOR, ref);
-    }
+    getCluster().getClusterControl().stop(ServerType.GARBAGE_COLLECTOR);
 
     final Connector conn = getConnector();
 
@@ -1184,7 +1079,6 @@ public class ReplicationIT extends ConfigurableMacIT {
 
     String table1 = "table1", table2 = "table2", table3 = "table3";
 
-    BatchWriter bw;
     try {
       conn.tableOperations().create(table1);
       conn.tableOperations().setProperty(table1, Property.TABLE_REPLICATION.getKey(), "true");
@@ -1193,51 +1087,19 @@ public class ReplicationIT extends ConfigurableMacIT {
           ReplicaSystemFactory.getPeerConfigurationValue(MockReplicaSystem.class, null));
 
       // Write some data to table1
-      bw = conn.createBatchWriter(table1, new BatchWriterConfig());
-      for (int rows = 0; rows < 200; rows++) {
-        Mutation m = new Mutation(Integer.toString(rows));
-        for (int cols = 0; cols < 500; cols++) {
-          String value = Integer.toString(cols);
-          m.put(value, "", value);
-        }
-        bw.addMutation(m);
-      }
-
-      bw.close();
+      writeSomeData(conn, table1, 200, 500);
 
       conn.tableOperations().create(table2);
       conn.tableOperations().setProperty(table2, Property.TABLE_REPLICATION.getKey(), "true");
       conn.tableOperations().setProperty(table2, Property.TABLE_REPLICATION_TARGET.getKey() + "cluster1", "1");
 
-      // Write some data to table2
-      bw = conn.createBatchWriter(table2, new BatchWriterConfig());
-      for (int rows = 0; rows < 200; rows++) {
-        Mutation m = new Mutation(Integer.toString(rows));
-        for (int cols = 0; cols < 500; cols++) {
-          String value = Integer.toString(cols);
-          m.put(value, "", value);
-        }
-        bw.addMutation(m);
-      }
-
-      bw.close();
+      writeSomeData(conn, table2, 200, 500);
 
       conn.tableOperations().create(table3);
       conn.tableOperations().setProperty(table3, Property.TABLE_REPLICATION.getKey(), "true");
       conn.tableOperations().setProperty(table3, Property.TABLE_REPLICATION_TARGET.getKey() + "cluster1", "1");
 
-      // Write some data to table3
-      bw = conn.createBatchWriter(table3, new BatchWriterConfig());
-      for (int rows = 0; rows < 200; rows++) {
-        Mutation m = new Mutation(Integer.toString(rows));
-        for (int cols = 0; cols < 500; cols++) {
-          String value = Integer.toString(cols);
-          m.put(value, "", value);
-        }
-        bw.addMutation(m);
-      }
-
-      bw.close();
+      writeSomeData(conn, table3, 200, 500);
 
       // Flush everything to try to make the replication records
       for (String table : Arrays.asList(table1, table2, table3)) {
@@ -1251,11 +1113,8 @@ public class ReplicationIT extends ConfigurableMacIT {
 
     // Kill the tserver(s) and restart them
     // to ensure that the WALs we previously observed all move to closed.
-    for (ProcessReference proc : cluster.getProcesses().get(ServerType.TABLET_SERVER)) {
-      cluster.killProcess(ServerType.TABLET_SERVER, proc);
-    }
-
-    cluster.exec(TabletServer.class);
+    cluster.getClusterControl().stop(ServerType.TABLET_SERVER);
+    cluster.getClusterControl().start(ServerType.TABLET_SERVER);
 
     // Make sure we can read all the tables (recovery complete)
     for (String table : Arrays.asList(table1, table2, table3)) {
@@ -1358,9 +1217,7 @@ public class ReplicationIT extends ConfigurableMacIT {
   @Test
   public void replicatedStatusEntriesAreDeleted() throws Exception {
     // Just stop it now, we'll restart it after we restart the tserver
-    for (ProcessReference proc : getCluster().getProcesses().get(ServerType.GARBAGE_COLLECTOR)) {
-      getCluster().killProcess(ServerType.GARBAGE_COLLECTOR, proc);
-    }
+    getCluster().getClusterControl().stop(ServerType.GARBAGE_COLLECTOR);
 
     final Connector conn = getConnector();
     log.info("Got connector to MAC");
@@ -1396,17 +1253,7 @@ public class ReplicationIT extends ConfigurableMacIT {
     Assert.assertNotNull("Could not determine table id for " + table1, tableId);
 
     // Write some data to table1
-    BatchWriter bw = conn.createBatchWriter(table1, new BatchWriterConfig());
-    for (int rows = 0; rows < 2000; rows++) {
-      Mutation m = new Mutation(Integer.toString(rows));
-      for (int cols = 0; cols < 50; cols++) {
-        String value = Integer.toString(cols);
-        m.put(value, "", value);
-      }
-      bw.addMutation(m);
-    }
-
-    bw.close();
+    writeSomeData(conn, table1, 2000, 50);
 
     // Make sure the replication table exists at this point
     boolean online = ReplicationTable.isOnline(conn);
@@ -1423,14 +1270,35 @@ public class ReplicationIT extends ConfigurableMacIT {
     // Grant ourselves the write permission for later
     conn.securityOperations().grantTablePermission("root", ReplicationTable.NAME, TablePermission.WRITE);
 
+    log.info("Checking for replication entries in replication");
+    // Then we need to get those records over to the replication table
+    Scanner s;
+    Set<String> entries = new HashSet<>();
+    for (int i = 0; i < 5; i++) {
+      s = conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
+      s.setRange(ReplicationSection.getRange());
+      entries.clear();
+      for (Entry<Key,Value> entry : s) {
+        entries.add(entry.getKey().getRow().toString());
+        log.info("{}={}", entry.getKey().toStringNoTruncate(), entry.getValue());
+      }
+      if (!entries.isEmpty()) {
+        log.info("Replication entries {}", entries);
+        break;
+      }
+      Thread.sleep(1000);
+    }
+
+    Assert.assertFalse("Did not find any replication entries in the replication table", entries.isEmpty());
+
     // Find the WorkSection record that will be created for that data we ingested
     boolean notFound = true;
-    Scanner s;
     for (int i = 0; i < 10 && notFound; i++) {
       try {
         s = ReplicationTable.getScanner(conn);
         WorkSection.limit(s);
         Entry<Key,Value> e = Iterables.getOnlyElement(s);
+        log.info("Found entry: " + e.getKey().toStringNoTruncate());
         Text expectedColqual = new ReplicationTarget("cluster1", "4", tableId).toText();
         Assert.assertEquals(expectedColqual, e.getKey().getColumnQualifier());
         notFound = false;
@@ -1481,14 +1349,13 @@ public class ReplicationIT extends ConfigurableMacIT {
     log.info("Killing tserver");
     // Kill the tserver(s) and restart them
     // to ensure that the WALs we previously observed all move to closed.
-    for (ProcessReference proc : cluster.getProcesses().get(ServerType.TABLET_SERVER)) {
-      cluster.killProcess(ServerType.TABLET_SERVER, proc);
-    }
+    cluster.getClusterControl().stop(ServerType.TABLET_SERVER);
 
     log.info("Starting tserver");
-    cluster.exec(TabletServer.class);
+    cluster.getClusterControl().start(ServerType.TABLET_SERVER);
 
     log.info("Waiting to read tables");
+    UtilWaitThread.sleep(2 * 3 * 1000);
 
     // Make sure we can read all the tables (recovery complete)
     for (String table : new String[] {MetadataTable.NAME, table1}) {
@@ -1497,55 +1364,48 @@ public class ReplicationIT extends ConfigurableMacIT {
       Entry<Key,Value> entry : s) {}
     }
 
-    log.info("Checking for replication entries in replication");
-    // Then we need to get those records over to the replication table
-    boolean foundResults = false;
-    for (int i = 0; i < 5; i++) {
-      s = ReplicationTable.getScanner(conn);
-      int count = 0;
-      for (Entry<Key,Value> entry : s) {
-        count++;
-        log.info("{}={}", entry.getKey().toStringNoTruncate(), entry.getValue());
-      }
-      if (count > 0) {
-        foundResults = true;
-        break;
-      }
-      Thread.sleep(1000);
+    log.info("Recovered metadata:");
+    s = conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
+    for (Entry<Key,Value> entry : s) {
+      log.info("{}={}", entry.getKey().toStringNoTruncate(), entry.getValue());
     }
 
-    Assert.assertTrue("Did not find any replication entries in the replication table", foundResults);
-
-    getCluster().exec(SimpleGarbageCollector.class);
+    cluster.getClusterControl().start(ServerType.GARBAGE_COLLECTOR);
 
     // Wait for a bit since the GC has to run (should be running after a one second delay)
     waitForGCLock(conn);
 
     Thread.sleep(1000);
 
+    log.info("After GC");
+    s = conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
+    for (Entry<Key,Value> entry : s) {
+      log.info("{}={}", entry.getKey().toStringNoTruncate(), entry.getValue());
+    }
+
     // We expect no records in the metadata table after compaction. We have to poll
     // because we have to wait for the StatusMaker's next iteration which will clean
     // up the dangling *closed* records after we create the record in the replication table.
     // We need the GC to close the file (CloseWriteAheadLogReferences) before we can remove the record
     log.info("Checking metadata table for replication entries");
-    foundResults = true;
+    Set<String> remaining = new HashSet<>();
     for (int i = 0; i < 10; i++) {
       s = conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
       s.setRange(ReplicationSection.getRange());
-      long size = 0;
+      remaining.clear();
       for (Entry<Key,Value> e : s) {
-        size++;
-        log.info("{}={}", e.getKey().toStringNoTruncate(), ProtobufUtil.toString(Status.parseFrom(e.getValue().get())));
+        remaining.add(e.getKey().getRow().toString());
       }
-      if (size == 0) {
-        foundResults = false;
+      remaining.retainAll(entries);
+      if (remaining.isEmpty()) {
         break;
       }
+      log.info("remaining {}", remaining);
       Thread.sleep(2000);
       log.info("");
     }
 
-    Assert.assertFalse("Replication status messages were not cleaned up from metadata table", foundResults);
+    Assert.assertTrue("Replication status messages were not cleaned up from metadata table", remaining.isEmpty());
 
     /**
      * After we close out and subsequently delete the metadata record, this will propagate to the replication table, which will cause those records to be
@@ -1558,10 +1418,10 @@ public class ReplicationIT extends ConfigurableMacIT {
       recordsFound = 0;
       for (Entry<Key,Value> entry : s) {
         recordsFound++;
-        log.info(entry.getKey().toStringNoTruncate() + " " + ProtobufUtil.toString(Status.parseFrom(entry.getValue().get())));
+        log.info("{} {}", entry.getKey().toStringNoTruncate(), ProtobufUtil.toString(Status.parseFrom(entry.getValue().get())));
       }
 
-      if (0 == recordsFound) {
+      if (recordsFound <= 2) {
         break;
       } else {
         Thread.sleep(1000);
@@ -1569,6 +1429,6 @@ public class ReplicationIT extends ConfigurableMacIT {
       }
     }
 
-    Assert.assertEquals("Found unexpected replication records in the replication table", 0, recordsFound);
+    Assert.assertTrue("Found unexpected replication records in the replication table", recordsFound <= 2);
   }
 }


[13/34] accumulo git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/accumulo

Posted by ec...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/accumulo


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

Branch: refs/heads/master
Commit: 34af43fd02626eca3c42bc3a39ccf4ab8b927806
Parents: ac176e1 9df45b2
Author: Eric C. Newton <er...@gmail.com>
Authored: Fri Mar 20 10:51:21 2015 -0400
Committer: Eric C. Newton <er...@gmail.com>
Committed: Fri Mar 20 10:51:21 2015 -0400

----------------------------------------------------------------------
 TESTING.md                                      |   1 +
 assemble/bin/accumulo                           |   3 +-
 .../core/client/impl/ThriftScanner.java         |  17 +-
 .../org/apache/accumulo/core/data/Mutation.java |  60 +++++-
 .../apache/accumulo/core/data/MutationTest.java |  42 +++++
 .../mapreduce/bulk/BulkIngestExample.java       |   4 +
 .../standalone/StandaloneAccumuloCluster.java   |  10 +-
 .../standalone/StandaloneClusterControl.java    |  25 ++-
 .../accumulo/harness/AccumuloClusterIT.java     |   2 +-
 .../StandaloneAccumuloClusterConfiguration.java |  16 ++
 .../apache/accumulo/proxy/SimpleProxyBase.java  | 185 ++++++++-----------
 .../apache/accumulo/proxy/TBinaryProxyIT.java   |   6 -
 .../apache/accumulo/proxy/TCompactProxyIT.java  |   6 -
 .../accumulo/proxy/TJsonProtocolProxyIT.java    |   6 -
 .../apache/accumulo/proxy/TTupleProxyIT.java    |   6 -
 .../accumulo/test/BulkImportVolumeIT.java       |   3 +
 .../apache/accumulo/test/ScanIteratorIT.java    |   7 +
 .../accumulo/test/UserCompactionStrategyIT.java |   4 +
 .../apache/accumulo/test/functional/BulkIT.java |  22 +--
 .../functional/BulkSplitOptimizationIT.java     |   1 +
 .../accumulo/test/functional/CompactionIT.java  |   1 +
 .../accumulo/test/functional/DeleteIT.java      |   5 -
 .../test/functional/FunctionalTestUtils.java    |   7 +
 23 files changed, 269 insertions(+), 170 deletions(-)
----------------------------------------------------------------------



[26/34] accumulo git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/accumulo

Posted by ec...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/accumulo


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

Branch: refs/heads/master
Commit: fb25f3b15ac0ca4262ac5c8664cea1ff81246bb6
Parents: f43b216 33ef1e0
Author: Eric C. Newton <er...@gmail.com>
Authored: Wed Apr 22 11:36:52 2015 -0400
Committer: Eric C. Newton <er...@gmail.com>
Committed: Wed Apr 22 11:36:52 2015 -0400

----------------------------------------------------------------------
 README.md                                       |  30 +-
 core/pom.xml                                    |  57 ++
 core/src/main/findbugs/exclude-filter.xml       |   1 +
 .../apache/accumulo/core/cli/ClientOpts.java    |   5 +-
 .../core/client/ClientConfiguration.java        |  50 +-
 .../core/client/ClientSideIteratorScanner.java  |  27 +-
 .../accumulo/core/client/IsolatedScanner.java   |   4 +-
 .../core/client/MutationsRejectedException.java |  68 +-
 .../accumulo/core/client/ZooKeeperInstance.java |  21 +-
 .../core/client/admin/ActiveCompaction.java     |  12 +-
 .../accumulo/core/client/admin/ActiveScan.java  |  12 +-
 .../client/admin/DelegationTokenConfig.java     |   4 +-
 .../core/client/admin/SecurityOperations.java   |   3 +-
 .../core/client/impl/ActiveCompactionImpl.java  |  19 +-
 .../core/client/impl/ActiveScanImpl.java        |  14 +-
 .../impl/AuthenticationTokenIdentifier.java     | 210 +++++
 .../core/client/impl/ClientContext.java         |   1 -
 .../core/client/impl/ConditionalWriterImpl.java |   2 +-
 .../accumulo/core/client/impl/Credentials.java  | 157 ++++
 .../core/client/impl/DelegationTokenImpl.java   | 144 ++++
 .../client/impl/InstanceOperationsImpl.java     |  18 +
 .../client/impl/NamespaceOperationsImpl.java    |   1 -
 .../core/client/impl/OfflineScanner.java        |   3 +-
 .../core/client/impl/RootTabletLocator.java     |   2 +-
 .../client/impl/SecurityOperationsImpl.java     |   4 +-
 .../core/client/impl/TableOperationsImpl.java   |   2 +-
 .../core/client/impl/TabletLocator.java         |   2 +-
 .../core/client/impl/TabletLocatorImpl.java     |   2 +-
 .../impl/TabletServerBatchReaderIterator.java   |   2 +-
 .../client/impl/TabletServerBatchWriter.java    |   8 +-
 .../accumulo/core/client/impl/TabletType.java   |   2 +-
 .../core/client/impl/ThriftScanner.java         |   2 +-
 .../core/client/impl/TimeoutTabletLocator.java  |   2 +-
 .../accumulo/core/client/impl/Translator.java   |   2 +-
 .../accumulo/core/client/impl/Writer.java       |   2 +-
 .../core/client/mapred/AbstractInputFormat.java |  78 +-
 .../client/mapred/AccumuloFileOutputFormat.java |   5 +-
 .../client/mapred/AccumuloOutputFormat.java     |  23 +-
 .../client/mapreduce/AbstractInputFormat.java   |  82 +-
 .../mapreduce/AccumuloFileOutputFormat.java     |   5 +-
 .../client/mapreduce/AccumuloOutputFormat.java  |  21 +-
 .../core/client/mapreduce/RangeInputSplit.java  |  24 +-
 .../mapreduce/impl/AccumuloInputSplit.java      |  73 +-
 .../client/mapreduce/impl/BatchInputSplit.java  |  21 +-
 .../core/client/mapreduce/impl/SplitUtils.java  |  59 ++
 .../mapreduce/lib/impl/ConfiguratorBase.java    |  16 +-
 .../mapreduce/lib/impl/InputConfigurator.java   |   8 +-
 .../core/client/mock/MockConnector.java         |   2 +-
 .../accumulo/core/client/mock/MockInstance.java |   2 +-
 .../client/mock/impl/MockTabletLocator.java     |   2 +-
 .../client/security/tokens/DelegationToken.java | 134 +---
 .../accumulo/core/constraints/Constraint.java   |   2 +-
 .../core/constraints/VisibilityConstraint.java  |  93 +++
 .../accumulo/core/data/ComparableBytes.java     |   5 +
 .../apache/accumulo/core/data/KeyExtent.java    | 717 +++--------------
 .../org/apache/accumulo/core/data/TabletID.java |  29 +
 .../core/data/impl/ComparableBytes.java         |  53 ++
 .../accumulo/core/data/impl/KeyExtent.java      | 768 +++++++++++++++++++
 .../accumulo/core/data/impl/TabletIDImpl.java   | 100 +++
 .../accumulo/core/iterators/IteratorUtil.java   |  40 +-
 .../core/iterators/system/MultiIterator.java    |   2 +-
 .../core/metadata/MetadataLocationObtainer.java |   2 +-
 .../core/metadata/MetadataServicer.java         |   2 +-
 .../accumulo/core/metadata/RootTable.java       |   2 +-
 .../core/metadata/ServicerForRootTable.java     |   2 +-
 .../core/metadata/TableMetadataServicer.java    |   2 +-
 .../ReplicationConfigurationUtil.java           |   2 +-
 .../rpc/SaslClientDigestCallbackHandler.java    |   4 +-
 .../accumulo/core/rpc/SaslConnectionParams.java |   6 +-
 .../security/AuthenticationTokenIdentifier.java | 210 -----
 .../accumulo/core/security/Credentials.java     | 157 ----
 .../core/security/VisibilityConstraint.java     |  76 +-
 .../core/security/VisibilityEvaluator.java      |  10 +-
 .../core/tabletserver/log/LogEntry.java         |   2 +-
 .../org/apache/accumulo/core/util/Merge.java    |   2 +-
 .../core/client/ClientConfigurationTest.java    |  51 +-
 .../core/client/impl/ScannerImplTest.java       |   1 -
 .../client/impl/TableOperationsImplTest.java    |   3 +-
 .../core/client/impl/TabletLocatorImplTest.java |   3 +-
 .../impl/TabletServerBatchReaderTest.java       |   1 -
 .../mapred/AccumuloFileOutputFormatTest.java    |   5 +-
 .../core/client/mapred/TokenFileTest.java       |   2 +-
 .../mapreduce/AccumuloFileOutputFormatTest.java |   5 +-
 .../core/client/mapreduce/TokenFileTest.java    |   2 +-
 .../tokens/DelegationTokenImplTest.java         |  73 ++
 .../security/tokens/DelegationTokenTest.java    |  72 --
 .../accumulo/core/data/KeyExtentTest.java       |   1 +
 .../apache/accumulo/core/data/RangeTest.java    |   1 +
 .../accumulo/core/file/rfile/RFileTest.java     |   2 +-
 .../core/iterators/IteratorUtilTest.java        |   2 +-
 .../iterators/system/MultiIteratorTest.java     |   2 +-
 .../core/metadata/MetadataServicerTest.java     |   2 +-
 .../ReplicationConfigurationUtilTest.java       |   2 +-
 .../core/rpc/SaslConnectionParamsTest.java      |  12 +-
 .../AuthenticationTokenIdentifierTest.java      |   1 +
 .../accumulo/core/security/CredentialsTest.java |   1 +
 .../core/security/VisibilityConstraintTest.java |   1 +
 .../apache/accumulo/core/util/MergeTest.java    |   2 +-
 .../src/test/resources/multi-valued.client.conf |  17 +
 .../simple/client/RandomBatchWriter.java        |   6 +-
 minicluster/pom.xml                             |  29 +
 .../minicluster/MiniAccumuloInstance.java       |   5 +-
 .../impl/MiniAccumuloClusterImpl.java           |   2 +-
 pom.xml                                         |   5 +
 .../org/apache/accumulo/proxy/ProxyServer.java  |  11 +-
 .../accumulo/server/AccumuloServerContext.java  |   2 +-
 .../accumulo/server/client/BulkImporter.java    |   2 +-
 .../server/client/ClientServiceHandler.java     |   2 +-
 .../accumulo/server/client/HdfsZooInstance.java |   6 +-
 .../server/conf/ServerConfiguration.java        |   2 +-
 .../server/conf/ServerConfigurationFactory.java |   2 +-
 .../server/constraints/MetadataConstraints.java |   2 +-
 .../accumulo/server/fs/VolumeManagerImpl.java   |   2 +-
 .../apache/accumulo/server/fs/VolumeUtil.java   |   2 +-
 .../apache/accumulo/server/init/Initialize.java |   2 +-
 .../accumulo/server/master/LiveTServerSet.java  |   2 +-
 .../master/balancer/ChaoticLoadBalancer.java    |   2 +-
 .../master/balancer/DefaultLoadBalancer.java    |   2 +-
 .../server/master/balancer/GroupBalancer.java   |   2 +-
 .../master/balancer/RegexGroupBalancer.java     |   2 +-
 .../master/balancer/TableLoadBalancer.java      |   2 +-
 .../server/master/balancer/TabletBalancer.java  |   2 +-
 .../server/master/state/Assignment.java         |   2 +-
 .../server/master/state/CurrentState.java       |   2 +-
 .../accumulo/server/master/state/MergeInfo.java |   2 +-
 .../master/state/MetaDataTableScanner.java      |   2 +-
 .../master/state/TabletLocationState.java       |   2 +-
 .../server/master/state/TabletMigration.java    |   2 +-
 .../master/state/TabletStateChangeIterator.java |   2 +-
 .../rpc/SaslServerDigestCallbackHandler.java    |   2 +-
 .../TCredentialsUpdatingInvocationHandler.java  |   4 +-
 .../security/AuditedSecurityOperation.java      |   4 +-
 .../server/security/SecurityOperation.java      |   2 +-
 .../server/security/SystemCredentials.java      |   2 +-
 .../AuthenticationTokenSecretManager.java       |   6 +-
 .../security/handler/KerberosAuthenticator.java |   4 +-
 .../tabletserver/LargestFirstMemoryManager.java |   2 +-
 .../tabletserver/MemoryManagementActions.java   |   2 +-
 .../server/tabletserver/MemoryManager.java      |   2 +-
 .../server/tabletserver/TabletState.java        |   2 +-
 .../org/apache/accumulo/server/util/Admin.java  |   2 +-
 .../server/util/CheckForMetadataProblems.java   |   2 +-
 .../apache/accumulo/server/util/FileUtil.java   |   2 +-
 .../server/util/FindOfflineTablets.java         |   2 +-
 .../server/util/MasterMetadataUtil.java         |   2 +-
 .../accumulo/server/util/MetadataTableUtil.java |   4 +-
 .../util/RemoveEntriesForMissingFiles.java      |   4 +-
 .../server/util/ReplicationTableUtil.java       |   4 +-
 .../accumulo/server/util/TableDiskUsage.java    |   2 +-
 .../accumulo/server/util/TabletIterator.java    |   2 +-
 .../server/util/VerifyTabletAssignments.java    |   4 +-
 .../server/AccumuloServerContextTest.java       |   2 +-
 .../server/client/BulkImporterTest.java         |   4 +-
 .../balancer/ChaoticLoadBalancerTest.java       |   2 +-
 .../balancer/DefaultLoadBalancerTest.java       |   2 +-
 .../master/balancer/GroupBalancerTest.java      |   2 +-
 .../master/balancer/TableLoadBalancerTest.java  |   2 +-
 .../server/master/state/MergeInfoTest.java      |   2 +-
 .../master/state/TabletLocationStateTest.java   |   2 +-
 .../rpc/SaslDigestCallbackHandlerTest.java      |   2 +-
 .../server/security/SystemCredentialsTest.java  |   2 +-
 .../AuthenticationTokenSecretManagerTest.java   |   2 +-
 .../apache/accumulo/server/util/CloneTest.java  |   2 +-
 .../server/util/ReplicationTableUtilTest.java   |   4 +-
 .../server/util/TabletIteratorTest.java         |   2 +-
 .../accumulo/gc/GarbageCollectionAlgorithm.java |   2 +-
 .../accumulo/gc/GarbageCollectionTest.java      |   2 +-
 .../accumulo/gc/SimpleGarbageCollectorTest.java |   2 +-
 .../CloseWriteAheadLogReferencesTest.java       |   1 +
 .../java/org/apache/accumulo/master/Master.java |   2 +-
 .../master/MasterClientServiceHandler.java      |   4 +-
 .../accumulo/master/TabletGroupWatcher.java     |   2 +-
 .../master/recovery/RecoveryManager.java        |   2 +-
 .../accumulo/master/state/MergeStats.java       |   2 +-
 .../accumulo/master/tableOps/BulkImport.java    |   2 +-
 .../accumulo/master/tableOps/CompactRange.java  |   2 +-
 .../accumulo/master/tableOps/CreateTable.java   |   2 +-
 .../accumulo/master/tableOps/DeleteTable.java   |   2 +-
 .../accumulo/master/tableOps/ExportTable.java   |   2 +-
 .../accumulo/master/tableOps/ImportTable.java   |   2 +-
 .../accumulo/master/tableOps/TableRangeOp.java  |   2 +-
 .../master/ReplicationOperationsImplTest.java   |   4 +-
 .../apache/accumulo/master/TestMergeState.java  |   2 +-
 .../replication/SequentialWorkAssignerTest.java |   2 +-
 .../master/replication/StatusMakerTest.java     |   2 +-
 .../replication/UnorderedWorkAssignerTest.java  |   2 +-
 .../master/state/RootTabletStateStoreTest.java  |   2 +-
 .../monitor/servlets/TServersServlet.java       |   2 +-
 .../monitor/servlets/TablesServlet.java         |   2 +-
 .../tserver/ActiveAssignmentRunnable.java       |   2 +-
 .../tserver/ConditionalMutationSet.java         |   2 +-
 .../apache/accumulo/tserver/FileManager.java    |   2 +-
 .../org/apache/accumulo/tserver/RowLocks.java   |   2 +-
 .../apache/accumulo/tserver/TabletServer.java   |   2 +-
 .../tserver/TabletServerResourceManager.java    |   2 +-
 .../accumulo/tserver/TservConstraintEnv.java    |   2 +-
 .../apache/accumulo/tserver/WriteTracker.java   |   2 +-
 .../compaction/MajorCompactionRequest.java      |   2 +-
 .../tserver/constraints/ConstraintChecker.java  |   2 +-
 .../apache/accumulo/tserver/log/DfsLogger.java  |   2 +-
 .../accumulo/tserver/log/SortedLogRecovery.java |   2 +-
 .../tserver/log/TabletServerLogger.java         |   2 +-
 .../accumulo/tserver/logger/LogFileKey.java     |   2 +-
 .../accumulo/tserver/logger/LogReader.java      |   2 +-
 .../mastermessage/SplitReportMessage.java       |   2 +-
 .../mastermessage/TabletStatusMessage.java      |   2 +-
 .../replication/AccumuloReplicaSystem.java      |   2 +-
 .../accumulo/tserver/scan/LookupTask.java       |   2 +-
 .../tserver/session/MultiScanSession.java       |   2 +-
 .../accumulo/tserver/session/ScanSession.java   |   2 +-
 .../accumulo/tserver/session/UpdateSession.java |   2 +-
 .../accumulo/tserver/tablet/CommitSession.java  |   2 +-
 .../accumulo/tserver/tablet/CompactionInfo.java |   2 +-
 .../accumulo/tserver/tablet/Compactor.java      |   2 +-
 .../tserver/tablet/DatafileManager.java         |   2 +-
 .../apache/accumulo/tserver/tablet/Tablet.java  |   2 +-
 .../tserver/tablet/TabletCommitter.java         |   2 +-
 .../LargestFirstMemoryManagerTest.java          |   2 +-
 .../accumulo/tserver/AssignmentWatcherTest.java |   2 +-
 .../tserver/CheckTabletMetadataTest.java        |   2 +-
 .../tserver/TabletResourceManagerTest.java      |   2 +-
 .../DefaultCompactionStrategyTest.java          |   2 +-
 .../SizeLimitCompactionStrategyTest.java        |   2 +-
 .../ConfigurableCompactionStrategyTest.java     |   2 +-
 .../constraints/ConstraintCheckerTest.java      |   2 +-
 .../tserver/log/SortedLogRecoveryTest.java      |   2 +-
 .../accumulo/tserver/logger/LogFileTest.java    |   2 +-
 .../replication/AccumuloReplicaSystemTest.java  |   2 +-
 .../replication/ReplicationProcessorTest.java   |   2 +-
 .../accumulo/tserver/tablet/TabletTest.java     |   2 +-
 .../apache/accumulo/shell/ShellOptionsJC.java   |   4 +-
 .../commands/ActiveCompactionIterator.java      |   2 +-
 .../shell/commands/ActiveScanIterator.java      |   2 +-
 .../shell/commands/CreateTableCommand.java      |   2 +-
 .../shell/commands/GetSplitsCommand.java        |   2 +-
 .../accumulo/shell/commands/InsertCommand.java  |   6 +-
 .../accumulo/test/QueryMetadataTable.java       |   2 +-
 .../org/apache/accumulo/test/TestIngest.java    |   6 +-
 .../apache/accumulo/test/WrongTabletTest.java   |   4 +-
 .../continuous/ContinuousStatsCollector.java    |   4 +-
 .../metadata/MetadataBatchScanTest.java         |   2 +-
 .../performance/scan/CollectTabletStats.java    |   4 +-
 .../test/performance/thrift/NullTserver.java    |   2 +-
 .../randomwalk/security/AlterTablePerm.java     |   2 +-
 .../test/randomwalk/security/Authenticate.java  |   2 +-
 .../test/randomwalk/security/ChangePass.java    |   2 +-
 .../test/randomwalk/security/DropTable.java     |   2 +-
 .../test/randomwalk/security/SetAuths.java      |   2 +-
 .../randomwalk/security/WalkingSecurity.java    |   2 +-
 .../randomwalk/sequential/MapRedVerifyTool.java |   4 +-
 .../StandaloneAccumuloClusterConfiguration.java |   3 +-
 .../server/security/SystemCredentialsIT.java    |   2 +-
 .../test/MasterRepairsDualAssignmentIT.java     |   4 +-
 .../accumulo/test/MetaConstraintRetryIT.java    |   4 +-
 .../accumulo/test/MultiTableBatchWriterIT.java  |   2 +-
 .../org/apache/accumulo/test/NamespacesIT.java  |   6 +-
 .../org/apache/accumulo/test/ShellServerIT.java |   3 +-
 .../apache/accumulo/test/SplitRecoveryIT.java   |   2 +-
 .../accumulo/test/TransportCachingIT.java       |   2 +-
 .../java/org/apache/accumulo/test/VolumeIT.java |   2 +-
 .../test/functional/AccumuloInputFormatIT.java  |   2 +-
 .../functional/BalanceAfterCommsFailureIT.java  |   2 +-
 .../BalanceInPresenceOfOfflineTableIT.java      |   2 +-
 .../test/functional/ConfigurableMacIT.java      |   3 +-
 .../test/functional/DynamicThreadPoolsIT.java   |   2 +-
 .../accumulo/test/functional/KerberosIT.java    |  37 +-
 .../test/functional/MasterAssignmentIT.java     |   4 +-
 .../test/functional/MetadataMaxFilesIT.java     |   2 +-
 .../accumulo/test/functional/PermissionsIT.java |   2 +-
 .../functional/SimpleBalancerFairnessIT.java    |   2 +-
 .../accumulo/test/functional/SplitIT.java       |   2 +-
 .../test/functional/SplitRecoveryIT.java        |   2 +-
 .../accumulo/test/functional/TableIT.java       |   2 +-
 .../functional/TabletStateChangeIteratorIT.java |   2 +-
 ...bageCollectorCommunicatesWithTServersIT.java |   2 +-
 ...UnusedWalDoesntCloseReplicationStatusIT.java |   2 +-
 276 files changed, 2699 insertions(+), 1798 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/fb25f3b1/core/src/main/java/org/apache/accumulo/core/metadata/RootTable.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/fb25f3b1/core/src/main/java/org/apache/accumulo/core/tabletserver/log/LogEntry.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/apache/accumulo/core/tabletserver/log/LogEntry.java
index 90ce692,7fe61d1..ab70bb0
--- a/core/src/main/java/org/apache/accumulo/core/tabletserver/log/LogEntry.java
+++ b/core/src/main/java/org/apache/accumulo/core/tabletserver/log/LogEntry.java
@@@ -16,14 -16,14 +16,14 @@@
   */
  package org.apache.accumulo.core.tabletserver.log;
  
 +import static java.nio.charset.StandardCharsets.UTF_8;
 +
  import java.io.IOException;
 -import java.util.ArrayList;
  import java.util.Arrays;
 -import java.util.Collection;
  
  import org.apache.accumulo.core.data.Key;
- import org.apache.accumulo.core.data.KeyExtent;
  import org.apache.accumulo.core.data.Value;
+ import org.apache.accumulo.core.data.impl.KeyExtent;
  import org.apache.accumulo.core.metadata.schema.MetadataSchema;
  import org.apache.hadoop.io.DataInputBuffer;
  import org.apache.hadoop.io.DataOutputBuffer;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/fb25f3b1/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeUtil.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/fb25f3b1/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/fb25f3b1/server/base/src/main/java/org/apache/accumulo/server/master/state/MetaDataTableScanner.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/fb25f3b1/server/base/src/main/java/org/apache/accumulo/server/master/state/TabletLocationState.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/fb25f3b1/server/base/src/main/java/org/apache/accumulo/server/util/MasterMetadataUtil.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/fb25f3b1/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/fb25f3b1/server/base/src/main/java/org/apache/accumulo/server/util/ReplicationTableUtil.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/fb25f3b1/server/base/src/test/java/org/apache/accumulo/server/util/ReplicationTableUtilTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/fb25f3b1/server/gc/src/test/java/org/apache/accumulo/gc/replication/CloseWriteAheadLogReferencesTest.java
----------------------------------------------------------------------
diff --cc server/gc/src/test/java/org/apache/accumulo/gc/replication/CloseWriteAheadLogReferencesTest.java
index 9fcfec9,3115de1..78a5bd5
--- a/server/gc/src/test/java/org/apache/accumulo/gc/replication/CloseWriteAheadLogReferencesTest.java
+++ b/server/gc/src/test/java/org/apache/accumulo/gc/replication/CloseWriteAheadLogReferencesTest.java
@@@ -48,9 -48,11 +48,10 @@@ import org.apache.accumulo.core.conf.Si
  import org.apache.accumulo.core.data.Key;
  import org.apache.accumulo.core.data.Mutation;
  import org.apache.accumulo.core.data.Value;
+ import org.apache.accumulo.core.data.impl.KeyExtent;
  import org.apache.accumulo.core.metadata.MetadataTable;
 +import org.apache.accumulo.core.metadata.schema.MetadataSchema.CurrentLogsSection;
  import org.apache.accumulo.core.metadata.schema.MetadataSchema.ReplicationSection;
 -import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
 -import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LogColumnFamily;
  import org.apache.accumulo.core.protobuf.ProtobufUtil;
  import org.apache.accumulo.core.replication.ReplicationSchema.StatusSection;
  import org.apache.accumulo.core.replication.ReplicationTable;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/fb25f3b1/server/master/src/main/java/org/apache/accumulo/master/Master.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/fb25f3b1/server/master/src/main/java/org/apache/accumulo/master/MasterClientServiceHandler.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/fb25f3b1/server/master/src/main/java/org/apache/accumulo/master/TabletGroupWatcher.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/fb25f3b1/server/master/src/main/java/org/apache/accumulo/master/state/MergeStats.java
----------------------------------------------------------------------
diff --cc server/master/src/main/java/org/apache/accumulo/master/state/MergeStats.java
index 838cce8,a3c7e46..f1763be
--- a/server/master/src/main/java/org/apache/accumulo/master/state/MergeStats.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/state/MergeStats.java
@@@ -25,12 -25,11 +25,12 @@@ import org.apache.accumulo.core.client.
  import org.apache.accumulo.core.client.Scanner;
  import org.apache.accumulo.core.client.TableNotFoundException;
  import org.apache.accumulo.core.data.Key;
- import org.apache.accumulo.core.data.KeyExtent;
  import org.apache.accumulo.core.data.Range;
  import org.apache.accumulo.core.data.Value;
+ import org.apache.accumulo.core.data.impl.KeyExtent;
  import org.apache.accumulo.core.metadata.MetadataTable;
  import org.apache.accumulo.core.metadata.RootTable;
 +import org.apache.accumulo.core.metadata.schema.MetadataSchema;
  import org.apache.accumulo.core.security.Authorizations;
  import org.apache.accumulo.core.zookeeper.ZooUtil;
  import org.apache.accumulo.server.cli.ClientOpts;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/fb25f3b1/server/master/src/test/java/org/apache/accumulo/master/ReplicationOperationsImplTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/fb25f3b1/server/master/src/test/java/org/apache/accumulo/master/TestMergeState.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/fb25f3b1/server/master/src/test/java/org/apache/accumulo/master/state/RootTabletStateStoreTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/fb25f3b1/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/fb25f3b1/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/fb25f3b1/server/tserver/src/main/java/org/apache/accumulo/tserver/log/SortedLogRecovery.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/fb25f3b1/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
----------------------------------------------------------------------
diff --cc server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
index 7d16aae,1d385d9..bc77ffb
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
@@@ -35,13 -33,11 +35,13 @@@ import java.util.concurrent.locks.ReadW
  import java.util.concurrent.locks.ReentrantReadWriteLock;
  
  import org.apache.accumulo.core.client.Durability;
- import org.apache.accumulo.core.data.KeyExtent;
  import org.apache.accumulo.core.data.Mutation;
+ import org.apache.accumulo.core.data.impl.KeyExtent;
  import org.apache.accumulo.core.protobuf.ProtobufUtil;
  import org.apache.accumulo.core.replication.ReplicationConfigurationUtil;
 +import org.apache.accumulo.core.util.SimpleThreadPool;
  import org.apache.accumulo.core.util.UtilWaitThread;
 +import org.apache.accumulo.server.TabletLevel;
  import org.apache.accumulo.server.conf.TableConfiguration;
  import org.apache.accumulo.server.fs.VolumeManager;
  import org.apache.accumulo.server.replication.StatusUtil;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/fb25f3b1/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CommitSession.java
----------------------------------------------------------------------
diff --cc server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CommitSession.java
index 41b7795,d908f1d..dee705c
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CommitSession.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CommitSession.java
@@@ -16,10 -16,11 +16,10 @@@
   */
  package org.apache.accumulo.tserver.tablet;
  
 -import java.util.ArrayList;
  import java.util.List;
  
- import org.apache.accumulo.core.data.KeyExtent;
  import org.apache.accumulo.core.data.Mutation;
+ import org.apache.accumulo.core.data.impl.KeyExtent;
  import org.apache.accumulo.tserver.InMemoryMap;
  import org.apache.accumulo.tserver.log.DfsLogger;
  import org.slf4j.Logger;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/fb25f3b1/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/DatafileManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/fb25f3b1/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/fb25f3b1/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletCommitter.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/fb25f3b1/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/fb25f3b1/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/fb25f3b1/test/src/test/java/org/apache/accumulo/test/VolumeIT.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/fb25f3b1/test/src/test/java/org/apache/accumulo/test/replication/GarbageCollectorCommunicatesWithTServersIT.java
----------------------------------------------------------------------


[11/34] accumulo git commit: ACCUMULO-3423 merging master

Posted by ec...@apache.org.
ACCUMULO-3423 merging master


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

Branch: refs/heads/master
Commit: f1591f033ed010f490686b836fe6d6d52cb98238
Parents: f4f28c7 5c670fa
Author: Eric C. Newton <er...@gmail.com>
Authored: Tue Mar 17 08:49:11 2015 -0400
Committer: Eric C. Newton <er...@gmail.com>
Committed: Tue Mar 17 08:49:11 2015 -0400

----------------------------------------------------------------------
 TESTING.md                                      |   28 +-
 core/pom.xml                                    |   27 -
 core/src/main/findbugs/exclude-filter.xml       |   46 +-
 .../accumulo/core/bloomfilter/BloomFilter.java  |    3 -
 .../client/admin/ReplicationOperations.java     |   13 +-
 .../core/client/impl/ClientContext.java         |    4 +-
 .../core/client/impl/ConditionalWriterImpl.java |    9 +-
 .../client/impl/NamespaceOperationsImpl.java    |    8 +-
 .../client/impl/ReplicationOperationsImpl.java  |  125 +-
 .../core/client/impl/TableOperationsImpl.java   |   17 +
 .../core/client/impl/TabletLocator.java         |    1 -
 .../client/mapreduce/AbstractInputFormat.java   |   16 +-
 .../core/client/replication/ReplicaSystem.java  |   50 -
 .../replication/ReplicaSystemFactory.java       |   82 --
 .../tokens/CredentialProviderToken.java         |    2 +-
 .../client/security/tokens/DelegationToken.java |    2 +-
 .../client/security/tokens/KerberosToken.java   |    7 +-
 .../core/client/security/tokens/NullToken.java  |    6 +-
 .../accumulo/core/conf/SiteConfiguration.java   |    4 +-
 .../core/file/blockfile/cache/CachedBlock.java  |   14 +
 .../core/file/blockfile/cache/ClassSize.java    |   47 +-
 .../file/blockfile/cache/LruBlockCache.java     |    6 +
 .../accumulo/core/iterators/OrIterator.java     |   12 +
 .../core/iterators/user/AgeOffFilter.java       |    1 -
 .../core/iterators/user/TimestampFilter.java    |    4 +-
 .../user/WholeColumnFamilyIterator.java         |    6 +-
 .../core/master/thrift/MasterClientService.java | 1213 ++++++++++++++++++
 .../replication/PrintReplicationRecords.java    |   97 --
 .../core/replication/ReplicaSystemHelper.java   |   72 --
 .../core/replication/ReplicationTable.java      |    1 -
 .../core/replication/StatusFormatter.java       |  187 ---
 .../accumulo/core/replication/StatusUtil.java   |  225 ----
 .../core/replication/proto/Replication.java     |  949 --------------
 .../apache/accumulo/core/rpc/ThriftUtil.java    |    8 +-
 .../CachingHDFSSecretKeyEncryptionStrategy.java |    2 +-
 .../apache/accumulo/core/util/CreateToken.java  |    6 +-
 .../org/apache/accumulo/core/util/Version.java  |    6 +-
 .../core/util/format/DefaultFormatter.java      |   35 +-
 core/src/main/protobuf/replication.proto        |   26 -
 core/src/main/scripts/generate-protobuf.sh      |   98 --
 core/src/main/thrift/master.thrift              |    3 +
 .../accumulo/core/cli/TestClientOpts.java       |    6 +-
 .../client/impl/TableOperationsHelperTest.java  |    4 +-
 .../mapred/AccumuloFileOutputFormatTest.java    |   15 +-
 .../mapred/AccumuloRowInputFormatTest.java      |    8 +-
 .../mapreduce/AccumuloFileOutputFormatTest.java |   11 +-
 .../mapreduce/AccumuloRowInputFormatTest.java   |    8 +-
 .../conf/CredentialProviderFactoryShimTest.java |    8 +-
 .../iterators/aggregation/NumSummationTest.java |   17 +-
 .../ReplicationOperationsImplTest.java          |  409 ------
 .../core/replication/StatusUtilTest.java        |   57 -
 .../core/replication/proto/StatusTest.java      |   36 -
 .../core/util/LocalityGroupUtilTest.java        |    4 +-
 .../core/util/format/HexFormatterTest.java      |    2 +-
 .../simple/src/main/findbugs/exclude-filter.xml |    5 +
 .../simple/client/RandomBatchScanner.java       |    3 +-
 .../examples/simple/client/RowOperations.java   |   30 +-
 .../simple/filedata/ChunkInputFormatTest.java   |   16 +-
 .../simple/filedata/ChunkInputStreamTest.java   |   40 +-
 fate/src/main/findbugs/exclude-filter.xml       |    5 +
 .../apache/accumulo/maven/plugin/StartMojo.java |    4 +-
 .../impl/MiniAccumuloClusterImpl.java           |   28 +-
 .../MiniAccumuloClusterStartStopTest.java       |    6 +-
 .../minicluster/MiniAccumuloClusterTest.java    |    5 +-
 .../impl/MiniAccumuloClusterImplTest.java       |    6 +-
 pom.xml                                         |    2 +-
 proxy/src/main/findbugs/exclude-filter.xml      |    7 +-
 .../org/apache/accumulo/proxy/ProxyServer.java  |   30 +-
 server/base/pom.xml                             |   29 +
 .../base/src/main/findbugs/exclude-filter.xml   |   12 +
 .../server/constraints/MetadataConstraints.java |    2 +-
 .../accumulo/server/data/ServerMutation.java    |   13 +-
 .../apache/accumulo/server/fs/VolumeUtil.java   |    4 +-
 .../apache/accumulo/server/init/Initialize.java |    3 +-
 .../iterators/MetadataBulkLoadFilter.java       |    2 +-
 .../accumulo/server/master/LiveTServerSet.java  |   25 +-
 .../master/balancer/DefaultLoadBalancer.java    |   11 +
 .../server/metrics/AbstractMetricsImpl.java     |    5 +-
 .../replication/PrintReplicationRecords.java    |   98 ++
 .../server/replication/ReplicaSystem.java       |   49 +
 .../replication/ReplicaSystemFactory.java       |   82 ++
 .../server/replication/ReplicaSystemHelper.java |   74 ++
 .../server/replication/ReplicationUtil.java     |    6 +-
 .../server/replication/StatusCombiner.java      |    6 +-
 .../server/replication/StatusFormatter.java     |  170 +++
 .../accumulo/server/replication/StatusUtil.java |  225 ++++
 .../server/replication/proto/Replication.java   |  949 ++++++++++++++
 .../org/apache/accumulo/server/util/Admin.java  |   16 +-
 .../server/util/ReplicationTableUtil.java       |    4 +-
 server/base/src/main/protobuf/replication.proto |   26 +
 .../base/src/main/scripts/generate-protobuf.sh  |   98 ++
 .../master/balancer/GroupBalancerTest.java      |    2 +-
 .../master/balancer/TableLoadBalancerTest.java  |    8 +-
 .../server/replication/StatusCombinerTest.java  |    6 +-
 .../server/replication/StatusUtilTest.java      |   54 +
 .../server/replication/proto/StatusTest.java    |   36 +
 .../server/security/SystemCredentialsTest.java  |    8 +-
 .../accumulo/server/util/FileUtilTest.java      |   39 +-
 .../server/util/ReplicationTableUtilTest.java   |    7 +-
 server/gc/src/main/findbugs/exclude-filter.xml  |    5 +
 .../gc/GarbageCollectWriteAheadLogs.java        |    4 +-
 .../accumulo/gc/GarbageCollectionAlgorithm.java |    4 +-
 .../gc/GarbageCollectionEnvironment.java        |    2 +-
 .../accumulo/gc/SimpleGarbageCollector.java     |    2 +-
 .../CloseWriteAheadLogReferences.java           |    4 +-
 .../accumulo/gc/GarbageCollectionTest.java      |    4 +-
 .../CloseWriteAheadLogReferencesTest.java       |    4 +-
 .../master/src/main/findbugs/exclude-filter.xml |   20 +
 .../accumulo/master/FateServiceHandler.java     |    2 +-
 .../java/org/apache/accumulo/master/Master.java |   13 +-
 .../master/MasterClientServiceHandler.java      |  111 +-
 .../accumulo/master/TabletGroupWatcher.java     |    3 +-
 .../DistributedWorkQueueWorkAssigner.java       |    5 +-
 .../master/replication/FinishedWorkUpdater.java |    4 +-
 .../RemoveCompleteReplicationRecords.java       |    4 +-
 .../master/replication/StatusMaker.java         |    2 +-
 .../accumulo/master/replication/WorkMaker.java  |    4 +-
 .../accumulo/master/tableOps/BulkImport.java    |    7 +-
 .../accumulo/master/tableOps/ImportTable.java   |   39 +-
 .../accumulo/master/tableOps/TraceRepo.java     |    2 +-
 .../master/ReplicationOperationsImplTest.java   |  452 +++++++
 .../replication/FinishedWorkUpdaterTest.java    |    5 +-
 .../RemoveCompleteReplicationRecordsTest.java   |    7 +-
 .../replication/SequentialWorkAssignerTest.java |    5 +-
 .../master/replication/StatusMakerTest.java     |    7 +-
 .../replication/UnorderedWorkAssignerTest.java  |    4 +-
 .../master/replication/WorkMakerTest.java       |    7 +-
 .../accumulo/monitor/ZooKeeperStatus.java       |   11 +
 .../accumulo/monitor/servlets/BasicServlet.java |    6 +-
 .../accumulo/monitor/servlets/JSONServlet.java  |    4 +-
 .../accumulo/monitor/servlets/LogServlet.java   |    2 +
 .../monitor/servlets/ProblemServlet.java        |   17 +-
 .../monitor/servlets/ReplicationServlet.java    |   25 +-
 .../accumulo/monitor/servlets/ShellServlet.java |   36 +-
 .../monitor/servlets/TServersServlet.java       |    2 +
 .../servlets/trace/NullKeyValueIterator.java    |    3 +-
 .../servlets/trace/ShowTraceLinkType.java       |    2 +
 .../monitor/servlets/trace/Summary.java         |    3 +
 .../monitor/util/celltypes/CellType.java        |    5 +-
 .../monitor/util/celltypes/CompactionsType.java |    1 +
 .../monitor/util/celltypes/DateTimeType.java    |    1 +
 .../monitor/util/celltypes/DurationType.java    |    1 +
 .../monitor/util/celltypes/NumberType.java      |    1 +
 .../monitor/util/celltypes/PercentageType.java  |    2 +
 .../util/celltypes/PreciseNumberType.java       |    2 +
 .../util/celltypes/ProgressChartType.java       |    1 +
 .../monitor/util/celltypes/StringType.java      |    2 +
 .../monitor/util/celltypes/TServerLinkType.java |    2 +
 .../monitor/util/celltypes/TableLinkType.java   |    1 +
 .../monitor/util/celltypes/TableStateType.java  |    2 +
 .../tracer/src/main/findbugs/exclude-filter.xml |    2 +-
 .../org/apache/accumulo/tracer/TraceServer.java |    2 +-
 .../src/main/findbugs/exclude-filter.xml        |    6 +
 .../tserver/TabletServerResourceManager.java    |    5 +
 .../compaction/MajorCompactionRequest.java      |    5 +
 .../accumulo/tserver/log/MultiReader.java       |   11 +
 .../tserver/log/TabletServerLogger.java         |    4 +-
 .../tserver/metrics/TabletServerMBeanImpl.java  |    2 +-
 .../metrics/TabletServerMinCMetrics.java        |    2 +-
 .../metrics/TabletServerScanMetrics.java        |    2 +-
 .../metrics/TabletServerUpdateMetrics.java      |    2 +-
 .../replication/AccumuloReplicaSystem.java      |   11 +-
 .../replication/ReplicationProcessor.java       |   10 +-
 .../tserver/tablet/CompactionRunner.java        |   12 +
 .../tserver/tablet/DatafileManager.java         |    2 +-
 .../apache/accumulo/tserver/tablet/Tablet.java  |    4 +-
 .../tserver/log/SortedLogRecoveryTest.java      |   11 +
 .../tserver/log/TestUpgradePathForWALogs.java   |    6 +-
 .../replication/AccumuloReplicaSystemTest.java  |    5 +-
 .../replication/ReplicationProcessorTest.java   |    9 +-
 .../accumulo/tserver/tablet/RootFilesTest.java  |    8 +-
 .../accumulo/tserver/tablet/TabletTest.java     |    8 +-
 shell/src/main/findbugs/exclude-filter.xml      |    5 +
 .../shell/commands/ExtensionCommand.java        |    5 +-
 .../apache/accumulo/shell/ShellConfigTest.java  |    7 +-
 .../org/apache/accumulo/shell/ShellTest.java    |    8 +-
 .../AccumuloReloadingVFSClassLoaderTest.java    |    8 +-
 test/src/main/findbugs/exclude-filter.xml       |   25 +-
 .../apache/accumulo/test/TestRandomDeletes.java |   13 +
 .../accumulo/test/continuous/Histogram.java     |   14 +
 .../test/functional/CacheTestClean.java         |    7 +-
 .../test/functional/CacheTestReader.java        |    6 +-
 .../accumulo/test/randomwalk/bulk/Verify.java   |    2 +-
 .../randomwalk/security/WalkingSecurity.java    |    4 +-
 .../test/replication/MockReplicaSystem.java     |    6 +-
 .../ReplicationTablesPrinterThread.java         |    2 +-
 .../accumulo/fate/zookeeper/ZooLockTest.java    |    2 +-
 .../org/apache/accumulo/harness/AccumuloIT.java |   26 +-
 .../accumulo/harness/MiniClusterHarness.java    |    9 +-
 .../accumulo/harness/SharedMiniClusterIT.java   |    4 +-
 .../org/apache/accumulo/harness/TestingKdc.java |    7 +-
 .../StandaloneAccumuloClusterConfiguration.java |    2 +-
 .../apache/accumulo/proxy/SimpleProxyBase.java  |   19 +-
 .../apache/accumulo/test/AuditMessageIT.java    |    7 +-
 .../accumulo/test/BulkImportVolumeIT.java       |    4 +-
 .../accumulo/test/ConditionalWriterIT.java      |    8 +-
 .../org/apache/accumulo/test/ExistingMacIT.java |    6 +-
 .../MissingWalHeaderCompletesRecoveryIT.java    |    6 +-
 .../org/apache/accumulo/test/NamespacesIT.java  |    4 +-
 .../test/RewriteTabletDirectoriesIT.java        |    4 +-
 .../org/apache/accumulo/test/ShellServerIT.java |    4 +-
 .../accumulo/test/UserCompactionStrategyIT.java |    5 +
 .../apache/accumulo/test/VolumeChooserIT.java   |    6 +-
 .../java/org/apache/accumulo/test/VolumeIT.java |    8 +-
 .../test/functional/ConfigurableMacIT.java      |    9 +-
 .../accumulo/test/functional/DeleteIT.java      |   15 +-
 .../functional/DeleteTableDuringSplitIT.java    |    4 +-
 .../test/functional/HalfDeadTServerIT.java      |    6 +-
 .../accumulo/test/functional/KerberosIT.java    |   16 +-
 .../test/functional/MonitorLoggingIT.java       |    2 +-
 .../accumulo/test/functional/MonitorSslIT.java  |    4 +-
 .../accumulo/test/functional/PermissionsIT.java |   10 +-
 .../accumulo/test/functional/ScanIdIT.java      |    9 +-
 .../apache/accumulo/test/functional/SslIT.java  |    2 +-
 .../accumulo/test/functional/ZooCacheIT.java    |    6 +-
 .../test/replication/CyclicReplicationIT.java   |   10 +-
 ...bageCollectorCommunicatesWithTServersIT.java |    2 +-
 .../replication/MultiInstanceReplicationIT.java |    6 +-
 .../test/replication/ReplicationIT.java         |    8 +-
 .../test/replication/StatusCombinerMacIT.java   |    4 +-
 .../UnorderedWorkAssignerReplicationIT.java     |    6 +-
 ...UnusedWalDoesntCloseReplicationStatusIT.java |   11 +-
 .../test/security/KerberosTokenTest.java        |   12 +-
 .../apache/accumulo/test/util/CertUtils.java    |   20 +-
 .../accumulo/test/util/CertUtilsTest.java       |   32 +-
 trace/src/main/findbugs/exclude-filter.xml      |    5 +-
 226 files changed, 4736 insertions(+), 3008 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/f1591f03/core/src/main/java/org/apache/accumulo/core/client/impl/ReplicationOperationsImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f1591f03/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeUtil.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f1591f03/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f1591f03/server/base/src/main/java/org/apache/accumulo/server/replication/StatusUtil.java
----------------------------------------------------------------------
diff --cc server/base/src/main/java/org/apache/accumulo/server/replication/StatusUtil.java
index 0000000,898e3d4..e973ebc
mode 000000,100644..100644
--- a/server/base/src/main/java/org/apache/accumulo/server/replication/StatusUtil.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/replication/StatusUtil.java
@@@ -1,0 -1,216 +1,225 @@@
+ /*
+  * 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.replication;
+ 
+ import org.apache.accumulo.core.data.Value;
+ import org.apache.accumulo.core.protobuf.ProtobufUtil;
+ import org.apache.accumulo.server.replication.proto.Replication.Status;
+ import org.apache.accumulo.server.replication.proto.Replication.Status.Builder;
+ 
+ import com.google.protobuf.InvalidProtocolBufferException;
+ 
+ /**
+  * Helper methods to create Status protobuf messages
+  */
+ public class StatusUtil {
+ 
+   private static final Status INF_END_REPLICATION_STATUS, CLOSED_STATUS;
+   private static final Value INF_END_REPLICATION_STATUS_VALUE, CLOSED_STATUS_VALUE;
+ 
+   private static final Status.Builder CREATED_STATUS_BUILDER;
++  private static final Status.Builder INF_END_REPLICATION_STATUS_BUILDER;
+ 
+   static {
+     CREATED_STATUS_BUILDER = Status.newBuilder();
+     CREATED_STATUS_BUILDER.setBegin(0);
+     CREATED_STATUS_BUILDER.setEnd(0);
+     CREATED_STATUS_BUILDER.setInfiniteEnd(false);
+     CREATED_STATUS_BUILDER.setClosed(false);
+ 
+     Builder builder = Status.newBuilder();
+     builder.setBegin(0);
+     builder.setEnd(0);
+     builder.setInfiniteEnd(true);
+     builder.setClosed(false);
++    INF_END_REPLICATION_STATUS_BUILDER = builder;
+     INF_END_REPLICATION_STATUS = builder.build();
+     INF_END_REPLICATION_STATUS_VALUE = ProtobufUtil.toValue(INF_END_REPLICATION_STATUS);
+ 
+     builder = Status.newBuilder();
+     builder.setBegin(0);
+     builder.setEnd(0);
+     builder.setInfiniteEnd(true);
+     builder.setClosed(true);
+     CLOSED_STATUS = builder.build();
+     CLOSED_STATUS_VALUE = ProtobufUtil.toValue(CLOSED_STATUS);
+   }
+ 
+   /**
+    * Creates a {@link Status} for newly-created data that must be replicated
+    *
+    * @param recordsIngested
+    *          Offset of records which need to be replicated
+    * @return A {@link Status} tracking data that must be replicated
+    */
+   public static Status ingestedUntil(long recordsIngested) {
+     return ingestedUntil(Status.newBuilder(), recordsIngested);
+   }
+ 
+   public static Status ingestedUntil(Builder builder, long recordsIngested) {
+     return replicatedAndIngested(builder, 0, recordsIngested);
+   }
+ 
+   /**
+    * @param recordsReplicated
+    *          Offset of records which have been replicated
+    * @return A {@link Status} tracking data that must be replicated
+    */
+   public static Status replicated(long recordsReplicated) {
+     return replicated(Status.newBuilder(), recordsReplicated);
+   }
+ 
+   /**
+    * @param builder
+    *          Existing {@link Builder} to use
+    * @param recordsReplicated
+    *          Offset of records which have been replicated
+    * @returnA {@link Status} tracking data that must be replicated
+    */
+   public static Status replicated(Status.Builder builder, long recordsReplicated) {
+     return replicatedAndIngested(builder, recordsReplicated, 0);
+   }
+ 
+   /**
+    * Creates a @{link Status} for a file which has new data and data which has been replicated
+    *
+    * @param recordsReplicated
+    *          Offset of records which have been replicated
+    * @param recordsIngested
+    *          Offset for records which need to be replicated
+    * @return A {@link Status} for the given parameters
+    */
+   public static Status replicatedAndIngested(long recordsReplicated, long recordsIngested) {
+     return replicatedAndIngested(Status.newBuilder(), recordsReplicated, recordsIngested);
+   }
+ 
+   /**
+    * Same as {@link #replicatedAndIngested(long, long)} but uses the provided {@link Builder}
+    *
+    * @param builder
+    *          An existing builder
+    * @param recordsReplicated
+    *          Offset of records which have been replicated
+    * @param recordsIngested
+    *          Offset of records which need to be replicated
+    * @return A {@link Status} for the given parameters using the builder
+    */
+   public static Status replicatedAndIngested(Status.Builder builder, long recordsReplicated, long recordsIngested) {
+     return builder.setBegin(recordsReplicated).setEnd(recordsIngested).setClosed(false).setInfiniteEnd(false).build();
+   }
+ 
+   /**
+    * @return A {@link Status} for a new file that was just created
+    */
+   public static synchronized Status fileCreated(long timeCreated) {
+     // We're using a shared builder, so we need to synchronize access on it until we make a Status (which is then immutable)
+     CREATED_STATUS_BUILDER.setCreatedTime(timeCreated);
+     return CREATED_STATUS_BUILDER.build();
+   }
+ 
+   /**
+    * @return A {@link Value} for a new file that was just created
+    */
+   public static Value fileCreatedValue(long timeCreated) {
+     return ProtobufUtil.toValue(fileCreated(timeCreated));
+   }
+ 
+   /**
+    * @return A Status representing a closed file
+    */
+   public static Status fileClosed() {
+     return CLOSED_STATUS;
+   }
+ 
+   /**
+    * @return A Value representing a closed file
+    */
+   public static Value fileClosedValue() {
+     return CLOSED_STATUS_VALUE;
+   }
+ 
+   /**
+    * @return A {@link Status} for an open file of unspecified length, all of which needs replicating.
+    */
++  public static synchronized Status openWithUnknownLength(long timeCreated) {
++    return INF_END_REPLICATION_STATUS_BUILDER.setCreatedTime(timeCreated).build();
++  }
++
++  /**
++   * @return A {@link Status} for an open file of unspecified length, all of which needs replicating.
++   */
+   public static Status openWithUnknownLength() {
+     return INF_END_REPLICATION_STATUS;
+   }
+ 
+   /**
+    * @return A {@link Value} for an open file of unspecified length, all of which needs replicating.
+    */
+   public static Value openWithUnknownLengthValue() {
+     return INF_END_REPLICATION_STATUS_VALUE;
+   }
+ 
+   /**
+    * @param v
+    *          Value with serialized Status
+    * @return A Status created from the Value
+    */
+   public static Status fromValue(Value v) throws InvalidProtocolBufferException {
+     return Status.parseFrom(v.get());
+   }
+ 
+   /**
+    * Is the given Status fully replicated and is its file ready for deletion on the source
+    *
+    * @param status
+    *          a Status protobuf
+    * @return True if the file this Status references can be deleted.
+    */
+   public static boolean isSafeForRemoval(Status status) {
+     return status.getClosed() && isFullyReplicated(status);
+   }
+ 
+   /**
+    * Is the given Status fully replicated but potentially not yet safe for deletion
+    *
+    * @param status
+    *          a Status protobuf
+    * @return True if the file this Status references is fully replicated so far
+    */
+   public static boolean isFullyReplicated(Status status) {
+     if (status.getInfiniteEnd()) {
+       return Long.MAX_VALUE == status.getBegin();
+     } else {
+       return status.getBegin() >= status.getEnd();
+     }
+   }
+ 
+   /**
+    * Given the {@link Status}, is there replication work to be done
+    *
+    * @param status
+    *          Status for a file
+    * @return true if replication work is required
+    */
+   public static boolean isWorkRequired(Status status) {
+     if (status.getInfiniteEnd()) {
+       return Long.MAX_VALUE != status.getBegin();
+     } else {
+       return status.getBegin() < status.getEnd();
+     }
+   }
+ }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f1591f03/server/base/src/main/java/org/apache/accumulo/server/util/ReplicationTableUtil.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f1591f03/server/base/src/test/java/org/apache/accumulo/server/util/ReplicationTableUtilTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f1591f03/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogs.java
----------------------------------------------------------------------
diff --cc server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogs.java
index f44a9d1,1735c0d..cf068ed
--- a/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogs.java
+++ b/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogs.java
@@@ -47,25 -46,26 +47,25 @@@ import org.apache.accumulo.core.protobu
  import org.apache.accumulo.core.replication.ReplicationSchema.StatusSection;
  import org.apache.accumulo.core.replication.ReplicationTable;
  import org.apache.accumulo.core.replication.ReplicationTableOfflineException;
- import org.apache.accumulo.core.replication.StatusUtil;
- import org.apache.accumulo.core.replication.proto.Replication.Status;
 -import org.apache.accumulo.core.rpc.ThriftUtil;
  import org.apache.accumulo.core.security.Authorizations;
  import org.apache.accumulo.core.tabletserver.log.LogEntry;
 -import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
 -import org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Client;
 -import org.apache.accumulo.core.trace.Span;
  import org.apache.accumulo.core.trace.Trace;
 -import org.apache.accumulo.core.trace.Tracer;
 -import org.apache.accumulo.core.util.AddressUtil;
  import org.apache.accumulo.core.zookeeper.ZooUtil;
  import org.apache.accumulo.server.AccumuloServerContext;
 -import org.apache.accumulo.server.ServerConstants;
  import org.apache.accumulo.server.fs.VolumeManager;
 +import org.apache.accumulo.server.master.LiveTServerSet;
 +import org.apache.accumulo.server.master.LiveTServerSet.Listener;
 +import org.apache.accumulo.server.master.state.MetaDataStateStore;
 +import org.apache.accumulo.server.master.state.RootTabletStateStore;
 +import org.apache.accumulo.server.master.state.TServerInstance;
 +import org.apache.accumulo.server.master.state.TabletLocationState;
 +import org.apache.accumulo.server.master.state.TabletState;
+ import org.apache.accumulo.server.replication.StatusUtil;
+ import org.apache.accumulo.server.replication.proto.Replication.Status;
 -import org.apache.accumulo.server.util.MetadataTableUtil;
  import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
 -import org.apache.hadoop.fs.FileStatus;
  import org.apache.hadoop.fs.Path;
 -import org.apache.thrift.TException;
 +import org.apache.hadoop.io.Text;
 +import org.apache.htrace.Span;
  import org.apache.zookeeper.KeeperException;
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f1591f03/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f1591f03/server/gc/src/main/java/org/apache/accumulo/gc/replication/CloseWriteAheadLogReferences.java
----------------------------------------------------------------------
diff --cc server/gc/src/main/java/org/apache/accumulo/gc/replication/CloseWriteAheadLogReferences.java
index 8185f23,3a32727..6686cb8
--- a/server/gc/src/main/java/org/apache/accumulo/gc/replication/CloseWriteAheadLogReferences.java
+++ b/server/gc/src/main/java/org/apache/accumulo/gc/replication/CloseWriteAheadLogReferences.java
@@@ -37,13 -37,13 +37,11 @@@ import org.apache.accumulo.core.file.rf
  import org.apache.accumulo.core.master.thrift.MasterClientService;
  import org.apache.accumulo.core.metadata.MetadataTable;
  import org.apache.accumulo.core.metadata.schema.MetadataSchema;
 +import org.apache.accumulo.core.metadata.schema.MetadataSchema.CurrentLogsSection;
  import org.apache.accumulo.core.metadata.schema.MetadataSchema.ReplicationSection;
 -import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
 -import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LogColumnFamily;
  import org.apache.accumulo.core.replication.ReplicationTable;
- import org.apache.accumulo.core.replication.StatusUtil;
- import org.apache.accumulo.core.replication.proto.Replication.Status;
  import org.apache.accumulo.core.rpc.ThriftUtil;
  import org.apache.accumulo.core.security.Authorizations;
 -import org.apache.accumulo.core.tabletserver.log.LogEntry;
  import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
  import org.apache.accumulo.core.trace.Span;
  import org.apache.accumulo.core.trace.Trace;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f1591f03/server/gc/src/test/java/org/apache/accumulo/gc/replication/CloseWriteAheadLogReferencesTest.java
----------------------------------------------------------------------
diff --cc server/gc/src/test/java/org/apache/accumulo/gc/replication/CloseWriteAheadLogReferencesTest.java
index f47f14b,23db83a..9fcfec9
--- a/server/gc/src/test/java/org/apache/accumulo/gc/replication/CloseWriteAheadLogReferencesTest.java
+++ b/server/gc/src/test/java/org/apache/accumulo/gc/replication/CloseWriteAheadLogReferencesTest.java
@@@ -54,9 -54,10 +54,7 @@@ import org.apache.accumulo.core.metadat
  import org.apache.accumulo.core.protobuf.ProtobufUtil;
  import org.apache.accumulo.core.replication.ReplicationSchema.StatusSection;
  import org.apache.accumulo.core.replication.ReplicationTable;
- import org.apache.accumulo.core.replication.StatusUtil;
- import org.apache.accumulo.core.replication.proto.Replication.Status;
  import org.apache.accumulo.core.security.Authorizations;
 -import org.apache.accumulo.core.tabletserver.log.LogEntry;
  import org.apache.accumulo.core.trace.thrift.TInfo;
  import org.apache.accumulo.server.AccumuloServerContext;
  import org.apache.accumulo.server.conf.ServerConfigurationFactory;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f1591f03/server/master/src/main/java/org/apache/accumulo/master/Master.java
----------------------------------------------------------------------
diff --cc server/master/src/main/java/org/apache/accumulo/master/Master.java
index 4a37f86,ad0598d..f897162
--- a/server/master/src/main/java/org/apache/accumulo/master/Master.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/Master.java
@@@ -421,12 -422,9 +422,12 @@@ public class Master extends AccumuloSer
            perm.grantNamespacePermission(user, Namespaces.ACCUMULO_NAMESPACE_ID, NamespacePermission.READ);
          }
          perm.grantNamespacePermission("root", Namespaces.ACCUMULO_NAMESPACE_ID, NamespacePermission.ALTER_TABLE);
 +
 +        // add the currlog location for root tablet current logs
 +        zoo.putPersistentData(ZooUtil.getRoot(getInstance()) + RootTable.ZROOT_TABLET_CURRENT_LOGS, new byte[0], NodeExistsPolicy.SKIP);
          haveUpgradedZooKeeper = true;
        } catch (Exception ex) {
-         log.fatal("Error performing upgrade", ex);
+         log.error("Error performing upgrade", ex);
          System.exit(1);
        }
      }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f1591f03/server/master/src/main/java/org/apache/accumulo/master/MasterClientServiceHandler.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f1591f03/server/master/src/main/java/org/apache/accumulo/master/TabletGroupWatcher.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f1591f03/server/master/src/main/java/org/apache/accumulo/master/replication/WorkMaker.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f1591f03/server/master/src/test/java/org/apache/accumulo/master/ReplicationOperationsImplTest.java
----------------------------------------------------------------------
diff --cc server/master/src/test/java/org/apache/accumulo/master/ReplicationOperationsImplTest.java
index 0000000,a127dcd..1d8eeec
mode 000000,100644..100644
--- a/server/master/src/test/java/org/apache/accumulo/master/ReplicationOperationsImplTest.java
+++ b/server/master/src/test/java/org/apache/accumulo/master/ReplicationOperationsImplTest.java
@@@ -1,0 -1,459 +1,452 @@@
+ /*
+  * 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.master;
+ 
 -import java.util.Arrays;
+ import java.util.Map.Entry;
+ import java.util.Set;
+ import java.util.UUID;
+ import java.util.concurrent.atomic.AtomicBoolean;
+ 
+ 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.ClientConfiguration;
+ import org.apache.accumulo.core.client.Connector;
+ import org.apache.accumulo.core.client.Instance;
+ import org.apache.accumulo.core.client.TableNotFoundException;
+ import org.apache.accumulo.core.client.impl.ClientContext;
+ import org.apache.accumulo.core.client.impl.ReplicationOperationsImpl;
+ import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException;
+ import org.apache.accumulo.core.client.mock.MockInstance;
+ import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+ 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.Value;
+ import org.apache.accumulo.core.metadata.MetadataTable;
+ import org.apache.accumulo.core.metadata.schema.MetadataSchema.ReplicationSection;
+ import org.apache.accumulo.core.protobuf.ProtobufUtil;
+ import org.apache.accumulo.core.replication.ReplicationSchema.StatusSection;
+ import org.apache.accumulo.core.replication.ReplicationTable;
+ import org.apache.accumulo.core.security.Authorizations;
+ import org.apache.accumulo.core.security.Credentials;
+ import org.apache.accumulo.core.security.thrift.TCredentials;
+ import org.apache.accumulo.core.tabletserver.log.LogEntry;
+ import org.apache.accumulo.core.trace.thrift.TInfo;
+ import org.apache.accumulo.server.replication.proto.Replication.Status;
+ import org.apache.hadoop.io.Text;
+ import org.apache.thrift.TException;
+ import org.easymock.EasyMock;
+ import org.junit.Assert;
+ import org.junit.Before;
+ import org.junit.Rule;
+ import org.junit.Test;
+ import org.junit.rules.TestName;
+ import org.slf4j.Logger;
+ import org.slf4j.LoggerFactory;
+ 
+ public class ReplicationOperationsImplTest {
+   private static final Logger log = LoggerFactory.getLogger(ReplicationOperationsImplTest.class);
+ 
+   private MockInstance inst;
+ 
+   @Rule
+   public TestName test = new TestName();
+ 
+   @Before
+   public void setup() {
+     inst = new MockInstance(test.getMethodName());
+   }
+ 
+   /**
+    * Spoof out the Master so we can call the implementation without starting a full instance.
+    */
+   private ReplicationOperationsImpl getReplicationOperations(ClientContext context) throws Exception {
+     Master master = EasyMock.createMock(Master.class);
+     EasyMock.expect(master.getConnector()).andReturn(inst.getConnector("root", new PasswordToken(""))).anyTimes();
+     EasyMock.expect(master.getInstance()).andReturn(inst).anyTimes();
+     EasyMock.replay(master);
+ 
+     final MasterClientServiceHandler mcsh = new MasterClientServiceHandler(master) {
+       @Override
+       protected String getTableId(Instance inst, String tableName) throws ThriftTableOperationException {
+         try {
+           return inst.getConnector("root", new PasswordToken("")).tableOperations().tableIdMap().get(tableName);
+         } catch (Exception e) {
+           throw new RuntimeException(e);
+         }
+       }
+     };
+ 
+     return new ReplicationOperationsImpl(context) {
+       @Override
+       protected boolean getMasterDrain(final TInfo tinfo, final TCredentials rpcCreds, final String tableName, final Set<String> wals)
+           throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
+         try {
+           return mcsh.drainReplicationTable(tinfo, rpcCreds, tableName, wals);
+         } catch (TException e) {
+           throw new RuntimeException(e);
+         }
+       }
+     };
+   }
+ 
+   @Test
+   public void waitsUntilEntriesAreReplicated() throws Exception {
+     Connector conn = inst.getConnector("root", new PasswordToken(""));
+     conn.tableOperations().create("foo");
+     Text tableId = new Text(conn.tableOperations().tableIdMap().get("foo"));
+ 
+     String file1 = "/accumulo/wals/tserver+port/" + UUID.randomUUID(), file2 = "/accumulo/wals/tserver+port/" + UUID.randomUUID();
+     Status stat = Status.newBuilder().setBegin(0).setEnd(10000).setInfiniteEnd(false).setClosed(false).build();
+ 
+     BatchWriter bw = ReplicationTable.getBatchWriter(conn);
+ 
+     Mutation m = new Mutation(file1);
+     StatusSection.add(m, tableId, ProtobufUtil.toValue(stat));
+     bw.addMutation(m);
+ 
+     m = new Mutation(file2);
+     StatusSection.add(m, tableId, ProtobufUtil.toValue(stat));
+     bw.addMutation(m);
+ 
+     bw.close();
+ 
+     bw = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
+     m = new Mutation(ReplicationSection.getRowPrefix() + file1);
+     m.put(ReplicationSection.COLF, tableId, ProtobufUtil.toValue(stat));
+ 
+     bw.addMutation(m);
+ 
+     m = new Mutation(ReplicationSection.getRowPrefix() + file2);
+     m.put(ReplicationSection.COLF, tableId, ProtobufUtil.toValue(stat));
+ 
+     bw.close();
+ 
+     final AtomicBoolean done = new AtomicBoolean(false);
+     final AtomicBoolean exception = new AtomicBoolean(false);
+     ClientContext context = new ClientContext(inst, new Credentials("root", new PasswordToken("")), new ClientConfiguration());
+     final ReplicationOperationsImpl roi = getReplicationOperations(context);
+     Thread t = new Thread(new Runnable() {
+       @Override
+       public void run() {
+         try {
+           roi.drain("foo");
+         } catch (Exception e) {
+           log.error("Got error", e);
+           exception.set(true);
+         }
+         done.set(true);
+       }
+     });
+ 
+     t.start();
+ 
+     // With the records, we shouldn't be drained
+     Assert.assertFalse(done.get());
+ 
+     bw = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
+     m = new Mutation(ReplicationSection.getRowPrefix() + file1);
+     m.putDelete(ReplicationSection.COLF, tableId);
+     bw.addMutation(m);
+     bw.flush();
+ 
+     Assert.assertFalse(done.get());
+ 
+     m = new Mutation(ReplicationSection.getRowPrefix() + file2);
+     m.putDelete(ReplicationSection.COLF, tableId);
+     bw.addMutation(m);
+     bw.flush();
+     bw.close();
+ 
+     // Removing metadata entries doesn't change anything
+     Assert.assertFalse(done.get());
+ 
+     // Remove the replication entries too
+     bw = ReplicationTable.getBatchWriter(conn);
+     m = new Mutation(file1);
+     m.putDelete(StatusSection.NAME, tableId);
+     bw.addMutation(m);
+     bw.flush();
+ 
+     Assert.assertFalse(done.get());
+ 
+     m = new Mutation(file2);
+     m.putDelete(StatusSection.NAME, tableId);
+     bw.addMutation(m);
+     bw.flush();
+ 
+     try {
+       t.join(5000);
+     } catch (InterruptedException e) {
+       Assert.fail("ReplicationOperations.drain did not complete");
+     }
+ 
+     // After both metadata and replication
+     Assert.assertTrue("Drain never finished", done.get());
+     Assert.assertFalse("Saw unexpectetd exception", exception.get());
+   }
+ 
+   @Test
+   public void unrelatedReplicationRecordsDontBlockDrain() throws Exception {
+     Connector conn = inst.getConnector("root", new PasswordToken(""));
+     conn.tableOperations().create("foo");
+     conn.tableOperations().create("bar");
+ 
+     Text tableId1 = new Text(conn.tableOperations().tableIdMap().get("foo"));
+     Text tableId2 = new Text(conn.tableOperations().tableIdMap().get("bar"));
+ 
+     String file1 = "/accumulo/wals/tserver+port/" + UUID.randomUUID(), file2 = "/accumulo/wals/tserver+port/" + UUID.randomUUID();
+     Status stat = Status.newBuilder().setBegin(0).setEnd(10000).setInfiniteEnd(false).setClosed(false).build();
+ 
+     BatchWriter bw = ReplicationTable.getBatchWriter(conn);
+ 
+     Mutation m = new Mutation(file1);
+     StatusSection.add(m, tableId1, ProtobufUtil.toValue(stat));
+     bw.addMutation(m);
+ 
+     m = new Mutation(file2);
+     StatusSection.add(m, tableId2, ProtobufUtil.toValue(stat));
+     bw.addMutation(m);
+ 
+     bw.close();
+ 
+     bw = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
+     m = new Mutation(ReplicationSection.getRowPrefix() + file1);
+     m.put(ReplicationSection.COLF, tableId1, ProtobufUtil.toValue(stat));
+ 
+     bw.addMutation(m);
+ 
+     m = new Mutation(ReplicationSection.getRowPrefix() + file2);
+     m.put(ReplicationSection.COLF, tableId2, ProtobufUtil.toValue(stat));
+ 
+     bw.close();
+ 
+     final AtomicBoolean done = new AtomicBoolean(false);
+     final AtomicBoolean exception = new AtomicBoolean(false);
+     ClientContext context = new ClientContext(inst, new Credentials("root", new PasswordToken("")), new ClientConfiguration());
+ 
+     final ReplicationOperationsImpl roi = getReplicationOperations(context);
+ 
+     Thread t = new Thread(new Runnable() {
+       @Override
+       public void run() {
+         try {
+           roi.drain("foo");
+         } catch (Exception e) {
+           log.error("Got error", e);
+           exception.set(true);
+         }
+         done.set(true);
+       }
+     });
+ 
+     t.start();
+ 
+     // With the records, we shouldn't be drained
+     Assert.assertFalse(done.get());
+ 
+     bw = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
+     m = new Mutation(ReplicationSection.getRowPrefix() + file1);
+     m.putDelete(ReplicationSection.COLF, tableId1);
+     bw.addMutation(m);
+     bw.flush();
+ 
+     // Removing metadata entries doesn't change anything
+     Assert.assertFalse(done.get());
+ 
+     // Remove the replication entries too
+     bw = ReplicationTable.getBatchWriter(conn);
+     m = new Mutation(file1);
+     m.putDelete(StatusSection.NAME, tableId1);
+     bw.addMutation(m);
+     bw.flush();
+ 
+     try {
+       t.join(5000);
+     } catch (InterruptedException e) {
+       Assert.fail("ReplicationOperations.drain did not complete");
+     }
+ 
+     // After both metadata and replication
+     Assert.assertTrue("Drain never completed", done.get());
+     Assert.assertFalse("Saw unexpected exception", exception.get());
+   }
+ 
+   @Test
+   public void inprogressReplicationRecordsBlockExecution() throws Exception {
+     Connector conn = inst.getConnector("root", new PasswordToken(""));
+     conn.tableOperations().create("foo");
+ 
+     Text tableId1 = new Text(conn.tableOperations().tableIdMap().get("foo"));
+ 
+     String file1 = "/accumulo/wals/tserver+port/" + UUID.randomUUID();
+     Status stat = Status.newBuilder().setBegin(0).setEnd(10000).setInfiniteEnd(false).setClosed(false).build();
+ 
+     BatchWriter bw = ReplicationTable.getBatchWriter(conn);
+ 
+     Mutation m = new Mutation(file1);
+     StatusSection.add(m, tableId1, ProtobufUtil.toValue(stat));
+     bw.addMutation(m);
+     bw.close();
+ 
 -    LogEntry logEntry = new LogEntry();
 -    logEntry.extent = new KeyExtent(new Text(tableId1), null, null);
 -    logEntry.server = "tserver";
 -    logEntry.filename = file1;
 -    logEntry.tabletId = 1;
 -    logEntry.logSet = Arrays.asList(file1);
 -    logEntry.timestamp = System.currentTimeMillis();
++    LogEntry logEntry = new LogEntry(new KeyExtent(new Text(tableId1), null, null), System.currentTimeMillis(), "tserver", file1);
+ 
+     bw = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
+     m = new Mutation(ReplicationSection.getRowPrefix() + file1);
+     m.put(ReplicationSection.COLF, tableId1, ProtobufUtil.toValue(stat));
+     bw.addMutation(m);
+ 
+     m = new Mutation(logEntry.getRow());
+     m.put(logEntry.getColumnFamily(), logEntry.getColumnQualifier(), logEntry.getValue());
+     bw.addMutation(m);
+ 
+     bw.close();
+ 
+     final AtomicBoolean done = new AtomicBoolean(false);
+     final AtomicBoolean exception = new AtomicBoolean(false);
+     ClientContext context = new ClientContext(inst, new Credentials("root", new PasswordToken("")), new ClientConfiguration());
+     final ReplicationOperationsImpl roi = getReplicationOperations(context);
+     Thread t = new Thread(new Runnable() {
+       @Override
+       public void run() {
+         try {
+           roi.drain("foo");
+         } catch (Exception e) {
+           log.error("Got error", e);
+           exception.set(true);
+         }
+         done.set(true);
+       }
+     });
+ 
+     t.start();
+ 
+     // With the records, we shouldn't be drained
+     Assert.assertFalse(done.get());
+ 
+     Status newStatus = Status.newBuilder().setBegin(1000).setEnd(2000).setInfiniteEnd(false).setClosed(true).build();
+     bw = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
+     m = new Mutation(ReplicationSection.getRowPrefix() + file1);
+     m.put(ReplicationSection.COLF, tableId1, ProtobufUtil.toValue(newStatus));
+     bw.addMutation(m);
+     bw.flush();
+ 
+     // Removing metadata entries doesn't change anything
+     Assert.assertFalse(done.get());
+ 
+     // Remove the replication entries too
+     bw = ReplicationTable.getBatchWriter(conn);
+     m = new Mutation(file1);
+     m.put(StatusSection.NAME, tableId1, ProtobufUtil.toValue(newStatus));
+     bw.addMutation(m);
+     bw.flush();
+ 
+     try {
+       t.join(5000);
+     } catch (InterruptedException e) {
+       Assert.fail("ReplicationOperations.drain did not complete");
+     }
+ 
+     // New records, but not fully replicated ones don't cause it to complete
+     Assert.assertFalse("Drain somehow finished", done.get());
+     Assert.assertFalse("Saw unexpected exception", exception.get());
+   }
+ 
+   @Test
+   public void laterCreatedLogsDontBlockExecution() throws Exception {
+     Connector conn = inst.getConnector("root", new PasswordToken(""));
+     conn.tableOperations().create("foo");
+ 
+     Text tableId1 = new Text(conn.tableOperations().tableIdMap().get("foo"));
+ 
+     String file1 = "/accumulo/wals/tserver+port/" + UUID.randomUUID();
+     Status stat = Status.newBuilder().setBegin(0).setEnd(10000).setInfiniteEnd(false).setClosed(false).build();
+ 
+     BatchWriter bw = ReplicationTable.getBatchWriter(conn);
+     Mutation m = new Mutation(file1);
+     StatusSection.add(m, tableId1, ProtobufUtil.toValue(stat));
+     bw.addMutation(m);
+     bw.close();
+ 
+     bw = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
+     m = new Mutation(ReplicationSection.getRowPrefix() + file1);
+     m.put(ReplicationSection.COLF, tableId1, ProtobufUtil.toValue(stat));
+     bw.addMutation(m);
+ 
+     bw.close();
+ 
+     System.out.println("Reading metadata first time");
+     for (Entry<Key,Value> e : conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
+       System.out.println(e.getKey());
+     }
+ 
+     final AtomicBoolean done = new AtomicBoolean(false);
+     final AtomicBoolean exception = new AtomicBoolean(false);
+     ClientContext context = new ClientContext(inst, new Credentials("root", new PasswordToken("")), new ClientConfiguration());
+     final ReplicationOperationsImpl roi = getReplicationOperations(context);
+     Thread t = new Thread(new Runnable() {
+       @Override
+       public void run() {
+         try {
+           roi.drain("foo");
+         } catch (Exception e) {
+           log.error("Got error", e);
+           exception.set(true);
+         }
+         done.set(true);
+       }
+     });
+ 
+     t.start();
+ 
+     // We need to wait long enough for the table to read once
+     Thread.sleep(2000);
+ 
+     // Write another file, but also delete the old files
+     bw = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
+     m = new Mutation(ReplicationSection.getRowPrefix() + "/accumulo/wals/tserver+port/" + UUID.randomUUID());
+     m.put(ReplicationSection.COLF, tableId1, ProtobufUtil.toValue(stat));
+     bw.addMutation(m);
+     m = new Mutation(ReplicationSection.getRowPrefix() + file1);
+     m.putDelete(ReplicationSection.COLF, tableId1);
+     bw.addMutation(m);
+     bw.close();
+ 
+     System.out.println("Reading metadata second time");
+     for (Entry<Key,Value> e : conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
+       System.out.println(e.getKey());
+     }
+ 
+     bw = ReplicationTable.getBatchWriter(conn);
+     m = new Mutation(file1);
+     m.putDelete(StatusSection.NAME, tableId1);
+     bw.addMutation(m);
+     bw.close();
+ 
+     try {
+       t.join(5000);
+     } catch (InterruptedException e) {
+       Assert.fail("ReplicationOperations.drain did not complete");
+     }
+ 
+     // We should pass immediately because we aren't waiting on both files to be deleted (just the one that we did)
+     Assert.assertTrue("Drain didn't finish", done.get());
+   }
+ 
+ }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f1591f03/server/tserver/src/main/findbugs/exclude-filter.xml
----------------------------------------------------------------------
diff --cc server/tserver/src/main/findbugs/exclude-filter.xml
index 45f6a78,47dd1f5..a334163
--- a/server/tserver/src/main/findbugs/exclude-filter.xml
+++ b/server/tserver/src/main/findbugs/exclude-filter.xml
@@@ -18,7 -18,13 +18,13 @@@
    <Match>
      <!-- locking is confusing, but probably correct -->
      <Class name="org.apache.accumulo.tserver.tablet.Tablet" />
 -    <Method name="beginUpdatingLogsUsed" params="org.apache.accumulo.tserver.InMemoryMap,java.util.Collection,boolean" returns="boolean" />
 +    <Method name="beginUpdatingLogsUsed" params="org.apache.accumulo.tserver.InMemoryMap,org.apache.accumulo.tserver.log.DfsLogger,boolean" returns="boolean" />
      <Bug code="UL" pattern="UL_UNRELEASED_LOCK" />
    </Match>
+   <Match>
+     <!-- false positive about forced garbage collection in resource manager -->
+     <Class name="org.apache.accumulo.tserver.TabletServerResourceManager" />
+     <Method name="&lt;init&gt;" params="org.apache.accumulo.tserver.TabletServer,org.apache.accumulo.server.fs.VolumeManager" returns="void" />
+     <Bug code="DM" pattern="DM_GC" />
+   </Match>
  </FindBugsFilter>

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f1591f03/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
----------------------------------------------------------------------
diff --cc server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
index 498cbdd,254e5d6..64d2052
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
@@@ -38,9 -37,6 +38,7 @@@ import org.apache.accumulo.core.data.Ke
  import org.apache.accumulo.core.data.Mutation;
  import org.apache.accumulo.core.protobuf.ProtobufUtil;
  import org.apache.accumulo.core.replication.ReplicationConfigurationUtil;
- import org.apache.accumulo.core.replication.StatusUtil;
- import org.apache.accumulo.core.replication.proto.Replication.Status;
 +import org.apache.accumulo.core.util.SimpleThreadPool;
  import org.apache.accumulo.core.util.UtilWaitThread;
  import org.apache.accumulo.server.conf.TableConfiguration;
  import org.apache.accumulo.server.fs.VolumeManager;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f1591f03/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/DatafileManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f1591f03/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f1591f03/test/src/test/java/org/apache/accumulo/test/ConditionalWriterIT.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f1591f03/test/src/test/java/org/apache/accumulo/test/MissingWalHeaderCompletesRecoveryIT.java
----------------------------------------------------------------------
diff --cc test/src/test/java/org/apache/accumulo/test/MissingWalHeaderCompletesRecoveryIT.java
index 685d71a,b78a311..27f1f69
--- a/test/src/test/java/org/apache/accumulo/test/MissingWalHeaderCompletesRecoveryIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/MissingWalHeaderCompletesRecoveryIT.java
@@@ -16,7 -16,10 +16,9 @@@
   */
  package org.apache.accumulo.test;
  
+ import static java.nio.charset.StandardCharsets.UTF_8;
+ 
  import java.io.File;
 -import java.util.Collections;
  import java.util.UUID;
  
  import org.apache.accumulo.core.client.BatchWriter;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f1591f03/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f1591f03/test/src/test/java/org/apache/accumulo/test/replication/GarbageCollectorCommunicatesWithTServersIT.java
----------------------------------------------------------------------
diff --cc test/src/test/java/org/apache/accumulo/test/replication/GarbageCollectorCommunicatesWithTServersIT.java
index 9af5445,5b89d9c..fbe6900
--- a/test/src/test/java/org/apache/accumulo/test/replication/GarbageCollectorCommunicatesWithTServersIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/replication/GarbageCollectorCommunicatesWithTServersIT.java
@@@ -38,10 -38,8 +38,9 @@@ import org.apache.accumulo.core.data.Va
  import org.apache.accumulo.core.master.thrift.MasterClientService;
  import org.apache.accumulo.core.metadata.MetadataTable;
  import org.apache.accumulo.core.metadata.schema.MetadataSchema;
 +import org.apache.accumulo.core.metadata.schema.MetadataSchema.CurrentLogsSection;
  import org.apache.accumulo.core.protobuf.ProtobufUtil;
  import org.apache.accumulo.core.replication.ReplicationTable;
- import org.apache.accumulo.core.replication.proto.Replication.Status;
  import org.apache.accumulo.core.rpc.ThriftUtil;
  import org.apache.accumulo.core.security.Authorizations;
  import org.apache.accumulo.core.security.Credentials;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f1591f03/test/src/test/java/org/apache/accumulo/test/replication/MultiInstanceReplicationIT.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f1591f03/test/src/test/java/org/apache/accumulo/test/replication/ReplicationIT.java
----------------------------------------------------------------------
diff --cc test/src/test/java/org/apache/accumulo/test/replication/ReplicationIT.java
index 718cda1,54348db..ca58a59
--- a/test/src/test/java/org/apache/accumulo/test/replication/ReplicationIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/replication/ReplicationIT.java
@@@ -44,10 -43,8 +44,9 @@@ import org.apache.accumulo.core.client.
  import org.apache.accumulo.core.client.TableNotFoundException;
  import org.apache.accumulo.core.client.ZooKeeperInstance;
  import org.apache.accumulo.core.client.admin.TableOperations;
- import org.apache.accumulo.core.client.replication.ReplicaSystemFactory;
  import org.apache.accumulo.core.conf.Property;
  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.Range;
  import org.apache.accumulo.core.data.Value;
@@@ -78,10 -71,15 +74,14 @@@ import org.apache.accumulo.fate.zookeep
  import org.apache.accumulo.gc.SimpleGarbageCollector;
  import org.apache.accumulo.minicluster.ServerType;
  import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 -import org.apache.accumulo.minicluster.impl.ProcessReference;
 +import org.apache.accumulo.server.master.state.TServerInstance;
+ import org.apache.accumulo.server.replication.ReplicaSystemFactory;
  import org.apache.accumulo.server.replication.StatusCombiner;
+ import org.apache.accumulo.server.replication.StatusFormatter;
+ import org.apache.accumulo.server.replication.StatusUtil;
+ import org.apache.accumulo.server.replication.proto.Replication.Status;
  import org.apache.accumulo.server.util.ReplicationTableUtil;
  import org.apache.accumulo.test.functional.ConfigurableMacIT;
 -import org.apache.accumulo.tserver.TabletServer;
  import org.apache.hadoop.conf.Configuration;
  import org.apache.hadoop.fs.FileSystem;
  import org.apache.hadoop.fs.Path;


[12/34] accumulo git commit: ACCUMULO-3423 merging master

Posted by ec...@apache.org.
ACCUMULO-3423 merging master


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

Branch: refs/heads/master
Commit: ac176e193769b37dd3d21894d1a084426d9e7e1d
Parents: f1591f0 bddb007
Author: Eric C. Newton <er...@gmail.com>
Authored: Wed Mar 18 15:41:32 2015 -0400
Committer: Eric C. Newton <er...@gmail.com>
Committed: Wed Mar 18 15:41:32 2015 -0400

----------------------------------------------------------------------
 .../accumulo/core/bloomfilter/BloomFilter.java  |   5 +-
 .../accumulo/core/client/impl/MasterClient.java |   5 +-
 .../client/impl/MultiTableBatchWriterImpl.java  |   5 +-
 .../core/client/impl/ScannerIterator.java       |  28 +-
 .../accumulo/core/client/impl/ServerClient.java |   5 +-
 .../accumulo/core/client/impl/Tables.java       |   5 +-
 .../client/impl/TabletServerBatchReader.java    |   6 +-
 .../client/impl/TabletServerBatchWriter.java    |   5 +-
 .../accumulo/core/client/impl/Writer.java       |   5 +-
 .../client/mock/MockInstanceOperations.java     |   5 +-
 .../client/mock/MockNamespaceOperations.java    |   5 +-
 .../core/client/mock/MockTableOperations.java   |   5 +-
 .../core/conf/AccumuloConfiguration.java        |   6 +-
 .../accumulo/core/conf/ConfigSanityCheck.java   |   8 +-
 .../accumulo/core/conf/ConfigurationDocGen.java |   7 +-
 .../org/apache/accumulo/core/conf/Property.java |  12 +-
 .../accumulo/core/conf/SiteConfiguration.java   |   6 +-
 .../file/blockfile/impl/CachableBlockFile.java  |   6 +-
 .../accumulo/core/file/rfile/CreateEmpty.java   |   5 +-
 .../accumulo/core/file/rfile/PrintInfo.java     |   5 +-
 .../apache/accumulo/core/file/rfile/RFile.java  |   6 +-
 .../core/iterators/AggregatingIterator.java     |   6 +-
 .../accumulo/core/iterators/DebugIterator.java  |   5 +-
 .../accumulo/core/iterators/IteratorUtil.java   |   5 +-
 .../iterators/aggregation/LongSummation.java    |   6 +-
 .../core/iterators/system/MapFileIterator.java  |   6 +-
 .../core/iterators/system/VisibilityFilter.java |   6 +-
 .../core/iterators/user/RegExFilter.java        |   6 +-
 .../iterators/user/TransformingIterator.java    |   6 +-
 .../core/replication/ReplicationTable.java      |   5 +-
 .../accumulo/core/rpc/SslConnectionParams.java  |   5 +-
 .../CachingHDFSSecretKeyEncryptionStrategy.java |  19 +-
 .../security/crypto/CryptoModuleFactory.java    |  13 +-
 .../security/crypto/DefaultCryptoModule.java    |   5 +-
 .../crypto/DefaultCryptoModuleUtils.java        |   5 +-
 .../crypto/DiscardCloseOutputStream.java        |   5 +-
 .../NonCachingSecretKeyEncryptionStrategy.java  |  19 +-
 .../accumulo/core/trace/DistributedTrace.java   |   5 +-
 .../org/apache/accumulo/core/util/CleanUp.java  |   5 +-
 .../org/apache/accumulo/core/util/Merge.java    |   5 +-
 .../accumulo/core/util/UtilWaitThread.java      |   6 +-
 .../core/util/format/FormatterFactory.java      |   6 +-
 .../apache/accumulo/core/volume/VolumeImpl.java |   6 +-
 .../apache/accumulo/core/zookeeper/ZooUtil.java |   6 +-
 .../core/file/BloomFilterLayerLookupTest.java   |   5 +-
 .../iterators/aggregation/NumSummationTest.java |   8 +-
 .../iterators/user/VersioningIteratorTest.java  |  13 +-
 .../main/asciidoc/accumulo_user_manual.asciidoc |   6 +-
 .../main/asciidoc/chapters/iterator_design.txt  | 386 +++++++++++++++++++
 docs/src/main/resources/isolation.html          |   2 +-
 .../simple/client/RandomBatchScanner.java       |   7 +-
 .../examples/simple/client/RowOperations.java   |   6 +-
 .../examples/simple/client/TracingExample.java  |   5 +-
 .../examples/simple/dirlist/Viewer.java         |   5 +-
 .../simple/filedata/ChunkInputStream.java       |   6 +-
 .../examples/simple/helloworld/ReadData.java    |   5 +-
 .../simple/isolation/InterferenceTest.java      |   7 +-
 .../examples/simple/mapreduce/NGramIngest.java  |   5 +-
 .../simple/mapreduce/TokenFileWordCount.java    |   6 +-
 .../examples/simple/mapreduce/WordCount.java    |   5 +-
 .../simple/mapreduce/bulk/VerifyIngest.java     |   5 +-
 .../examples/simple/reservations/ARS.java       |   6 +-
 .../examples/simple/dirlist/CountTest.java      |   6 +-
 .../simple/filedata/ChunkInputStreamTest.java   |  15 +-
 .../org/apache/accumulo/fate/AdminUtil.java     |   7 +-
 .../org/apache/accumulo/fate/AgeOffStore.java   |   6 +-
 .../java/org/apache/accumulo/fate/Fate.java     |   6 +-
 .../apache/accumulo/fate/util/AddressUtil.java  |   6 +-
 .../accumulo/fate/util/LoggingRunnable.java     |   3 +-
 .../accumulo/fate/util/UtilWaitThread.java      |   6 +-
 .../zookeeper/DistributedReadWriteLock.java     |   6 +-
 .../apache/accumulo/fate/zookeeper/Retry.java   |   6 +-
 .../fate/zookeeper/TransactionWatcher.java      |   6 +-
 .../accumulo/fate/zookeeper/ZooCache.java       |   7 +-
 .../apache/accumulo/fate/zookeeper/ZooLock.java |   5 +-
 .../accumulo/fate/zookeeper/ZooReader.java      |   5 +-
 .../fate/zookeeper/ZooReaderWriter.java         |   5 +-
 .../accumulo/fate/zookeeper/ZooReservation.java |   4 +-
 .../accumulo/fate/zookeeper/ZooSession.java     |   5 +-
 .../apache/accumulo/fate/zookeeper/ZooUtil.java |   5 +-
 .../accumulo/fate/util/AddressUtilTest.java     |   6 +-
 .../minicluster/MiniAccumuloRunner.java         |   5 +-
 .../impl/MiniAccumuloClusterControl.java        |   5 +-
 .../impl/MiniAccumuloConfigImpl.java            |   5 +-
 .../MiniAccumuloClusterStartStopTest.java       |   5 +-
 .../java/org/apache/accumulo/proxy/Proxy.java   |  14 +-
 .../org/apache/accumulo/proxy/ProxyServer.java  |   7 +-
 .../server/client/ClientServiceHandler.java     |   9 +-
 .../server/conf/NamespaceConfiguration.java     |   5 +-
 .../server/conf/TableConfiguration.java         |   5 +-
 .../server/conf/ZooCachePropertyAccessor.java   |   5 +-
 .../accumulo/server/conf/ZooConfiguration.java  |   5 +-
 .../server/constraints/MetadataConstraints.java |   5 +-
 .../server/fs/PreferredVolumeChooser.java       |   5 +-
 .../accumulo/server/fs/VolumeManagerImpl.java   |  10 +-
 .../apache/accumulo/server/fs/VolumeUtil.java   |   5 +-
 .../apache/accumulo/server/init/Initialize.java |  55 +--
 .../iterators/MetadataBulkLoadFilter.java       |   7 +-
 .../accumulo/server/master/LiveTServerSet.java  |  12 +-
 .../master/balancer/ChaoticLoadBalancer.java    |   5 +-
 .../master/balancer/DefaultLoadBalancer.java    |   5 +-
 .../master/balancer/TableLoadBalancer.java      |   5 +-
 .../server/master/balancer/TabletBalancer.java  |   5 +-
 .../server/master/recovery/HadoopLogCloser.java |   5 +-
 .../server/master/recovery/MapRLogCloser.java   |   5 +-
 .../server/master/state/DeadServerList.java     |  11 +-
 .../master/state/MetaDataTableScanner.java      |   5 +-
 .../master/state/TabletStateChangeIterator.java |   7 +-
 .../accumulo/server/master/state/ZooStore.java  |   5 +-
 .../master/state/ZooTabletStateStore.java       |   5 +-
 .../server/metrics/AbstractMetricsImpl.java     |   2 +-
 .../server/metrics/MetricsConfiguration.java    |   2 +-
 .../accumulo/server/metrics/ThriftMetrics.java  |   2 +-
 .../accumulo/server/monitor/LogService.java     |   7 +-
 .../server/replication/StatusCombiner.java      |   7 +-
 .../server/rpc/CustomNonBlockingServer.java     |   5 +-
 .../server/rpc/TNonblockingServerSocket.java    |   5 +-
 .../server/security/SecurityOperation.java      |   7 +-
 .../accumulo/server/security/SecurityUtil.java  |   5 +-
 .../security/handler/ZKAuthenticator.java       |  28 +-
 .../server/security/handler/ZKAuthorizor.java   |  21 +-
 .../server/security/handler/ZKPermHandler.java  |  49 +--
 .../server/security/handler/ZKSecurityTool.java |  11 +-
 .../accumulo/server/tables/TableManager.java    |  14 +-
 .../tabletserver/LargestFirstMemoryManager.java |   5 +-
 .../org/apache/accumulo/server/util/Admin.java  |  11 +-
 .../accumulo/server/util/CleanZookeeper.java    |   8 +-
 .../apache/accumulo/server/util/FileUtil.java   |  19 +-
 .../server/util/FindOfflineTablets.java         |   5 +-
 .../org/apache/accumulo/server/util/Halt.java   |  10 +-
 .../accumulo/server/util/ListInstances.java     |   7 +-
 .../server/util/MasterMetadataUtil.java         |  13 +-
 .../accumulo/server/util/MetadataTableUtil.java |  21 +-
 .../accumulo/server/util/RandomWriter.java      |   9 +-
 .../accumulo/server/util/RandomizeVolumes.java  |   7 +-
 .../accumulo/server/util/TableDiskUsage.java    |   5 +-
 .../accumulo/server/util/TabletIterator.java    |   7 +-
 .../server/util/VerifyTabletAssignments.java    |   5 +-
 .../org/apache/accumulo/server/util/ZooZap.java |   7 +-
 .../accumulo/server/util/time/SimpleTimer.java  |   5 +-
 .../server/zookeeper/DistributedWorkQueue.java  |   5 +-
 .../security/handler/ZKAuthenticatorTest.java   |   7 +-
 .../accumulo/gc/GarbageCollectionAlgorithm.java |   5 +-
 .../accumulo/gc/SimpleGarbageCollector.java     |  27 +-
 .../accumulo/master/EventCoordinator.java       |   5 +-
 .../java/org/apache/accumulo/master/Master.java |   9 +-
 .../master/metrics/ReplicationMetrics.java      |   5 +-
 .../master/recovery/RecoveryManager.java        |   7 +-
 .../accumulo/master/state/MergeStats.java       |   7 +-
 .../accumulo/master/tableOps/BulkImport.java    |   9 +-
 .../master/tableOps/ChangeTableState.java       |   4 +-
 .../accumulo/master/tableOps/CloneTable.java    |   8 +-
 .../accumulo/master/tableOps/CompactRange.java  |   4 +-
 .../master/tableOps/CreateNamespace.java        |   6 +-
 .../accumulo/master/tableOps/CreateTable.java   |   6 +-
 .../master/tableOps/DeleteNamespace.java        |   7 +-
 .../accumulo/master/tableOps/DeleteTable.java   |   7 +-
 .../accumulo/master/tableOps/ImportTable.java   |   9 +-
 .../accumulo/master/tableOps/MasterRepo.java    |   5 +-
 .../master/tableOps/RenameNamespace.java        |   4 +-
 .../accumulo/master/tableOps/RenameTable.java   |   4 +-
 .../apache/accumulo/master/tableOps/Utils.java  |   7 +-
 .../master/tserverOps/ShutdownTServer.java      |   5 +-
 .../accumulo/monitor/ZooKeeperStatus.java       |   7 +-
 .../accumulo/tracer/AsyncSpanReceiver.java      |   5 +-
 .../accumulo/tracer/SendSpansViaThrift.java     |   4 +-
 .../org/apache/accumulo/tracer/TraceServer.java |   7 +-
 .../apache/accumulo/tracer/ZooTraceClient.java  |   5 +-
 .../server/GarbageCollectionLogger.java         |   5 +-
 .../tserver/BulkFailedCopyProcessor.java        |   5 +-
 .../apache/accumulo/tserver/FileManager.java    |   7 +-
 .../apache/accumulo/tserver/InMemoryMap.java    |   7 +-
 .../org/apache/accumulo/tserver/NativeMap.java  |   5 +-
 .../apache/accumulo/tserver/WriteTracker.java   |   7 +-
 .../tserver/constraints/ConstraintChecker.java  |   5 +-
 .../accumulo/tserver/log/LocalWALRecovery.java  |   5 +-
 .../apache/accumulo/tserver/log/LogSorter.java  |   7 +-
 .../accumulo/tserver/log/SortedLogRecovery.java |   5 +-
 .../tserver/log/TabletServerLogger.java         |   7 +-
 .../accumulo/tserver/logger/LogReader.java      |   5 +-
 .../tserver/metrics/TabletServerMBeanImpl.java  |   5 +-
 .../metrics/TabletServerMinCMetrics.java        |   2 +-
 .../metrics/TabletServerScanMetrics.java        |   2 +-
 .../metrics/TabletServerUpdateMetrics.java      |   2 +-
 .../accumulo/tserver/scan/LookupTask.java       |   5 +-
 .../accumulo/tserver/scan/NextBatchTask.java    |   5 +-
 .../tserver/session/SessionManager.java         |   5 +-
 .../accumulo/tserver/tablet/CommitSession.java  |   7 +-
 .../tserver/tablet/CompactionWatcher.java       |   6 +-
 .../accumulo/tserver/tablet/Compactor.java      |  15 +-
 .../tserver/tablet/MinorCompactionTask.java     |   5 +-
 .../accumulo/tserver/tablet/MinorCompactor.java |   5 +-
 .../accumulo/tserver/tablet/RootFiles.java      |   5 +-
 .../apache/accumulo/tserver/tablet/Scanner.java |   5 +-
 .../accumulo/tserver/tablet/TabletMemory.java   |   7 +-
 .../shell/commands/DeleteScanIterCommand.java   |   1 +
 .../accumulo/shell/commands/MaxRowCommand.java  |   5 +-
 .../ShellPluginConfigurationCommand.java        |  10 +-
 .../accumulo/shell/format/DeleterFormatter.java |   5 +-
 .../java/org/apache/accumulo/start/Main.java    |   5 +-
 .../start/classloader/AccumuloClassLoader.java  |   5 +-
 .../vfs/AccumuloReloadingVFSClassLoader.java    |   5 +-
 .../classloader/vfs/AccumuloVFSClassLoader.java |   9 +-
 .../classloader/vfs/UniqueFileReplicator.java   |   5 +-
 .../org/apache/accumulo/test/CreateRFiles.java  |   5 +-
 .../accumulo/test/NativeMapConcurrencyTest.java |   5 +-
 .../org/apache/accumulo/test/TestIngest.java    |   3 -
 .../apache/accumulo/test/TestRandomDeletes.java |   5 +-
 .../org/apache/accumulo/test/VerifyIngest.java  |   5 +-
 .../continuous/ContinuousStatsCollector.java    |   5 +-
 .../test/continuous/ContinuousVerify.java       |   5 +-
 .../test/continuous/PrintScanTimeHistogram.java |   9 +-
 .../accumulo/test/functional/ZombieTServer.java |   9 +-
 .../metadata/MetadataBatchScanTest.java         |   5 +-
 .../performance/scan/CollectTabletStats.java    |   5 +-
 .../accumulo/test/randomwalk/Environment.java   |   7 +-
 .../apache/accumulo/test/randomwalk/State.java  |   2 +-
 .../test/randomwalk/bulk/SelectiveQueueing.java |   5 +-
 .../test/randomwalk/concurrent/Replication.java | 201 ++++++++++
 .../randomwalk/security/SecurityHelper.java     |   5 +-
 .../randomwalk/security/WalkingSecurity.java    |   5 +-
 .../test/randomwalk/shard/SortTool.java         |   5 +-
 .../accumulo/test/scalability/Ingest.java       |   6 +-
 .../apache/accumulo/test/scalability/Run.java   |   5 +-
 .../accumulo/fate/zookeeper/ZooLockTest.java    |   4 -
 .../org/apache/accumulo/harness/AccumuloIT.java |  17 +-
 .../accumulo/harness/MiniClusterHarness.java    |   2 +-
 .../apache/accumulo/start/KeywordStartIT.java   |   5 +-
 .../test/ArbitraryTablePropertiesIT.java        |   5 +-
 .../test/BadDeleteMarkersCreatedIT.java         |   5 +-
 .../accumulo/test/BulkImportVolumeIT.java       |   2 -
 .../org/apache/accumulo/test/CleanWalIT.java    |   5 +-
 .../accumulo/test/ConditionalWriterIT.java      |   9 +-
 .../test/CreateTableWithNewTableConfigIT.java   |   5 +-
 .../apache/accumulo/test/LargeSplitRowIT.java   |   5 +-
 .../apache/accumulo/test/MetaGetsReadersIT.java |   2 +-
 .../accumulo/test/MultiTableRecoveryIT.java     |   2 +-
 .../test/RewriteTabletDirectoriesIT.java        |   2 -
 .../org/apache/accumulo/test/ShellServerIT.java |   5 +-
 .../test/TableConfigurationUpdateIT.java        |   5 +-
 .../apache/accumulo/test/VolumeChooserIT.java   |   3 -
 .../java/org/apache/accumulo/test/VolumeIT.java |   2 -
 .../test/functional/BatchScanSplitIT.java       |   5 +-
 .../accumulo/test/functional/BloomFilterIT.java |   5 +-
 .../accumulo/test/functional/CleanUpIT.java     |   5 +-
 .../test/functional/ConfigurableMacIT.java      |  15 +-
 .../accumulo/test/functional/DeleteRowsIT.java  |   5 +-
 .../test/functional/DeleteRowsSplitIT.java      |   7 +-
 .../accumulo/test/functional/ExamplesIT.java    |   5 +-
 .../test/functional/HalfDeadTServerIT.java      |   2 +-
 .../accumulo/test/functional/LargeRowIT.java    |   5 +-
 .../accumulo/test/functional/LogicalTimeIT.java |   5 +-
 .../test/functional/MonitorLoggingIT.java       |   5 +-
 .../accumulo/test/functional/MonitorSslIT.java  |   2 +-
 .../accumulo/test/functional/PermissionsIT.java |   5 +-
 .../functional/RecoveryWithEmptyRFileIT.java    |   5 +-
 .../apache/accumulo/test/functional/SslIT.java  |   2 +-
 .../test/replication/CyclicReplicationIT.java   |   2 +-
 .../replication/ReplicationRandomWalkIT.java    |  67 ++++
 .../apache/accumulo/test/util/CertUtils.java    |   5 +-
 .../randomwalk/conf/modules/Concurrent.xml      |   5 +
 261 files changed, 1639 insertions(+), 746 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/ac176e19/core/src/main/java/org/apache/accumulo/core/conf/Property.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ac176e19/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeUtil.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ac176e19/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ac176e19/server/base/src/main/java/org/apache/accumulo/server/master/state/MetaDataTableScanner.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ac176e19/server/base/src/main/java/org/apache/accumulo/server/master/state/ZooTabletStateStore.java
----------------------------------------------------------------------
diff --cc server/base/src/main/java/org/apache/accumulo/server/master/state/ZooTabletStateStore.java
index eca8e7f,ab99396..bce20fd
--- a/server/base/src/main/java/org/apache/accumulo/server/master/state/ZooTabletStateStore.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/state/ZooTabletStateStore.java
@@@ -21,18 -21,14 +21,19 @@@ import static java.nio.charset.Standard
  import java.io.IOException;
  import java.util.ArrayList;
  import java.util.Collection;
 +import java.util.Collections;
  import java.util.Iterator;
  import java.util.List;
 +import java.util.Map;
  
  import org.apache.accumulo.core.metadata.RootTable;
 +import org.apache.accumulo.core.metadata.schema.MetadataSchema;
  import org.apache.accumulo.core.tabletserver.log.LogEntry;
 +import org.apache.accumulo.server.AccumuloServerContext;
  import org.apache.commons.lang.NotImplementedException;
 +import org.apache.hadoop.fs.Path;
- import org.apache.log4j.Logger;
+ import org.slf4j.Logger;
+ import org.slf4j.LoggerFactory;
  
  import com.google.common.net.HostAndPort;
  

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ac176e19/server/base/src/main/java/org/apache/accumulo/server/util/MasterMetadataUtil.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ac176e19/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 96f9c9e,10cd749..db00b9c
--- 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
@@@ -252,39 -253,6 +253,35 @@@ public class MetadataTableUtil 
      }
    }
  
 +  private static interface ZooOperation {
 +    void run(IZooReaderWriter rw) throws KeeperException, InterruptedException, IOException;
 +  }
 +
 +  private static void retryZooKeeperUpdate(ClientContext context, ZooLock zooLock, ZooOperation op) {
 +    while (true) {
 +      try {
 +        IZooReaderWriter zoo = ZooReaderWriter.getInstance();
 +        if (zoo.isLockHeld(zooLock.getLockID())) {
 +          op.run(zoo);
 +        }
 +        break;
-       } catch (KeeperException e) {
-         log.error(e, e);
-       } catch (InterruptedException e) {
-         log.error(e, e);
-       } catch (IOException e) {
-         log.error(e, e);
++      } catch (Exception e) {
++        log.error("Unexpected exception {}", e.getMessage(), e);
 +      }
 +      UtilWaitThread.sleep(1000);
 +    }
 +  }
 +
 +  private static void addRootLogEntry(AccumuloServerContext context, ZooLock zooLock, final LogEntry entry) {
 +    retryZooKeeperUpdate(context, zooLock, new ZooOperation() {
 +      @Override
 +      public void run(IZooReaderWriter rw) throws KeeperException, InterruptedException, IOException {
 +        String root = getZookeeperLogLocation();
 +        rw.putPersistentData(root + "/" + entry.getUniqueID(), entry.toBytes(), NodeExistsPolicy.OVERWRITE);
 +      }
 +    });
 +  }
 +
    public static SortedMap<FileRef,DataFileValue> getDataFileSizes(KeyExtent extent, ClientContext context) throws IOException {
      TreeMap<FileRef,DataFileValue> sizes = new TreeMap<FileRef,DataFileValue>();
  

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ac176e19/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ac176e19/server/master/src/main/java/org/apache/accumulo/master/Master.java
----------------------------------------------------------------------
diff --cc server/master/src/main/java/org/apache/accumulo/master/Master.java
index f897162,3762f32..2be7238
--- a/server/master/src/main/java/org/apache/accumulo/master/Master.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/Master.java
@@@ -422,12 -422,10 +422,13 @@@ public class Master extends AccumuloSer
            perm.grantNamespacePermission(user, Namespaces.ACCUMULO_NAMESPACE_ID, NamespacePermission.READ);
          }
          perm.grantNamespacePermission("root", Namespaces.ACCUMULO_NAMESPACE_ID, NamespacePermission.ALTER_TABLE);
 +
 +        // add the currlog location for root tablet current logs
 +        zoo.putPersistentData(ZooUtil.getRoot(getInstance()) + RootTable.ZROOT_TABLET_CURRENT_LOGS, new byte[0], NodeExistsPolicy.SKIP);
          haveUpgradedZooKeeper = true;
        } catch (Exception ex) {
-         log.error("Error performing upgrade", ex);
+         // ACCUMULO-3651 Changed level to error and added FATAL to message for slf4j compatibility
+         log.error("FATAL: Error performing upgrade", ex);
          System.exit(1);
        }
      }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ac176e19/server/master/src/main/java/org/apache/accumulo/master/state/MergeStats.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ac176e19/server/tserver/src/main/java/org/apache/accumulo/server/GarbageCollectionLogger.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ac176e19/server/tserver/src/main/java/org/apache/accumulo/tserver/log/SortedLogRecovery.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ac176e19/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
----------------------------------------------------------------------
diff --cc server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
index 64d2052,711c497..accfc5e
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
@@@ -51,9 -49,9 +51,10 @@@ import org.apache.accumulo.tserver.Tabl
  import org.apache.accumulo.tserver.TabletServer;
  import org.apache.accumulo.tserver.log.DfsLogger.LoggerOperation;
  import org.apache.accumulo.tserver.tablet.CommitSession;
 +import org.apache.accumulo.tserver.tablet.Tablet;
  import org.apache.hadoop.fs.Path;
- import org.apache.log4j.Logger;
+ import org.slf4j.Logger;
+ import org.slf4j.LoggerFactory;
  
  import com.google.common.cache.Cache;
  import com.google.common.cache.CacheBuilder;
@@@ -220,35 -211,8 +221,35 @@@ public class TabletServerLogger 
      }
    }
  
 +  // callers are synchronized already
 +  private void createNextLog() {
 +    if (nextLogMaker.getActiveCount() == 0) {
 +      nextLogMaker.submit(new Runnable() {
 +        @Override
 +        public void run() {
 +          try {
 +            log.debug("Creating next WAL");
 +            DfsLogger alog = new DfsLogger(tserver.getServerConfig(), syncCounter, flushCounter);
 +            alog.open(tserver.getClientAddressString());
 +            for (Tablet tablet : tserver.getOnlineTablets()) {
 +              tserver.addLoggersToMetadata(alog, tablet.getExtent());
 +            }
 +            log.debug("Created next WAL " + alog.getFileName());
 +            alog = nextLog.getAndSet(alog);
 +            if (alog != null) {
 +              log.debug("closing unused next log: " + alog.getFileName());
 +              alog.close();
 +            }
 +          } catch (Exception t) {
-             log.error(t, t);
++            log.error("{}", t.getMessage(), t);
 +          }
 +        }
 +      });
 +    }
 +  }
 +
    public void resetLoggers() throws IOException {
 -    logSetLock.writeLock().lock();
 +    logIdLock.writeLock().lock();
      try {
        close();
      } finally {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ac176e19/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CommitSession.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ac176e19/test/src/test/java/org/apache/accumulo/test/BadDeleteMarkersCreatedIT.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ac176e19/test/src/test/java/org/apache/accumulo/test/CleanWalIT.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ac176e19/test/src/test/java/org/apache/accumulo/test/ConditionalWriterIT.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ac176e19/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
----------------------------------------------------------------------


[10/34] accumulo git commit: ACCUMULO-3423 merge master

Posted by ec...@apache.org.
ACCUMULO-3423 merge master


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

Branch: refs/heads/master
Commit: f4f28c7bad0df92fade2e88da217f60bfb826783
Parents: 31ee26b 01558a4
Author: Eric C. Newton <er...@gmail.com>
Authored: Wed Mar 11 14:41:15 2015 -0400
Committer: Eric C. Newton <er...@gmail.com>
Committed: Wed Mar 11 14:41:15 2015 -0400

----------------------------------------------------------------------
 assemble/bin/tool.sh                            |  7 +-
 .../client/security/tokens/KerberosToken.java   |  3 +-
 .../accumulo/core/file/rfile/PrintInfo.java     | 15 +++-
 .../simple/client/TraceDumpExample.java         | 28 ++++++-
 .../standalone/StandaloneAccumuloCluster.java   | 14 ++--
 .../standalone/StandaloneClusterControl.java    |  5 +-
 .../org/apache/accumulo/proxy/ProxyServer.java  | 18 ++++-
 .../accumulo/test/SizeCompactionStrategy.java   | 59 ++++++++++++++
 .../accumulo/test/TestCompactionStrategy.java   | 72 +++++++++++++++++
 .../accumulo/harness/AccumuloClusterIT.java     |  2 +-
 .../StandaloneAccumuloClusterConfiguration.java | 14 +++-
 .../accumulo/proxy/ProxyDurabilityIT.java       |  8 +-
 .../apache/accumulo/proxy/SimpleProxyBase.java  | 20 ++++-
 .../accumulo/test/BulkImportVolumeIT.java       |  1 +
 .../accumulo/test/ConditionalWriterIT.java      | 21 +++++
 .../org/apache/accumulo/test/MetaSplitIT.java   | 39 ++++++++-
 .../org/apache/accumulo/test/NamespacesIT.java  |  3 +
 .../org/apache/accumulo/test/ShellServerIT.java |  1 -
 .../accumulo/test/UserCompactionStrategyIT.java | 85 --------------------
 .../accumulo/test/functional/CleanUpIT.java     |  4 +-
 .../accumulo/test/functional/CloneTestIT.java   |  4 +-
 .../accumulo/test/functional/CredentialsIT.java | 15 ++--
 .../accumulo/test/functional/ExamplesIT.java    | 45 ++++++++---
 .../accumulo/test/functional/ReadWriteIT.java   | 16 +++-
 .../accumulo/test/functional/ScanIdIT.java      | 36 +++++----
 test/src/test/resources/log4j.properties        |  1 +
 26 files changed, 388 insertions(+), 148 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/f4f28c7b/test/src/test/java/org/apache/accumulo/proxy/ProxyDurabilityIT.java
----------------------------------------------------------------------
diff --cc test/src/test/java/org/apache/accumulo/proxy/ProxyDurabilityIT.java
index de8ebc8,404a8fd..654d27d
--- a/test/src/test/java/org/apache/accumulo/proxy/ProxyDurabilityIT.java
+++ b/test/src/test/java/org/apache/accumulo/proxy/ProxyDurabilityIT.java
@@@ -58,14 -60,9 +60,14 @@@ import com.google.common.net.HostAndPor
  public class ProxyDurabilityIT extends ConfigurableMacIT {
  
    @Override
 +  protected int defaultTimeoutSeconds() {
 +    return 60;
 +  }
 +
 +  @Override
    public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
      hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
-     cfg.setProperty(Property.INSTANCE_ZK_TIMEOUT, "5s");
+     cfg.setProperty(Property.INSTANCE_ZK_TIMEOUT, "10s");
      cfg.setNumTservers(1);
    }
  

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f4f28c7b/test/src/test/java/org/apache/accumulo/test/ConditionalWriterIT.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f4f28c7b/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
----------------------------------------------------------------------


[33/34] accumulo git commit: Merge branch 'master' of https://github.com/ericnewton/accumulo-3423

Posted by ec...@apache.org.
Merge branch 'master' of https://github.com/ericnewton/accumulo-3423


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

Branch: refs/heads/master
Commit: d0a0ac0598b23046de0599c2c061adb173ce6840
Parents: 8b3358c fb25f3b
Author: Eric C. Newton <er...@gmail.com>
Authored: Fri Apr 24 19:05:58 2015 -0400
Committer: Eric C. Newton <er...@gmail.com>
Committed: Fri Apr 24 19:05:58 2015 -0400

----------------------------------------------------------------------

----------------------------------------------------------------------



[29/34] accumulo git commit: ACCUMULO-3423 optimize WAL metadata table updates

Posted by ec...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/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 1735c0d..9f537af 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
@@ -18,7 +18,7 @@ package org.apache.accumulo.gc;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
-import java.util.ArrayList;
+import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
@@ -28,49 +28,56 @@ import java.util.Map.Entry;
 import java.util.Set;
 import java.util.UUID;
 
-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.Connector;
-import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.gc.thrift.GCStatus;
 import org.apache.accumulo.core.gc.thrift.GcCycleStats;
 import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.RootTable;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.CurrentLogsSection;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.ReplicationSection;
 import org.apache.accumulo.core.protobuf.ProtobufUtil;
 import org.apache.accumulo.core.replication.ReplicationSchema.StatusSection;
 import org.apache.accumulo.core.replication.ReplicationTable;
 import org.apache.accumulo.core.replication.ReplicationTableOfflineException;
-import org.apache.accumulo.core.rpc.ThriftUtil;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.tabletserver.log.LogEntry;
-import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
-import org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Client;
-import org.apache.accumulo.core.trace.Span;
 import org.apache.accumulo.core.trace.Trace;
-import org.apache.accumulo.core.trace.Tracer;
-import org.apache.accumulo.core.util.AddressUtil;
+import org.apache.accumulo.core.volume.Volume;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.server.AccumuloServerContext;
 import org.apache.accumulo.server.ServerConstants;
 import org.apache.accumulo.server.fs.VolumeManager;
+import org.apache.accumulo.server.fs.VolumeManagerImpl;
+import org.apache.accumulo.server.master.LiveTServerSet;
+import org.apache.accumulo.server.master.LiveTServerSet.Listener;
+import org.apache.accumulo.server.master.state.MetaDataStateStore;
+import org.apache.accumulo.server.master.state.RootTabletStateStore;
+import org.apache.accumulo.server.master.state.TServerInstance;
+import org.apache.accumulo.server.master.state.TabletLocationState;
+import org.apache.accumulo.server.master.state.TabletState;
 import org.apache.accumulo.server.replication.StatusUtil;
 import org.apache.accumulo.server.replication.proto.Replication.Status;
-import org.apache.accumulo.server.util.MetadataTableUtil;
 import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
-import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.Path;
-import org.apache.thrift.TException;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.io.Text;
+import org.apache.htrace.Span;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.google.common.collect.Iterables;
+import com.google.common.collect.Iterators;
 import com.google.common.net.HostAndPort;
 import com.google.protobuf.InvalidProtocolBufferException;
 
@@ -79,8 +86,8 @@ public class GarbageCollectWriteAheadLogs {
 
   private final AccumuloServerContext context;
   private final VolumeManager fs;
-
-  private boolean useTrash;
+  private final boolean useTrash;
+  private final LiveTServerSet liveServers;
 
   /**
    * Creates a new GC WAL object.
@@ -96,56 +103,37 @@ public class GarbageCollectWriteAheadLogs {
     this.context = context;
     this.fs = fs;
     this.useTrash = useTrash;
-  }
-
-  /**
-   * Gets the instance used by this object.
-   *
-   * @return instance
-   */
-  Instance getInstance() {
-    return context.getInstance();
-  }
-
-  /**
-   * Gets the volume manager used by this object.
-   *
-   * @return volume manager
-   */
-  VolumeManager getVolumeManager() {
-    return fs;
-  }
-
-  /**
-   * Checks if the volume manager should move files to the trash rather than delete them.
-   *
-   * @return true if trash is used
-   */
-  boolean isUsingTrash() {
-    return useTrash;
+    this.liveServers = new LiveTServerSet(context, new Listener() {
+      @Override
+      public void update(LiveTServerSet current, Set<TServerInstance> deleted, Set<TServerInstance> added) {
+        log.debug("New tablet servers noticed: " + added);
+        log.debug("Tablet servers removed: " + deleted);
+      }
+    });
+    liveServers.startListeningForTabletServerChanges();
   }
 
   public void collect(GCStatus status) {
 
-    Span span = Trace.start("scanServers");
+    Span span = Trace.start("getCandidates");
     try {
+      Set<TServerInstance> currentServers = liveServers.getCurrentServers();
 
-      Map<String,Path> sortedWALogs = getSortedWALogs();
 
       status.currentLog.started = System.currentTimeMillis();
 
-      Map<Path,String> fileToServerMap = new HashMap<Path,String>();
-      Map<String,Path> nameToFileMap = new HashMap<String,Path>();
-      int count = scanServers(fileToServerMap, nameToFileMap);
+      Map<TServerInstance, Set<Path> > candidates = new HashMap<>();
+      long count = getCurrent(candidates, currentServers);
       long fileScanStop = System.currentTimeMillis();
-      log.info(String.format("Fetched %d files from %d servers in %.2f seconds", fileToServerMap.size(), count,
+
+      log.info(String.format("Fetched %d files for %d servers in %.2f seconds", count, candidates.size(),
           (fileScanStop - status.currentLog.started) / 1000.));
-      status.currentLog.candidates = fileToServerMap.size();
+      status.currentLog.candidates = count;
       span.stop();
 
-      span = Trace.start("removeMetadataEntries");
+      span = Trace.start("removeEntriesInUse");
       try {
-        count = removeMetadataEntries(nameToFileMap, sortedWALogs, status);
+        count = removeEntriesInUse(candidates, status, currentServers);
       } catch (Exception ex) {
         log.error("Unable to scan metadata table", ex);
         return;
@@ -158,7 +146,7 @@ public class GarbageCollectWriteAheadLogs {
 
       span = Trace.start("removeReplicationEntries");
       try {
-        count = removeReplicationEntries(nameToFileMap, sortedWALogs, status);
+        count = removeReplicationEntries(candidates, status);
       } catch (Exception ex) {
         log.error("Unable to scan replication table", ex);
         return;
@@ -170,16 +158,23 @@ public class GarbageCollectWriteAheadLogs {
       log.info(String.format("%d replication entries scanned in %.2f seconds", count, (replicationEntryScanStop - logEntryScanStop) / 1000.));
 
       span = Trace.start("removeFiles");
-      Map<String,ArrayList<Path>> serverToFileMap = mapServersToFiles(fileToServerMap, nameToFileMap);
 
-      count = removeFiles(nameToFileMap, serverToFileMap, sortedWALogs, status);
+      count = removeFiles(candidates, status);
 
       long removeStop = System.currentTimeMillis();
-      log.info(String.format("%d total logs removed from %d servers in %.2f seconds", count, serverToFileMap.size(), (removeStop - logEntryScanStop) / 1000.));
+      log.info(String.format("%d total logs removed from %d servers in %.2f seconds", count, candidates.size(), (removeStop - logEntryScanStop) / 1000.));
+      span.stop();
+
+      span = Trace.start("removeMarkers");
+      count = removeTabletServerMarkers(candidates);
+      long removeMarkersStop = System.currentTimeMillis();
+      log.info(String.format("%d markers removed in %.2f seconds", count, (removeMarkersStop - removeStop) / 1000.));
+      span.stop();
+
+
       status.currentLog.finished = removeStop;
       status.lastLog = status.currentLog;
       status.currentLog = new GcCycleStats();
-      span.stop();
 
     } catch (Exception e) {
       log.error("exception occured while garbage collecting write ahead logs", e);
@@ -188,161 +183,101 @@ public class GarbageCollectWriteAheadLogs {
     }
   }
 
-  boolean holdsLock(HostAndPort addr) {
+  private long removeTabletServerMarkers(Map<TServerInstance,Set<Path>> candidates) {
+    long result = 0;
     try {
-      String zpath = ZooUtil.getRoot(context.getInstance()) + Constants.ZTSERVERS + "/" + addr.toString();
-      List<String> children = ZooReaderWriter.getInstance().getChildren(zpath);
-      return !(children == null || children.isEmpty());
-    } catch (KeeperException.NoNodeException ex) {
-      return false;
-    } catch (Exception ex) {
-      log.debug(ex.toString(), ex);
-      return true;
-    }
-  }
-
-  private int removeFiles(Map<String,Path> nameToFileMap, Map<String,ArrayList<Path>> serverToFileMap, Map<String,Path> sortedWALogs, final GCStatus status) {
-    for (Entry<String,ArrayList<Path>> entry : serverToFileMap.entrySet()) {
-      if (entry.getKey().isEmpty()) {
-        // old-style log entry, just remove it
-        for (Path path : entry.getValue()) {
-          log.debug("Removing old-style WAL " + path);
-          try {
-            if (!useTrash || !fs.moveToTrash(path))
-              fs.deleteRecursively(path);
-            status.currentLog.deleted++;
-          } catch (FileNotFoundException ex) {
-            // ignored
-          } catch (IOException ex) {
-            log.error("Unable to delete wal " + path + ": " + ex);
-          }
-        }
-      } else {
-        HostAndPort address = AddressUtil.parseAddress(entry.getKey(), false);
-        if (!holdsLock(address)) {
+      BatchWriter root = null;
+      BatchWriter meta = null;
+      try {
+        root = context.getConnector().createBatchWriter(RootTable.NAME, null);
+        meta = context.getConnector().createBatchWriter(MetadataTable.NAME, null);
+        for (Entry<TServerInstance,Set<Path>> entry : candidates.entrySet()) {
+          Mutation m = new Mutation(CurrentLogsSection.getRowPrefix() + entry.getKey().toString());
           for (Path path : entry.getValue()) {
-            log.debug("Removing WAL for offline server " + path);
-            try {
-              if (!useTrash || !fs.moveToTrash(path))
-                fs.deleteRecursively(path);
-              status.currentLog.deleted++;
-            } catch (FileNotFoundException ex) {
-              // ignored
-            } catch (IOException ex) {
-              log.error("Unable to delete wal " + path + ": " + ex);
-            }
-          }
-          continue;
-        } else {
-          Client tserver = null;
-          try {
-            tserver = ThriftUtil.getClient(new TabletClientService.Client.Factory(), address, context);
-            tserver.removeLogs(Tracer.traceInfo(), context.rpcCreds(), paths2strings(entry.getValue()));
-            log.debug("deleted " + entry.getValue() + " from " + entry.getKey());
-            status.currentLog.deleted += entry.getValue().size();
-          } catch (TException e) {
-            log.warn("Error talking to " + address + ": " + e);
-          } finally {
-            if (tserver != null)
-              ThriftUtil.returnClient(tserver);
+            m.putDelete(CurrentLogsSection.COLF, new Text(path.toString()));
+            result++;
           }
+          root.addMutation(m);
+          meta.addMutation(m);
+        }
+      } finally  {
+        if (meta != null) {
+          meta.close();
+        }
+        if (root != null) {
+          root.close();
         }
       }
+    } catch (Exception ex) {
+      throw new RuntimeException(ex);
     }
+    return result;
+  }
 
-    for (Path swalog : sortedWALogs.values()) {
-      log.debug("Removing sorted WAL " + swalog);
-      try {
-        if (!useTrash || !fs.moveToTrash(swalog)) {
-          fs.deleteRecursively(swalog);
-        }
-      } catch (FileNotFoundException ex) {
-        // ignored
-      } catch (IOException ioe) {
+  private long removeFiles(Map<TServerInstance, Set<Path> > candidates, final GCStatus status) {
+    for (Entry<TServerInstance,Set<Path>> entry : candidates.entrySet()) {
+      for (Path path : entry.getValue()) {
+        log.debug("Removing unused WAL for server " + entry.getKey() + " log " + path);
         try {
-          if (fs.exists(swalog)) {
-            log.error("Unable to delete sorted walog " + swalog + ": " + ioe);
-          }
+          if (!useTrash || !fs.moveToTrash(path))
+            fs.deleteRecursively(path);
+          status.currentLog.deleted++;
+        } catch (FileNotFoundException ex) {
+          // ignored
         } catch (IOException ex) {
-          log.error("Unable to check for the existence of " + swalog, ex);
+          log.error("Unable to delete wal " + path + ": " + ex);
         }
       }
     }
-
-    return 0;
+    return status.currentLog.deleted;
   }
 
-  /**
-   * Converts a list of paths to their corresponding strings.
-   *
-   * @param paths
-   *          list of paths
-   * @return string forms of paths
-   */
-  static List<String> paths2strings(List<Path> paths) {
-    List<String> result = new ArrayList<String>(paths.size());
-    for (Path path : paths)
-      result.add(path.toString());
-    return result;
+  private UUID path2uuid(Path path) {
+    return UUID.fromString(path.getName());
   }
 
-  /**
-   * Reverses the given mapping of file paths to servers. The returned map provides a list of file paths for each server. Any path whose name is not in the
-   * mapping of file names to paths is skipped.
-   *
-   * @param fileToServerMap
-   *          map of file paths to servers
-   * @param nameToFileMap
-   *          map of file names to paths
-   * @return map of servers to lists of file paths
-   */
-  static Map<String,ArrayList<Path>> mapServersToFiles(Map<Path,String> fileToServerMap, Map<String,Path> nameToFileMap) {
-    Map<String,ArrayList<Path>> result = new HashMap<String,ArrayList<Path>>();
-    for (Entry<Path,String> fileServer : fileToServerMap.entrySet()) {
-      if (!nameToFileMap.containsKey(fileServer.getKey().getName()))
-        continue;
-      ArrayList<Path> files = result.get(fileServer.getValue());
-      if (files == null) {
-        files = new ArrayList<Path>();
-        result.put(fileServer.getValue(), files);
-      }
-      files.add(fileServer.getKey());
-    }
-    return result;
-  }
-
-  protected int removeMetadataEntries(Map<String,Path> nameToFileMap, Map<String,Path> sortedWALogs, GCStatus status) throws IOException, KeeperException,
+  private long removeEntriesInUse(Map<TServerInstance, Set<Path> > candidates, GCStatus status, Set<TServerInstance> liveServers) throws IOException, KeeperException,
       InterruptedException {
-    int count = 0;
-    Iterator<LogEntry> iterator = MetadataTableUtil.getLogEntries(context);
-
-    // For each WAL reference in the metadata table
-    while (iterator.hasNext()) {
-      // Each metadata reference has at least one WAL file
-      for (String entry : iterator.next().logSet) {
-        // old style WALs will have the IP:Port of their logger and new style will either be a Path either absolute or relative, in all cases
-        // the last "/" will mark a UUID file name.
-        String uuid = entry.substring(entry.lastIndexOf("/") + 1);
-        if (!isUUID(uuid)) {
-          // fully expect this to be a uuid, if its not then something is wrong and walog GC should not proceed!
-          throw new IllegalArgumentException("Expected uuid, but got " + uuid + " from " + entry);
-        }
 
-        Path pathFromNN = nameToFileMap.remove(uuid);
-        if (pathFromNN != null) {
-          status.currentLog.inUse++;
-          sortedWALogs.remove(uuid);
-        }
+    // remove any entries if there's a log reference, or a tablet is still assigned to the dead server
 
-        count++;
+    Map<UUID, TServerInstance> walToDeadServer = new HashMap<>();
+    for (Entry<TServerInstance,Set<Path>> entry : candidates.entrySet()) {
+      for (Path file : entry.getValue()) {
+        walToDeadServer.put(path2uuid(file), entry.getKey());
+      }
+    }
+    long count = 0;
+    RootTabletStateStore root = new RootTabletStateStore(context);
+    MetaDataStateStore meta = new MetaDataStateStore(context);
+    Iterator<TabletLocationState> states = Iterators.concat(root.iterator(), meta.iterator());
+    while (states.hasNext()) {
+      count++;
+      TabletLocationState state = states.next();
+      if (state.getState(liveServers) == TabletState.ASSIGNED_TO_DEAD_SERVER) {
+        candidates.remove(state.current);
+      }
+      for (Collection<String> wals : state.walogs) {
+        for (String wal : wals) {
+          UUID walUUID = path2uuid(new Path(wal));
+          TServerInstance dead = walToDeadServer.get(walUUID);
+          if (dead != null) {
+            Iterator<Path> iter = candidates.get(dead).iterator();
+            while (iter.hasNext()) {
+              if (path2uuid(iter.next()).equals(walUUID)) {
+                iter.remove();
+                break;
+              }
+            }
+          }
+        }
       }
     }
-
     return count;
   }
 
-  protected int removeReplicationEntries(Map<String,Path> nameToFileMap, Map<String,Path> sortedWALogs, GCStatus status) throws IOException, KeeperException,
-      InterruptedException {
+  protected int removeReplicationEntries(Map<TServerInstance, Set<Path> > candidates, GCStatus status) throws IOException, KeeperException,
+  InterruptedException {
     Connector conn;
     try {
       conn = context.getConnector();
@@ -353,21 +288,25 @@ public class GarbageCollectWriteAheadLogs {
 
     int count = 0;
 
-    Iterator<Entry<String,Path>> walIter = nameToFileMap.entrySet().iterator();
+    Iterator<Entry<TServerInstance,Set<Path>>> walIter = candidates.entrySet().iterator();
 
     while (walIter.hasNext()) {
-      Entry<String,Path> wal = walIter.next();
-      String fullPath = wal.getValue().toString();
-      if (neededByReplication(conn, fullPath)) {
-        log.debug("Removing WAL from candidate deletion as it is still needed for replication: {}", fullPath);
-        // If we haven't already removed it, check to see if this WAL is
-        // "in use" by replication (needed for replication purposes)
-        status.currentLog.inUse++;
-
+      Entry<TServerInstance,Set<Path>> wal = walIter.next();
+      Iterator<Path> paths = wal.getValue().iterator();
+      while (paths.hasNext()) {
+        Path fullPath = paths.next();
+        if (neededByReplication(conn, fullPath)) {
+          log.debug("Removing WAL from candidate deletion as it is still needed for replication: {}", fullPath);
+          // If we haven't already removed it, check to see if this WAL is
+          // "in use" by replication (needed for replication purposes)
+          status.currentLog.inUse++;
+          paths.remove();
+        } else {
+          log.debug("WAL not needed for replication {}", fullPath);
+        }
+      }
+      if (wal.getValue().isEmpty()) {
         walIter.remove();
-        sortedWALogs.remove(wal.getKey());
-      } else {
-        log.debug("WAL not needed for replication {}", fullPath);
       }
       count++;
     }
@@ -375,6 +314,7 @@ public class GarbageCollectWriteAheadLogs {
     return count;
   }
 
+
   /**
    * Determine if the given WAL is needed for replication
    *
@@ -382,7 +322,7 @@ public class GarbageCollectWriteAheadLogs {
    *          The full path (URI)
    * @return True if the WAL is still needed by replication (not a candidate for deletion)
    */
-  protected boolean neededByReplication(Connector conn, String wal) {
+  protected boolean neededByReplication(Connector conn, Path wal) {
     log.info("Checking replication table for " + wal);
 
     Iterable<Entry<Key,Value>> iter = getReplicationStatusForFile(conn, wal);
@@ -405,7 +345,7 @@ public class GarbageCollectWriteAheadLogs {
     return false;
   }
 
-  protected Iterable<Entry<Key,Value>> getReplicationStatusForFile(Connector conn, String wal) {
+  protected Iterable<Entry<Key,Value>> getReplicationStatusForFile(Connector conn, Path wal) {
     Scanner metaScanner;
     try {
       metaScanner = conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
@@ -425,7 +365,7 @@ public class GarbageCollectWriteAheadLogs {
       StatusSection.limit(replScanner);
 
       // Only look for this specific WAL
-      replScanner.setRange(Range.exact(wal));
+      replScanner.setRange(Range.exact(wal.toString()));
 
       return Iterables.concat(metaScanner, replScanner);
     } catch (ReplicationTableOfflineException e) {
@@ -435,107 +375,84 @@ public class GarbageCollectWriteAheadLogs {
     return metaScanner;
   }
 
-  private int scanServers(Map<Path,String> fileToServerMap, Map<String,Path> nameToFileMap) throws Exception {
-    return scanServers(ServerConstants.getWalDirs(), fileToServerMap, nameToFileMap);
-  }
-
-  /**
-   * Scans write-ahead log directories for logs. The maps passed in are populated with scan information.
-   *
-   * @param walDirs
-   *          write-ahead log directories
-   * @param fileToServerMap
-   *          map of file paths to servers
-   * @param nameToFileMap
-   *          map of file names to paths
-   * @return number of servers located (including those with no logs present)
-   */
-  int scanServers(String[] walDirs, Map<Path,String> fileToServerMap, Map<String,Path> nameToFileMap) throws Exception {
-    Set<String> servers = new HashSet<String>();
-    for (String walDir : walDirs) {
-      Path walRoot = new Path(walDir);
-      FileStatus[] listing = null;
-      try {
-        listing = fs.listStatus(walRoot);
-      } catch (FileNotFoundException e) {
-        // ignore dir
-      }
 
-      if (listing == null)
-        continue;
-      for (FileStatus status : listing) {
-        String server = status.getPath().getName();
-        if (status.isDirectory()) {
-          servers.add(server);
-          for (FileStatus file : fs.listStatus(new Path(walRoot, server))) {
-            if (isUUID(file.getPath().getName())) {
-              fileToServerMap.put(file.getPath(), server);
-              nameToFileMap.put(file.getPath().getName(), file.getPath());
-            } else {
-              log.info("Ignoring file " + file.getPath() + " because it doesn't look like a uuid");
-            }
-          }
-        } else if (isUUID(server)) {
-          // old-style WAL are not under a directory
-          servers.add("");
-          fileToServerMap.put(status.getPath(), "");
-          nameToFileMap.put(server, status.getPath());
-        } else {
-          log.info("Ignoring file " + status.getPath() + " because it doesn't look like a uuid");
-        }
-      }
-    }
-    return servers.size();
-  }
 
-  private Map<String,Path> getSortedWALogs() throws IOException {
-    return getSortedWALogs(ServerConstants.getRecoveryDirs());
-  }
 
   /**
-   * Looks for write-ahead logs in recovery directories.
+   * Scans log markers. The map passed in is populated with the logs for dead servers.
    *
-   * @param recoveryDirs
-   *          recovery directories
-   * @return map of log file names to paths
+   * @param unusedLogs
+   *          map of dead server to log file entries
+   * @return total number of log files
    */
-  Map<String,Path> getSortedWALogs(String[] recoveryDirs) throws IOException {
-    Map<String,Path> result = new HashMap<String,Path>();
-
-    for (String dir : recoveryDirs) {
-      Path recoveryDir = new Path(dir);
-
-      if (fs.exists(recoveryDir)) {
-        for (FileStatus status : fs.listStatus(recoveryDir)) {
-          String name = status.getPath().getName();
-          if (isUUID(name)) {
-            result.put(name, status.getPath());
-          } else {
-            log.debug("Ignoring file " + status.getPath() + " because it doesn't look like a uuid");
-          }
+  private long getCurrent(Map<TServerInstance, Set<Path> > unusedLogs, Set<TServerInstance> currentServers) throws Exception {
+    Set<Path> rootWALs = new HashSet<>();
+    // Get entries in zookeeper:
+    String zpath = ZooUtil.getRoot(context.getInstance()) + RootTable.ZROOT_TABLET_WALOGS;
+    ZooReaderWriter zoo = ZooReaderWriter.getInstance();
+    List<String> children = zoo.getChildren(zpath);
+    for (String child : children) {
+      LogEntry entry = LogEntry.fromBytes(zoo.getData(zpath + "/" + child, null));
+      rootWALs.add(new Path(entry.filename));
+    }
+    long count = 0;
+
+    // get all the WAL markers that are not in zookeeper for dead servers
+    Scanner rootScanner = context.getConnector().createScanner(RootTable.NAME, Authorizations.EMPTY);
+    rootScanner.setRange(CurrentLogsSection.getRange());
+    Scanner metaScanner = context.getConnector().createScanner(MetadataTable.NAME, Authorizations.EMPTY);
+    metaScanner.setRange(CurrentLogsSection.getRange());
+    Iterator<Entry<Key,Value>> entries = Iterators.concat(rootScanner.iterator(), metaScanner.iterator());
+    Text hostAndPort = new Text();
+    Text sessionId = new Text();
+    Text filename = new Text();
+    while (entries.hasNext()) {
+      Entry<Key,Value> entry = entries.next();
+      CurrentLogsSection.getTabletServer(entry.getKey(), hostAndPort, sessionId);
+      CurrentLogsSection.getPath(entry.getKey(), filename);
+      TServerInstance tsi = new TServerInstance(HostAndPort.fromString(hostAndPort.toString()), sessionId.toString());
+      Path path = new Path(filename.toString());
+      if (!currentServers.contains(tsi) || entry.getValue().equals(CurrentLogsSection.UNUSED) && !rootWALs.contains(path)) {
+        Set<Path> logs = unusedLogs.get(tsi);
+        if (logs == null) {
+          unusedLogs.put(tsi, logs = new HashSet<Path>());
+        }
+        if (logs.add(path)) {
+          count++;
         }
       }
     }
-    return result;
-  }
 
-  /**
-   * Checks if a string is a valid UUID.
-   *
-   * @param name
-   *          string to check
-   * @return true if string is a UUID
-   */
-  static boolean isUUID(String name) {
-    if (name == null || name.length() != 36) {
-      return false;
-    }
-    try {
-      UUID.fromString(name);
-      return true;
-    } catch (IllegalArgumentException ex) {
-      return false;
+    // scan HDFS for logs for dead servers
+    for (Volume volume : VolumeManagerImpl.get().getVolumes()) {
+      RemoteIterator<LocatedFileStatus> iter =  volume.getFileSystem().listFiles(volume.prefixChild(ServerConstants.WAL_DIR), true);
+      while (iter.hasNext()) {
+        LocatedFileStatus next = iter.next();
+        // recursive listing returns directories, too
+        if (next.isDirectory()) {
+          continue;
+        }
+        // make sure we've waited long enough for zookeeper propagation
+        if (System.currentTimeMillis() - next.getModificationTime() < context.getConnector().getInstance().getZooKeepersSessionTimeOut()) {
+          continue;
+        }
+        Path path = next.getPath();
+        String hostPlusPort = path.getParent().getName();
+        // server is still alive, or has a replacement
+        TServerInstance instance = liveServers.find(hostPlusPort);
+        if (instance != null) {
+          continue;
+        }
+        TServerInstance fake = new TServerInstance(hostPlusPort, 0L);
+        Set<Path> paths = unusedLogs.get(fake);
+        if (paths == null) {
+          paths = new HashSet<>();
+        }
+        paths.add(path);
+        unusedLogs.put(fake, paths);
+      }
     }
+    return count;
   }
 
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/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 037023a..4f64c15 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
@@ -569,7 +569,6 @@ public class SimpleGarbageCollector extends AccumuloServerContext implements Ifa
         replSpan.stop();
       }
 
-      // Clean up any unused write-ahead logs
       Span waLogs = Trace.start("walogs");
       try {
         GarbageCollectWriteAheadLogs walogCollector = new GarbageCollectWriteAheadLogs(this, fs, isUsingTrash());

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/server/gc/src/main/java/org/apache/accumulo/gc/replication/CloseWriteAheadLogReferences.java
----------------------------------------------------------------------
diff --git a/server/gc/src/main/java/org/apache/accumulo/gc/replication/CloseWriteAheadLogReferences.java b/server/gc/src/main/java/org/apache/accumulo/gc/replication/CloseWriteAheadLogReferences.java
index 3a32727..455aaee 100644
--- a/server/gc/src/main/java/org/apache/accumulo/gc/replication/CloseWriteAheadLogReferences.java
+++ b/server/gc/src/main/java/org/apache/accumulo/gc/replication/CloseWriteAheadLogReferences.java
@@ -37,13 +37,11 @@ import org.apache.accumulo.core.file.rfile.RFile;
 import org.apache.accumulo.core.master.thrift.MasterClientService;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.CurrentLogsSection;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.ReplicationSection;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LogColumnFamily;
 import org.apache.accumulo.core.replication.ReplicationTable;
 import org.apache.accumulo.core.rpc.ThriftUtil;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.tabletserver.log.LogEntry;
 import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
 import org.apache.accumulo.core.trace.Span;
 import org.apache.accumulo.core.trace.Trace;
@@ -186,20 +184,21 @@ public class CloseWriteAheadLogReferences implements Runnable {
     try {
       // TODO Configurable number of threads
       bs = conn.createBatchScanner(MetadataTable.NAME, Authorizations.EMPTY, 4);
-      bs.setRanges(Collections.singleton(TabletsSection.getRange()));
-      bs.fetchColumnFamily(LogColumnFamily.NAME);
+      bs.setRanges(Collections.singleton(CurrentLogsSection.getRange()));
+      bs.fetchColumnFamily(CurrentLogsSection.COLF);
 
       // For each log key/value in the metadata table
       for (Entry<Key,Value> entry : bs) {
-        // The value may contain multiple WALs
-        LogEntry logEntry = LogEntry.fromKeyValue(entry.getKey(), entry.getValue());
-
-        log.debug("Found WALs for table(" + logEntry.extent.getTableId() + "): " + logEntry.logSet);
+        if (entry.getValue().equals(CurrentLogsSection.UNUSED)) {
+          continue;
+        }
+        Text tpath = new Text();
+        CurrentLogsSection.getPath(entry.getKey(), tpath);
+        String path = new Path(tpath.toString()).toString();
+        log.debug("Found WAL " + path.toString());
 
         // Normalize each log file (using Path) and add it to the set
-        for (String logFile : logEntry.logSet) {
-          referencedWals.add(normalizedWalPaths.get(logFile));
-        }
+        referencedWals.add(normalizedWalPaths.get(path));
       }
     } catch (TableNotFoundException e) {
       // uhhhh

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/server/gc/src/test/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogsTest.java
----------------------------------------------------------------------
diff --git a/server/gc/src/test/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogsTest.java b/server/gc/src/test/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogsTest.java
deleted file mode 100644
index 5801faa..0000000
--- a/server/gc/src/test/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogsTest.java
+++ /dev/null
@@ -1,567 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.gc;
-
-import static org.easymock.EasyMock.createMock;
-import static org.easymock.EasyMock.expect;
-import static org.easymock.EasyMock.replay;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertSame;
-import static org.junit.Assert.assertTrue;
-
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.UUID;
-
-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.mock.MockInstance;
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.ConfigurationCopy;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.conf.SiteConfiguration;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.gc.thrift.GCStatus;
-import org.apache.accumulo.core.gc.thrift.GcCycleStats;
-import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.ReplicationSection;
-import org.apache.accumulo.core.protobuf.ProtobufUtil;
-import org.apache.accumulo.core.replication.ReplicationSchema.StatusSection;
-import org.apache.accumulo.core.replication.ReplicationTable;
-import org.apache.accumulo.server.AccumuloServerContext;
-import org.apache.accumulo.server.conf.ServerConfigurationFactory;
-import org.apache.accumulo.server.fs.VolumeManager;
-import org.apache.accumulo.server.replication.StatusUtil;
-import org.apache.accumulo.server.replication.proto.Replication.Status;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
-import org.easymock.EasyMock;
-import org.easymock.IAnswer;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
-
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Maps;
-
-public class GarbageCollectWriteAheadLogsTest {
-  private static final long BLOCK_SIZE = 64000000L;
-
-  private static final Path DIR_1_PATH = new Path("/dir1");
-  private static final Path DIR_2_PATH = new Path("/dir2");
-  private static final Path DIR_3_PATH = new Path("/dir3");
-  private static final String UUID1 = UUID.randomUUID().toString();
-  private static final String UUID2 = UUID.randomUUID().toString();
-  private static final String UUID3 = UUID.randomUUID().toString();
-
-  private Instance instance;
-  private AccumuloConfiguration systemConfig;
-  private VolumeManager volMgr;
-  private GarbageCollectWriteAheadLogs gcwal;
-  private long modTime;
-
-  @Rule
-  public TestName testName = new TestName();
-
-  @Before
-  public void setUp() throws Exception {
-    SiteConfiguration siteConfig = EasyMock.createMock(SiteConfiguration.class);
-    instance = createMock(Instance.class);
-    expect(instance.getInstanceID()).andReturn("mock").anyTimes();
-    expect(instance.getZooKeepers()).andReturn("localhost").anyTimes();
-    expect(instance.getZooKeepersSessionTimeOut()).andReturn(30000).anyTimes();
-    systemConfig = new ConfigurationCopy(new HashMap<String,String>());
-    volMgr = createMock(VolumeManager.class);
-    ServerConfigurationFactory factory = createMock(ServerConfigurationFactory.class);
-    expect(factory.getConfiguration()).andReturn(systemConfig).anyTimes();
-    expect(factory.getInstance()).andReturn(instance).anyTimes();
-    expect(factory.getSiteConfiguration()).andReturn(siteConfig).anyTimes();
-
-    // Just make the SiteConfiguration delegate to our AccumuloConfiguration
-    // Presently, we only need get(Property) and iterator().
-    EasyMock.expect(siteConfig.get(EasyMock.anyObject(Property.class))).andAnswer(new IAnswer<String>() {
-      @Override
-      public String answer() {
-        Object[] args = EasyMock.getCurrentArguments();
-        return systemConfig.get((Property) args[0]);
-      }
-    }).anyTimes();
-    EasyMock.expect(siteConfig.getBoolean(EasyMock.anyObject(Property.class))).andAnswer(new IAnswer<Boolean>() {
-      @Override
-      public Boolean answer() {
-        Object[] args = EasyMock.getCurrentArguments();
-        return systemConfig.getBoolean((Property) args[0]);
-      }
-    }).anyTimes();
-
-    EasyMock.expect(siteConfig.iterator()).andAnswer(new IAnswer<Iterator<Entry<String,String>>>() {
-      @Override
-      public Iterator<Entry<String,String>> answer() {
-        return systemConfig.iterator();
-      }
-    }).anyTimes();
-
-    replay(instance, factory, siteConfig);
-    AccumuloServerContext context = new AccumuloServerContext(factory);
-    gcwal = new GarbageCollectWriteAheadLogs(context, volMgr, false);
-    modTime = System.currentTimeMillis();
-  }
-
-  @Test
-  public void testGetters() {
-    assertSame(instance, gcwal.getInstance());
-    assertSame(volMgr, gcwal.getVolumeManager());
-    assertFalse(gcwal.isUsingTrash());
-  }
-
-  @Test
-  public void testPathsToStrings() {
-    ArrayList<Path> paths = new ArrayList<Path>();
-    paths.add(new Path(DIR_1_PATH, "file1"));
-    paths.add(DIR_2_PATH);
-    paths.add(new Path(DIR_3_PATH, "file3"));
-    List<String> strings = GarbageCollectWriteAheadLogs.paths2strings(paths);
-    int len = 3;
-    assertEquals(len, strings.size());
-    for (int i = 0; i < len; i++) {
-      assertEquals(paths.get(i).toString(), strings.get(i));
-    }
-  }
-
-  @Test
-  public void testMapServersToFiles() {
-    // @formatter:off
-    /*
-     * Test fileToServerMap:
-     * /dir1/server1/uuid1 -> server1 (new-style)
-     * /dir1/uuid2 -> "" (old-style)
-     * /dir3/server3/uuid3 -> server3 (new-style)
-     */
-    // @formatter:on
-    Map<Path,String> fileToServerMap = new java.util.HashMap<Path,String>();
-    Path path1 = new Path(new Path(DIR_1_PATH, "server1"), UUID1);
-    fileToServerMap.put(path1, "server1"); // new-style
-    Path path2 = new Path(DIR_1_PATH, UUID2);
-    fileToServerMap.put(path2, ""); // old-style
-    Path path3 = new Path(new Path(DIR_3_PATH, "server3"), UUID3);
-    fileToServerMap.put(path3, "server3"); // old-style
-    // @formatter:off
-    /*
-     * Test nameToFileMap:
-     * uuid1 -> /dir1/server1/uuid1
-     * uuid3 -> /dir3/server3/uuid3
-     */
-    // @formatter:on
-    Map<String,Path> nameToFileMap = new java.util.HashMap<String,Path>();
-    nameToFileMap.put(UUID1, path1);
-    nameToFileMap.put(UUID3, path3);
-
-    // @formatter:off
-    /*
-     * Expected map:
-     * server1 -> [ /dir1/server1/uuid1 ]
-     * server3 -> [ /dir3/server3/uuid3 ]
-     */
-    // @formatter:on
-    Map<String,ArrayList<Path>> result = GarbageCollectWriteAheadLogs.mapServersToFiles(fileToServerMap, nameToFileMap);
-    assertEquals(2, result.size());
-    ArrayList<Path> list1 = result.get("server1");
-    assertEquals(1, list1.size());
-    assertTrue(list1.contains(path1));
-    ArrayList<Path> list3 = result.get("server3");
-    assertEquals(1, list3.size());
-    assertTrue(list3.contains(path3));
-  }
-
-  private FileStatus makeFileStatus(int size, Path path) {
-    boolean isDir = (size == 0);
-    return new FileStatus(size, isDir, 3, BLOCK_SIZE, modTime, path);
-  }
-
-  private void mockListStatus(Path dir, FileStatus... fileStatuses) throws Exception {
-    expect(volMgr.listStatus(dir)).andReturn(fileStatuses);
-  }
-
-  @Test
-  public void testScanServers_NewStyle() throws Exception {
-    String[] walDirs = new String[] {"/dir1", "/dir2", "/dir3"};
-    // @formatter:off
-    /*
-     * Test directory layout:
-     * /dir1/
-     *   server1/
-     *     uuid1
-     *     file2
-     *   subdir2/
-     * /dir2/ missing
-     * /dir3/
-     *   server3/
-     *     uuid3
-     */
-    // @formatter:on
-    Path serverDir1Path = new Path(DIR_1_PATH, "server1");
-    FileStatus serverDir1 = makeFileStatus(0, serverDir1Path);
-    Path subDir2Path = new Path(DIR_1_PATH, "subdir2");
-    FileStatus serverDir2 = makeFileStatus(0, subDir2Path);
-    mockListStatus(DIR_1_PATH, serverDir1, serverDir2);
-    Path path1 = new Path(serverDir1Path, UUID1);
-    FileStatus file1 = makeFileStatus(100, path1);
-    FileStatus file2 = makeFileStatus(200, new Path(serverDir1Path, "file2"));
-    mockListStatus(serverDir1Path, file1, file2);
-    mockListStatus(subDir2Path);
-    expect(volMgr.listStatus(DIR_2_PATH)).andThrow(new FileNotFoundException());
-    Path serverDir3Path = new Path(DIR_3_PATH, "server3");
-    FileStatus serverDir3 = makeFileStatus(0, serverDir3Path);
-    mockListStatus(DIR_3_PATH, serverDir3);
-    Path path3 = new Path(serverDir3Path, UUID3);
-    FileStatus file3 = makeFileStatus(300, path3);
-    mockListStatus(serverDir3Path, file3);
-    replay(volMgr);
-
-    Map<Path,String> fileToServerMap = new java.util.HashMap<Path,String>();
-    Map<String,Path> nameToFileMap = new java.util.HashMap<String,Path>();
-    int count = gcwal.scanServers(walDirs, fileToServerMap, nameToFileMap);
-    assertEquals(3, count);
-    // @formatter:off
-    /*
-     * Expected fileToServerMap:
-     * /dir1/server1/uuid1 -> server1
-     * /dir3/server3/uuid3 -> server3
-     */
-    // @formatter:on
-    assertEquals(2, fileToServerMap.size());
-    assertEquals("server1", fileToServerMap.get(path1));
-    assertEquals("server3", fileToServerMap.get(path3));
-    // @formatter:off
-    /*
-     * Expected nameToFileMap:
-     * uuid1 -> /dir1/server1/uuid1
-     * uuid3 -> /dir3/server3/uuid3
-     */
-    // @formatter:on
-    assertEquals(2, nameToFileMap.size());
-    assertEquals(path1, nameToFileMap.get(UUID1));
-    assertEquals(path3, nameToFileMap.get(UUID3));
-  }
-
-  @Test
-  public void testScanServers_OldStyle() throws Exception {
-    // @formatter:off
-    /*
-     * Test directory layout:
-     * /dir1/
-     *   uuid1
-     * /dir3/
-     *   uuid3
-     */
-    // @formatter:on
-    String[] walDirs = new String[] {"/dir1", "/dir3"};
-    Path serverFile1Path = new Path(DIR_1_PATH, UUID1);
-    FileStatus serverFile1 = makeFileStatus(100, serverFile1Path);
-    mockListStatus(DIR_1_PATH, serverFile1);
-    Path serverFile3Path = new Path(DIR_3_PATH, UUID3);
-    FileStatus serverFile3 = makeFileStatus(300, serverFile3Path);
-    mockListStatus(DIR_3_PATH, serverFile3);
-    replay(volMgr);
-
-    Map<Path,String> fileToServerMap = new java.util.HashMap<Path,String>();
-    Map<String,Path> nameToFileMap = new java.util.HashMap<String,Path>();
-    int count = gcwal.scanServers(walDirs, fileToServerMap, nameToFileMap);
-    /*
-     * Expect only a single server, the non-server entry for upgrade WALs
-     */
-    assertEquals(1, count);
-    // @formatter:off
-    /*
-     * Expected fileToServerMap:
-     * /dir1/uuid1 -> ""
-     * /dir3/uuid3 -> ""
-     */
-    // @formatter:on
-    assertEquals(2, fileToServerMap.size());
-    assertEquals("", fileToServerMap.get(serverFile1Path));
-    assertEquals("", fileToServerMap.get(serverFile3Path));
-    // @formatter:off
-    /*
-     * Expected nameToFileMap:
-     * uuid1 -> /dir1/uuid1
-     * uuid3 -> /dir3/uuid3
-     */
-    // @formatter:on
-    assertEquals(2, nameToFileMap.size());
-    assertEquals(serverFile1Path, nameToFileMap.get(UUID1));
-    assertEquals(serverFile3Path, nameToFileMap.get(UUID3));
-  }
-
-  @Test
-  public void testGetSortedWALogs() throws Exception {
-    String[] recoveryDirs = new String[] {"/dir1", "/dir2", "/dir3"};
-    // @formatter:off
-    /*
-     * Test directory layout:
-     * /dir1/
-     *   uuid1
-     *   file2
-     * /dir2/ missing
-     * /dir3/
-     *   uuid3
-     */
-    // @formatter:on
-    expect(volMgr.exists(DIR_1_PATH)).andReturn(true);
-    expect(volMgr.exists(DIR_2_PATH)).andReturn(false);
-    expect(volMgr.exists(DIR_3_PATH)).andReturn(true);
-    Path path1 = new Path(DIR_1_PATH, UUID1);
-    FileStatus file1 = makeFileStatus(100, path1);
-    FileStatus file2 = makeFileStatus(200, new Path(DIR_1_PATH, "file2"));
-    mockListStatus(DIR_1_PATH, file1, file2);
-    Path path3 = new Path(DIR_3_PATH, UUID3);
-    FileStatus file3 = makeFileStatus(300, path3);
-    mockListStatus(DIR_3_PATH, file3);
-    replay(volMgr);
-
-    Map<String,Path> sortedWalogs = gcwal.getSortedWALogs(recoveryDirs);
-    // @formatter:off
-    /*
-     * Expected map:
-     * uuid1 -> /dir1/uuid1
-     * uuid3 -> /dir3/uuid3
-     */
-    // @formatter:on
-    assertEquals(2, sortedWalogs.size());
-    assertEquals(path1, sortedWalogs.get(UUID1));
-    assertEquals(path3, sortedWalogs.get(UUID3));
-  }
-
-  @Test
-  public void testIsUUID() {
-    assertTrue(GarbageCollectWriteAheadLogs.isUUID(UUID.randomUUID().toString()));
-    assertFalse(GarbageCollectWriteAheadLogs.isUUID("foo"));
-    assertFalse(GarbageCollectWriteAheadLogs.isUUID("0" + UUID.randomUUID().toString()));
-    assertFalse(GarbageCollectWriteAheadLogs.isUUID(null));
-  }
-
-  // It was easier to do this than get the mocking working for me
-  private static class ReplicationGCWAL extends GarbageCollectWriteAheadLogs {
-
-    private List<Entry<Key,Value>> replData;
-
-    ReplicationGCWAL(AccumuloServerContext context, VolumeManager fs, boolean useTrash, List<Entry<Key,Value>> replData) throws IOException {
-      super(context, fs, useTrash);
-      this.replData = replData;
-    }
-
-    @Override
-    protected Iterable<Entry<Key,Value>> getReplicationStatusForFile(Connector conn, String wal) {
-      return this.replData;
-    }
-  }
-
-  @Test
-  public void replicationEntriesAffectGC() throws Exception {
-    String file1 = UUID.randomUUID().toString(), file2 = UUID.randomUUID().toString();
-    Connector conn = createMock(Connector.class);
-
-    // Write a Status record which should prevent file1 from being deleted
-    LinkedList<Entry<Key,Value>> replData = new LinkedList<>();
-    replData.add(Maps.immutableEntry(new Key("/wals/" + file1, StatusSection.NAME.toString(), "1"), StatusUtil.fileCreatedValue(System.currentTimeMillis())));
-
-    ReplicationGCWAL replGC = new ReplicationGCWAL(null, volMgr, false, replData);
-
-    replay(conn);
-
-    // Open (not-closed) file must be retained
-    assertTrue(replGC.neededByReplication(conn, "/wals/" + file1));
-
-    // No replication data, not needed
-    replData.clear();
-    assertFalse(replGC.neededByReplication(conn, "/wals/" + file2));
-
-    // The file is closed but not replicated, must be retained
-    replData.add(Maps.immutableEntry(new Key("/wals/" + file1, StatusSection.NAME.toString(), "1"), StatusUtil.fileClosedValue()));
-    assertTrue(replGC.neededByReplication(conn, "/wals/" + file1));
-
-    // File is closed and fully replicated, can be deleted
-    replData.clear();
-    replData.add(Maps.immutableEntry(new Key("/wals/" + file1, StatusSection.NAME.toString(), "1"),
-        ProtobufUtil.toValue(Status.newBuilder().setInfiniteEnd(true).setBegin(Long.MAX_VALUE).setClosed(true).build())));
-    assertFalse(replGC.neededByReplication(conn, "/wals/" + file1));
-  }
-
-  @Test
-  public void removeReplicationEntries() throws Exception {
-    String file1 = UUID.randomUUID().toString(), file2 = UUID.randomUUID().toString();
-
-    Instance inst = new MockInstance(testName.getMethodName());
-    AccumuloServerContext context = new AccumuloServerContext(new ServerConfigurationFactory(inst));
-
-    GarbageCollectWriteAheadLogs gcWALs = new GarbageCollectWriteAheadLogs(context, volMgr, false);
-
-    long file1CreateTime = System.currentTimeMillis();
-    long file2CreateTime = file1CreateTime + 50;
-    BatchWriter bw = ReplicationTable.getBatchWriter(context.getConnector());
-    Mutation m = new Mutation("/wals/" + file1);
-    StatusSection.add(m, new Text("1"), StatusUtil.fileCreatedValue(file1CreateTime));
-    bw.addMutation(m);
-    m = new Mutation("/wals/" + file2);
-    StatusSection.add(m, new Text("1"), StatusUtil.fileCreatedValue(file2CreateTime));
-    bw.addMutation(m);
-
-    // These WALs are potential candidates for deletion from fs
-    Map<String,Path> nameToFileMap = new HashMap<>();
-    nameToFileMap.put(file1, new Path("/wals/" + file1));
-    nameToFileMap.put(file2, new Path("/wals/" + file2));
-
-    Map<String,Path> sortedWALogs = Collections.emptyMap();
-
-    // Make the GCStatus and GcCycleStats
-    GCStatus status = new GCStatus();
-    GcCycleStats cycleStats = new GcCycleStats();
-    status.currentLog = cycleStats;
-
-    // We should iterate over two entries
-    Assert.assertEquals(2, gcWALs.removeReplicationEntries(nameToFileMap, sortedWALogs, status));
-
-    // We should have noted that two files were still in use
-    Assert.assertEquals(2l, cycleStats.inUse);
-
-    // Both should have been deleted
-    Assert.assertEquals(0, nameToFileMap.size());
-  }
-
-  @Test
-  public void replicationEntriesOnlyInMetaPreventGC() throws Exception {
-    String file1 = UUID.randomUUID().toString(), file2 = UUID.randomUUID().toString();
-
-    Instance inst = new MockInstance(testName.getMethodName());
-    AccumuloServerContext context = new AccumuloServerContext(new ServerConfigurationFactory(inst));
-
-    Connector conn = context.getConnector();
-
-    GarbageCollectWriteAheadLogs gcWALs = new GarbageCollectWriteAheadLogs(context, volMgr, false);
-
-    long file1CreateTime = System.currentTimeMillis();
-    long file2CreateTime = file1CreateTime + 50;
-    // Write some records to the metadata table, we haven't yet written status records to the replication table
-    BatchWriter bw = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
-    Mutation m = new Mutation(ReplicationSection.getRowPrefix() + "/wals/" + file1);
-    m.put(ReplicationSection.COLF, new Text("1"), StatusUtil.fileCreatedValue(file1CreateTime));
-    bw.addMutation(m);
-
-    m = new Mutation(ReplicationSection.getRowPrefix() + "/wals/" + file2);
-    m.put(ReplicationSection.COLF, new Text("1"), StatusUtil.fileCreatedValue(file2CreateTime));
-    bw.addMutation(m);
-
-    // These WALs are potential candidates for deletion from fs
-    Map<String,Path> nameToFileMap = new HashMap<>();
-    nameToFileMap.put(file1, new Path("/wals/" + file1));
-    nameToFileMap.put(file2, new Path("/wals/" + file2));
-
-    Map<String,Path> sortedWALogs = Collections.emptyMap();
-
-    // Make the GCStatus and GcCycleStats objects
-    GCStatus status = new GCStatus();
-    GcCycleStats cycleStats = new GcCycleStats();
-    status.currentLog = cycleStats;
-
-    // We should iterate over two entries
-    Assert.assertEquals(2, gcWALs.removeReplicationEntries(nameToFileMap, sortedWALogs, status));
-
-    // We should have noted that two files were still in use
-    Assert.assertEquals(2l, cycleStats.inUse);
-
-    // Both should have been deleted
-    Assert.assertEquals(0, nameToFileMap.size());
-  }
-
-  @Test
-  public void noReplicationTableDoesntLimitMetatdataResults() throws Exception {
-    Instance inst = new MockInstance(testName.getMethodName());
-    AccumuloServerContext context = new AccumuloServerContext(new ServerConfigurationFactory(inst));
-    Connector conn = context.getConnector();
-
-    String wal = "hdfs://localhost:8020/accumulo/wal/tserver+port/123456-1234-1234-12345678";
-    BatchWriter bw = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
-    Mutation m = new Mutation(ReplicationSection.getRowPrefix() + wal);
-    m.put(ReplicationSection.COLF, new Text("1"), StatusUtil.fileCreatedValue(System.currentTimeMillis()));
-    bw.addMutation(m);
-    bw.close();
-
-    GarbageCollectWriteAheadLogs gcWALs = new GarbageCollectWriteAheadLogs(context, volMgr, false);
-
-    Iterable<Entry<Key,Value>> data = gcWALs.getReplicationStatusForFile(conn, wal);
-    Entry<Key,Value> entry = Iterables.getOnlyElement(data);
-
-    Assert.assertEquals(ReplicationSection.getRowPrefix() + wal, entry.getKey().getRow().toString());
-  }
-
-  @Test
-  public void fetchesReplicationEntriesFromMetadataAndReplicationTables() throws Exception {
-    Instance inst = new MockInstance(testName.getMethodName());
-    AccumuloServerContext context = new AccumuloServerContext(new ServerConfigurationFactory(inst));
-    Connector conn = context.getConnector();
-
-    long walCreateTime = System.currentTimeMillis();
-    String wal = "hdfs://localhost:8020/accumulo/wal/tserver+port/123456-1234-1234-12345678";
-    BatchWriter bw = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
-    Mutation m = new Mutation(ReplicationSection.getRowPrefix() + wal);
-    m.put(ReplicationSection.COLF, new Text("1"), StatusUtil.fileCreatedValue(walCreateTime));
-    bw.addMutation(m);
-    bw.close();
-
-    bw = ReplicationTable.getBatchWriter(conn);
-    m = new Mutation(wal);
-    StatusSection.add(m, new Text("1"), StatusUtil.fileCreatedValue(walCreateTime));
-    bw.addMutation(m);
-    bw.close();
-
-    GarbageCollectWriteAheadLogs gcWALs = new GarbageCollectWriteAheadLogs(context, volMgr, false);
-
-    Iterable<Entry<Key,Value>> iter = gcWALs.getReplicationStatusForFile(conn, wal);
-    Map<Key,Value> data = new HashMap<>();
-    for (Entry<Key,Value> e : iter) {
-      data.put(e.getKey(), e.getValue());
-    }
-
-    Assert.assertEquals(2, data.size());
-
-    // Should get one element from each table (metadata and replication)
-    for (Key k : data.keySet()) {
-      String row = k.getRow().toString();
-      if (row.startsWith(ReplicationSection.getRowPrefix())) {
-        Assert.assertTrue(row.endsWith(wal));
-      } else {
-        Assert.assertEquals(wal, row);
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/server/gc/src/test/java/org/apache/accumulo/gc/replication/CloseWriteAheadLogReferencesTest.java
----------------------------------------------------------------------
diff --git a/server/gc/src/test/java/org/apache/accumulo/gc/replication/CloseWriteAheadLogReferencesTest.java b/server/gc/src/test/java/org/apache/accumulo/gc/replication/CloseWriteAheadLogReferencesTest.java
index 3115de1..78a5bd5 100644
--- a/server/gc/src/test/java/org/apache/accumulo/gc/replication/CloseWriteAheadLogReferencesTest.java
+++ b/server/gc/src/test/java/org/apache/accumulo/gc/replication/CloseWriteAheadLogReferencesTest.java
@@ -50,14 +50,12 @@ import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.data.impl.KeyExtent;
 import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.CurrentLogsSection;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.ReplicationSection;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LogColumnFamily;
 import org.apache.accumulo.core.protobuf.ProtobufUtil;
 import org.apache.accumulo.core.replication.ReplicationSchema.StatusSection;
 import org.apache.accumulo.core.replication.ReplicationTable;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.tabletserver.log.LogEntry;
 import org.apache.accumulo.core.trace.thrift.TInfo;
 import org.apache.accumulo.server.AccumuloServerContext;
 import org.apache.accumulo.server.conf.ServerConfigurationFactory;
@@ -130,22 +128,16 @@ public class CloseWriteAheadLogReferencesTest {
   public void findOneWalFromMetadata() throws Exception {
     Connector conn = createMock(Connector.class);
     BatchScanner bs = createMock(BatchScanner.class);
-
     // Fake out some data
     final ArrayList<Entry<Key,Value>> data = new ArrayList<>();
-    LogEntry logEntry = new LogEntry();
-    logEntry.extent = new KeyExtent(new Text("1"), new Text("b"), new Text("a"));
-    logEntry.filename = "hdfs://localhost:8020/accumulo/wal/tserver+port/" + UUID.randomUUID();
-    logEntry.server = "tserver1";
-    logEntry.tabletId = 1;
-    logEntry.logSet = Collections.singleton(logEntry.filename);
-    data.add(Maps.immutableEntry(new Key(logEntry.getRow(), logEntry.getColumnFamily(), logEntry.getColumnQualifier()), new Value(logEntry.getValue())));
+    String file = "hdfs://localhost:8020/accumulo/wal/tserver1+9997/" + UUID.randomUUID();
+    data.add(entry("tserver1:9997[1234567890]", file));
 
     // Get a batchscanner, scan the tablets section, fetch only the logs
     expect(conn.createBatchScanner(MetadataTable.NAME, Authorizations.EMPTY, 4)).andReturn(bs);
-    bs.setRanges(Collections.singleton(TabletsSection.getRange()));
+    bs.setRanges(Collections.singleton(CurrentLogsSection.getRange()));
     expectLastCall().once();
-    bs.fetchColumnFamily(LogColumnFamily.NAME);
+    bs.fetchColumnFamily(CurrentLogsSection.COLF);
     expectLastCall().once();
     expect(bs.iterator()).andAnswer(new IAnswer<Iterator<Entry<Key,Value>>>() {
 
@@ -163,54 +155,12 @@ public class CloseWriteAheadLogReferencesTest {
 
     // Validate
     Set<String> wals = refs.getReferencedWals(conn);
-    Assert.assertEquals(Collections.singleton(logEntry.filename), wals);
-
-    verify(conn, bs);
-  }
-
-  @Test
-  public void findManyWalFromSingleMetadata() throws Exception {
-    Connector conn = createMock(Connector.class);
-    BatchScanner bs = createMock(BatchScanner.class);
-
-    // Fake out some data
-    final ArrayList<Entry<Key,Value>> data = new ArrayList<>();
-    LogEntry logEntry = new LogEntry();
-    logEntry.extent = new KeyExtent(new Text("1"), new Text("b"), new Text("a"));
-    logEntry.filename = "hdfs://localhost:8020/accumulo/wal/tserver+port/" + UUID.randomUUID();
-    logEntry.server = "tserver1";
-    logEntry.tabletId = 1;
-    // Multiple DFSLoggers
-    logEntry.logSet = Sets.newHashSet(logEntry.filename, "hdfs://localhost:8020/accumulo/wal/tserver+port/" + UUID.randomUUID());
-    data.add(Maps.immutableEntry(new Key(logEntry.getRow(), logEntry.getColumnFamily(), logEntry.getColumnQualifier()), new Value(logEntry.getValue())));
-
-    // Get a batchscanner, scan the tablets section, fetch only the logs
-    expect(conn.createBatchScanner(MetadataTable.NAME, Authorizations.EMPTY, 4)).andReturn(bs);
-    bs.setRanges(Collections.singleton(TabletsSection.getRange()));
-    expectLastCall().once();
-    bs.fetchColumnFamily(LogColumnFamily.NAME);
-    expectLastCall().once();
-    expect(bs.iterator()).andAnswer(new IAnswer<Iterator<Entry<Key,Value>>>() {
-
-      @Override
-      public Iterator<Entry<Key,Value>> answer() throws Throwable {
-        return data.iterator();
-      }
-
-    });
-    // Close the bs
-    bs.close();
-    expectLastCall().once();
-
-    replay(conn, bs);
-
-    // Validate
-    Set<String> wals = refs.getReferencedWals(conn);
-    Assert.assertEquals(logEntry.logSet, wals);
+    Assert.assertEquals(Collections.singleton(file), wals);
 
     verify(conn, bs);
   }
 
+  // This is a silly test now
   @Test
   public void findManyRefsToSingleWalFromMetadata() throws Exception {
     Connector conn = createMock(Connector.class);
@@ -220,31 +170,14 @@ public class CloseWriteAheadLogReferencesTest {
 
     // Fake out some data
     final ArrayList<Entry<Key,Value>> data = new ArrayList<>();
-    LogEntry logEntry = new LogEntry();
-    logEntry.extent = new KeyExtent(new Text("1"), new Text("b"), new Text("a"));
-    logEntry.filename = "hdfs://localhost:8020/accumulo/wal/tserver+port/" + uuid;
-    logEntry.server = "tserver1";
-    logEntry.tabletId = 1;
-    logEntry.logSet = Collections.singleton(logEntry.filename);
-    data.add(Maps.immutableEntry(new Key(logEntry.getRow(), logEntry.getColumnFamily(), logEntry.getColumnQualifier()), new Value(logEntry.getValue())));
-
-    logEntry.extent = new KeyExtent(new Text("1"), new Text("c"), new Text("b"));
-    logEntry.server = "tserver1";
-    logEntry.tabletId = 2;
-    logEntry.logSet = Collections.singleton(logEntry.filename);
-    data.add(Maps.immutableEntry(new Key(logEntry.getRow(), logEntry.getColumnFamily(), logEntry.getColumnQualifier()), new Value(logEntry.getValue())));
-
-    logEntry.extent = new KeyExtent(new Text("1"), null, new Text("c"));
-    logEntry.server = "tserver1";
-    logEntry.tabletId = 3;
-    logEntry.logSet = Collections.singleton(logEntry.filename);
-    data.add(Maps.immutableEntry(new Key(logEntry.getRow(), logEntry.getColumnFamily(), logEntry.getColumnQualifier()), new Value(logEntry.getValue())));
+    String filename = "hdfs://localhost:8020/accumulo/wal/tserver+9997/" + uuid;
+    data.add(entry("tserver1:9997[0123456789]", filename));
 
     // Get a batchscanner, scan the tablets section, fetch only the logs
     expect(conn.createBatchScanner(MetadataTable.NAME, Authorizations.EMPTY, 4)).andReturn(bs);
-    bs.setRanges(Collections.singleton(TabletsSection.getRange()));
+    bs.setRanges(Collections.singleton(CurrentLogsSection.getRange()));
     expectLastCall().once();
-    bs.fetchColumnFamily(LogColumnFamily.NAME);
+    bs.fetchColumnFamily(CurrentLogsSection.COLF);
     expectLastCall().once();
     expect(bs.iterator()).andAnswer(new IAnswer<Iterator<Entry<Key,Value>>>() {
 
@@ -262,7 +195,7 @@ public class CloseWriteAheadLogReferencesTest {
 
     // Validate
     Set<String> wals = refs.getReferencedWals(conn);
-    Assert.assertEquals(Collections.singleton(logEntry.filename), wals);
+    Assert.assertEquals(Collections.singleton(filename), wals);
 
     verify(conn, bs);
   }
@@ -272,59 +205,22 @@ public class CloseWriteAheadLogReferencesTest {
     Connector conn = createMock(Connector.class);
     BatchScanner bs = createMock(BatchScanner.class);
 
-    String file1 = "hdfs://localhost:8020/accumulo/wal/tserver1+port/" + UUID.randomUUID(), file2 = "hdfs://localhost:8020/accumulo/wal/tserver2+port/"
-        + UUID.randomUUID(), file3 = "hdfs://localhost:8020/accumulo/wal/tserver3+port/" + UUID.randomUUID();
+    String file1 = "hdfs://localhost:8020/accumulo/wal/tserver1+9997/" + UUID.randomUUID();
+    String file2 = "hdfs://localhost:8020/accumulo/wal/tserver2+9997/" + UUID.randomUUID();
+    String file3 = "hdfs://localhost:8020/accumulo/wal/tserver3+9997/" + UUID.randomUUID();
 
     // Fake out some data
     final ArrayList<Entry<Key,Value>> data = new ArrayList<>();
-    LogEntry logEntry = new LogEntry();
-    logEntry.extent = new KeyExtent(new Text("1"), new Text("b"), new Text("a"));
-    logEntry.filename = file1;
-    logEntry.server = "tserver1";
-    logEntry.tabletId = 1;
-    logEntry.logSet = Collections.singleton(logEntry.filename);
-    data.add(Maps.immutableEntry(new Key(logEntry.getRow(), logEntry.getColumnFamily(), logEntry.getColumnQualifier()), new Value(logEntry.getValue())));
-
-    logEntry.extent = new KeyExtent(new Text("5"), null, null);
-    logEntry.tabletId = 2;
-    data.add(Maps.immutableEntry(new Key(logEntry.getRow(), logEntry.getColumnFamily(), logEntry.getColumnQualifier()), new Value(logEntry.getValue())));
-
-    logEntry.extent = new KeyExtent(new Text("3"), new Text("b"), new Text("a"));
-    logEntry.filename = file2;
-    logEntry.server = "tserver2";
-    logEntry.tabletId = 3;
-    logEntry.logSet = Collections.singleton(logEntry.filename);
-    data.add(Maps.immutableEntry(new Key(logEntry.getRow(), logEntry.getColumnFamily(), logEntry.getColumnQualifier()), new Value(logEntry.getValue())));
-
-    logEntry.extent = new KeyExtent(new Text("3"), new Text("c"), new Text("b"));
-    logEntry.tabletId = 4;
-    logEntry.logSet = Collections.singleton(logEntry.filename);
-    data.add(Maps.immutableEntry(new Key(logEntry.getRow(), logEntry.getColumnFamily(), logEntry.getColumnQualifier()), new Value(logEntry.getValue())));
-
-    logEntry.extent = new KeyExtent(new Text("4"), new Text("5"), new Text("0"));
-    logEntry.filename = file3;
-    logEntry.server = "tserver3";
-    logEntry.tabletId = 5;
-    logEntry.logSet = Collections.singleton(logEntry.filename);
-    data.add(Maps.immutableEntry(new Key(logEntry.getRow(), logEntry.getColumnFamily(), logEntry.getColumnQualifier()), new Value(logEntry.getValue())));
-
-    logEntry.extent = new KeyExtent(new Text("4"), new Text("8"), new Text("5"));
-    logEntry.server = "tserver3";
-    logEntry.tabletId = 7;
-    logEntry.logSet = Collections.singleton(logEntry.filename);
-    data.add(Maps.immutableEntry(new Key(logEntry.getRow(), logEntry.getColumnFamily(), logEntry.getColumnQualifier()), new Value(logEntry.getValue())));
-
-    logEntry.extent = new KeyExtent(new Text("4"), null, new Text("8"));
-    logEntry.server = "tserver3";
-    logEntry.tabletId = 15;
-    logEntry.logSet = Collections.singleton(logEntry.filename);
-    data.add(Maps.immutableEntry(new Key(logEntry.getRow(), logEntry.getColumnFamily(), logEntry.getColumnQualifier()), new Value(logEntry.getValue())));
+
+    data.add(entry("tserver1:9997[1234567890]", file1));
+    data.add(entry("tserver2:9997[1234567891]", file2));
+    data.add(entry("tserver3:9997[1234567891]", file3));
 
     // Get a batchscanner, scan the tablets section, fetch only the logs
     expect(conn.createBatchScanner(MetadataTable.NAME, Authorizations.EMPTY, 4)).andReturn(bs);
-    bs.setRanges(Collections.singleton(TabletsSection.getRange()));
+    bs.setRanges(Collections.singleton(CurrentLogsSection.getRange()));
     expectLastCall().once();
-    bs.fetchColumnFamily(LogColumnFamily.NAME);
+    bs.fetchColumnFamily(CurrentLogsSection.COLF);
     expectLastCall().once();
     expect(bs.iterator()).andAnswer(new IAnswer<Iterator<Entry<Key,Value>>>() {
 
@@ -347,6 +243,11 @@ public class CloseWriteAheadLogReferencesTest {
     verify(conn, bs);
   }
 
+  private static Entry<Key,Value> entry(String session, String file) {
+    Key key = new Key(new Text(CurrentLogsSection.getRowPrefix() + session), CurrentLogsSection.COLF, new Text(file));
+    return Maps.immutableEntry(key, new Value());
+  }
+
   @Test
   public void unusedWalsAreClosed() throws Exception {
     Set<String> wals = Collections.emptySet();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/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 4d52cc5..810134b 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
@@ -422,6 +422,9 @@ public class Master extends AccumuloServerContext implements LiveTServerSet.List
           perm.grantNamespacePermission(user, Namespaces.ACCUMULO_NAMESPACE_ID, NamespacePermission.READ);
         }
         perm.grantNamespacePermission("root", Namespaces.ACCUMULO_NAMESPACE_ID, NamespacePermission.ALTER_TABLE);
+
+        // add the currlog location for root tablet current logs
+        zoo.putPersistentData(ZooUtil.getRoot(getInstance()) + RootTable.ZROOT_TABLET_CURRENT_LOGS, new byte[0], NodeExistsPolicy.SKIP);
         haveUpgradedZooKeeper = true;
       } catch (Exception ex) {
         // ACCUMULO-3651 Changed level to error and added FATAL to message for slf4j compatibility

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/server/master/src/main/java/org/apache/accumulo/master/MasterClientServiceHandler.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/MasterClientServiceHandler.java b/server/master/src/main/java/org/apache/accumulo/master/MasterClientServiceHandler.java
index 592d9ae..3f15b39 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/MasterClientServiceHandler.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/MasterClientServiceHandler.java
@@ -173,7 +173,8 @@ class MasterClientServiceHandler extends FateServiceHandler implements MasterCli
           scanner.setRange(MetadataSchema.TabletsSection.getRange());
         } else {
           scanner = new IsolatedScanner(conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY));
-          scanner.setRange(new KeyExtent(new Text(tableId), null, ByteBufferUtil.toText(startRow)).toMetadataRange());
+          Range range = new KeyExtent(new Text(tableId), null, ByteBufferUtil.toText(startRow)).toMetadataRange();
+          scanner.setRange(range.clip(MetadataSchema.TabletsSection.getRange()));
         }
         TabletsSection.ServerColumnFamily.FLUSH_COLUMN.fetch(scanner);
         TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN.fetch(scanner);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/server/master/src/main/java/org/apache/accumulo/master/TabletGroupWatcher.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/TabletGroupWatcher.java b/server/master/src/main/java/org/apache/accumulo/master/TabletGroupWatcher.java
index 4c47953..20917a4 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/TabletGroupWatcher.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/TabletGroupWatcher.java
@@ -163,6 +163,7 @@ class TabletGroupWatcher extends Daemon {
         List<Assignment> assigned = new ArrayList<Assignment>();
         List<TabletLocationState> assignedToDeadServers = new ArrayList<TabletLocationState>();
         Map<KeyExtent,TServerInstance> unassigned = new HashMap<KeyExtent,TServerInstance>();
+        Map<TServerInstance, List<Path>> logsForDeadServers = new TreeMap<>();
 
         MasterState masterState = master.getMasterState();
         int[] counts = new int[TabletState.values().length];
@@ -175,6 +176,7 @@ class TabletGroupWatcher extends Daemon {
           if (tls == null) {
             continue;
           }
+          Master.log.debug(store.name() + " location State: " + tls);
           // ignore entries for tables that do not exist in zookeeper
           if (TableManager.getInstance().getTableState(tls.extent.getTableId().toString()) == null)
             continue;
@@ -184,7 +186,7 @@ class TabletGroupWatcher extends Daemon {
 
           // Don't overwhelm the tablet servers with work
           if (unassigned.size() + unloaded > Master.MAX_TSERVER_WORK_CHUNK * currentTServers.size()) {
-            flushChanges(destinations, assignments, assigned, assignedToDeadServers, unassigned);
+            flushChanges(destinations, assignments, assigned, assignedToDeadServers, logsForDeadServers, unassigned);
             assignments.clear();
             assigned.clear();
             assignedToDeadServers.clear();
@@ -204,8 +206,9 @@ class TabletGroupWatcher extends Daemon {
           TabletGoalState goal = this.master.getGoalState(tls, mergeStats.getMergeInfo());
           TServerInstance server = tls.getServer();
           TabletState state = tls.getState(currentTServers.keySet());
-          if (Master.log.isTraceEnabled())
-            Master.log.trace("Goal state " + goal + " current " + state);
+          if (Master.log.isTraceEnabled()) {
+            Master.log.trace("Goal state " + goal + " current " + state + " for " + tls.extent);
+          }
           stats.update(tableId, state);
           mergeStats.update(tls.extent, state, tls.chopped, !tls.walogs.isEmpty());
           sendChopRequest(mergeStats.getMergeInfo(), state, tls);
@@ -239,7 +242,7 @@ class TabletGroupWatcher extends Daemon {
                 assignedToDeadServers.add(tls);
                 if (server.equals(this.master.migrations.get(tls.extent)))
                   this.master.migrations.remove(tls.extent);
-                // log.info("Current servers " + currentTServers.keySet());
+                MetadataTableUtil.fetchLogsForDeadServer(master, master.getMasterLock(), tls.extent, tls.futureOrCurrent(), logsForDeadServers);
                 break;
               case UNASSIGNED:
                 // maybe it's a finishing migration
@@ -273,7 +276,7 @@ class TabletGroupWatcher extends Daemon {
                 break;
               case ASSIGNED_TO_DEAD_SERVER:
                 assignedToDeadServers.add(tls);
-                // log.info("Current servers " + currentTServers.keySet());
+                MetadataTableUtil.fetchLogsForDeadServer(master, master.getMasterLock(), tls.extent, tls.futureOrCurrent(), logsForDeadServers);
                 break;
               case HOSTED:
                 TServerConnection conn = this.master.tserverSet.getConnection(server);
@@ -292,7 +295,8 @@ class TabletGroupWatcher extends Daemon {
           counts[state.ordinal()]++;
         }
 
-        flushChanges(destinations, assignments, assigned, assignedToDeadServers, unassigned);
+        flushChanges(destinations, assignments, assigned, assignedToDeadServers, logsForDeadServers, unassigned);
+        store.markLogsAsUnused(master, logsForDeadServers);
 
         // provide stats after flushing changes to avoid race conditions w/ delete table
         stats.end(masterState);
@@ -312,8 +316,12 @@ class TabletGroupWatcher extends Daemon {
 
         updateMergeState(mergeStatsCache);
 
-        Master.log.debug(String.format("[%s] sleeping for %.2f seconds", store.name(), Master.TIME_TO_WAIT_BETWEEN_SCANS / 1000.));
-        eventListener.waitForEvents(Master.TIME_TO_WAIT_BETWEEN_SCANS);
+        if (this.master.tserverSet.getCurrentServers().equals(currentTServers.keySet())) {
+          Master.log.debug(String.format("[%s] sleeping for %.2f seconds", store.name(), Master.TIME_TO_WAIT_BETWEEN_SCANS / 1000.));
+          eventListener.waitForEvents(Master.TIME_TO_WAIT_BETWEEN_SCANS);
+        } else {
+          Master.log.info("Detected change in current tserver set, re-running state machine.");
+        }
       } catch (Exception ex) {
         Master.log.error("Error processing table state for store " + store.name(), ex);
         if (ex.getCause() != null && ex.getCause() instanceof BadLocationStateException) {
@@ -730,12 +738,19 @@ class TabletGroupWatcher extends Daemon {
     }
   }
 
-  private void flushChanges(SortedMap<TServerInstance,TabletServerStatus> currentTServers, List<Assignment> assignments, List<Assignment> assigned,
-      List<TabletLocationState> assignedToDeadServers, Map<KeyExtent,TServerInstance> unassigned) throws DistributedStoreException, TException {
+  private void flushChanges(
+      SortedMap<TServerInstance,TabletServerStatus> currentTServers,
+      List<Assignment> assignments,
+      List<Assignment> assigned,
+      List<TabletLocationState> assignedToDeadServers,
+      Map<TServerInstance, List<Path>> logsForDeadServers,
+      Map<KeyExtent,TServerInstance> unassigned)
+          throws DistributedStoreException, TException {
     if (!assignedToDeadServers.isEmpty()) {
       int maxServersToShow = min(assignedToDeadServers.size(), 100);
       Master.log.debug(assignedToDeadServers.size() + " assigned to dead servers: " + assignedToDeadServers.subList(0, maxServersToShow) + "...");
-      store.unassign(assignedToDeadServers);
+      Master.log.debug("logs for dead servers: " + logsForDeadServers);
+      store.unassign(assignedToDeadServers, logsForDeadServers);
       this.master.nextEvent.event("Marked %d tablets as unassigned because they don't have current servers", assignedToDeadServers.size());
     }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/server/master/src/main/java/org/apache/accumulo/master/replication/WorkMaker.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/replication/WorkMaker.java b/server/master/src/main/java/org/apache/accumulo/master/replication/WorkMaker.java
index 8532e1b..74e9b78 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/replication/WorkMaker.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/replication/WorkMaker.java
@@ -107,6 +107,7 @@ public class WorkMaker {
         // Don't create the record if we have nothing to do.
         // TODO put this into a filter on serverside
         if (!shouldCreateWork(status)) {
+          log.debug("Not creating work: " + status.toString());
           continue;
         }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/server/master/src/main/java/org/apache/accumulo/master/state/MergeStats.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/state/MergeStats.java b/server/master/src/main/java/org/apache/accumulo/master/state/MergeStats.java
index a3c7e46..f1763be 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/state/MergeStats.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/state/MergeStats.java
@@ -30,6 +30,7 @@ import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.data.impl.KeyExtent;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.RootTable;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.server.cli.ClientOpts;
@@ -187,7 +188,7 @@ public class MergeStats {
     Text tableId = extent.getTableId();
     Text first = KeyExtent.getMetadataEntry(tableId, start);
     Range range = new Range(first, false, null, true);
-    scanner.setRange(range);
+    scanner.setRange(range.clip(MetadataSchema.TabletsSection.getRange()));
     KeyExtent prevExtent = null;
 
     log.debug("Scanning range " + range);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/server/master/src/test/java/org/apache/accumulo/master/ReplicationOperationsImplTest.java
----------------------------------------------------------------------
diff --git a/server/master/src/test/java/org/apache/accumulo/master/ReplicationOperationsImplTest.java b/server/master/src/test/java/org/apache/accumulo/master/ReplicationOperationsImplTest.java
index 6790858..3c3bc37 100644
--- a/server/master/src/test/java/org/apache/accumulo/master/ReplicationOperationsImplTest.java
+++ b/server/master/src/test/java/org/apache/accumulo/master/ReplicationOperationsImplTest.java
@@ -16,7 +16,6 @@
  */
 package org.apache.accumulo.master;
 
-import java.util.Arrays;
 import java.util.Map.Entry;
 import java.util.Set;
 import java.util.UUID;
@@ -306,13 +305,7 @@ public class ReplicationOperationsImplTest {
     bw.addMutation(m);
     bw.close();
 
-    LogEntry logEntry = new LogEntry();
-    logEntry.extent = new KeyExtent(new Text(tableId1), null, null);
-    logEntry.server = "tserver";
-    logEntry.filename = file1;
-    logEntry.tabletId = 1;
-    logEntry.logSet = Arrays.asList(file1);
-    logEntry.timestamp = System.currentTimeMillis();
+    LogEntry logEntry = new LogEntry(new KeyExtent(new Text(tableId1), null, null), System.currentTimeMillis(), "tserver", file1);
 
     bw = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
     m = new Mutation(ReplicationSection.getRowPrefix() + file1);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/server/master/src/test/java/org/apache/accumulo/master/TestMergeState.java
----------------------------------------------------------------------
diff --git a/server/master/src/test/java/org/apache/accumulo/master/TestMergeState.java b/server/master/src/test/java/org/apache/accumulo/master/TestMergeState.java
index 634ee89..8cbea68 100644
--- a/server/master/src/test/java/org/apache/accumulo/master/TestMergeState.java
+++ b/server/master/src/test/java/org/apache/accumulo/master/TestMergeState.java
@@ -186,7 +186,7 @@ public class TestMergeState {
     // take it offline
     m = tablet.getPrevRowUpdateMutation();
     Collection<Collection<String>> walogs = Collections.emptyList();
-    metaDataStateStore.unassign(Collections.singletonList(new TabletLocationState(tablet, null, state.someTServer, null, walogs, false)));
+    metaDataStateStore.unassign(Collections.singletonList(new TabletLocationState(tablet, null, state.someTServer, null, walogs, false)), null);
 
     // now we can split
     stats = scan(state, metaDataStateStore);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/server/master/src/test/java/org/apache/accumulo/master/state/RootTabletStateStoreTest.java
----------------------------------------------------------------------
diff --git a/server/master/src/test/java/org/apache/accumulo/master/state/RootTabletStateStoreTest.java b/server/master/src/test/java/org/apache/accumulo/master/state/RootTabletStateStoreTest.java
index d2cc0cf..4002da5 100644
--- a/server/master/src/test/java/org/apache/accumulo/master/state/RootTabletStateStoreTest.java
+++ b/server/master/src/test/java/org/apache/accumulo/master/state/RootTabletStateStoreTest.java
@@ -181,7 +181,7 @@ public class RootTabletStateStoreTest {
     } catch (BadLocationStateException e) {
       fail("Unexpected error " + e);
     }
-    tstore.unassign(Collections.singletonList(assigned));
+    tstore.unassign(Collections.singletonList(assigned), null);
     count = 0;
     for (TabletLocationState location : tstore) {
       assertEquals(location.extent, root);
@@ -209,7 +209,7 @@ public class RootTabletStateStoreTest {
       fail("Unexpected error " + e);
     }
     try {
-      tstore.unassign(Collections.singletonList(broken));
+      tstore.unassign(Collections.singletonList(broken), null);
       Assert.fail("should not get here");
     } catch (IllegalArgumentException ex) {}
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/server/tserver/src/main/findbugs/exclude-filter.xml
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/findbugs/exclude-filter.xml b/server/tserver/src/main/findbugs/exclude-filter.xml
index 47dd1f5..a334163 100644
--- a/server/tserver/src/main/findbugs/exclude-filter.xml
+++ b/server/tserver/src/main/findbugs/exclude-filter.xml
@@ -18,7 +18,7 @@
   <Match>
     <!-- locking is confusing, but probably correct -->
     <Class name="org.apache.accumulo.tserver.tablet.Tablet" />
-    <Method name="beginUpdatingLogsUsed" params="org.apache.accumulo.tserver.InMemoryMap,java.util.Collection,boolean" returns="boolean" />
+    <Method name="beginUpdatingLogsUsed" params="org.apache.accumulo.tserver.InMemoryMap,org.apache.accumulo.tserver.log.DfsLogger,boolean" returns="boolean" />
     <Bug code="UL" pattern="UL_UNRELEASED_LOCK" />
   </Match>
   <Match>

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/server/tserver/src/main/java/org/apache/accumulo/server/GarbageCollectionLogger.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/server/GarbageCollectionLogger.java b/server/tserver/src/main/java/org/apache/accumulo/server/GarbageCollectionLogger.java
index 389a544..5fe2548 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/server/GarbageCollectionLogger.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/server/GarbageCollectionLogger.java
@@ -35,7 +35,8 @@ public class GarbageCollectionLogger {
   private long gcTimeIncreasedCount = 0;
   private static long lastMemoryCheckTime = 0;
 
-  public GarbageCollectionLogger() {}
+  public GarbageCollectionLogger() {
+  }
 
   public synchronized void logGCInfo(AccumuloConfiguration conf) {
     final long now = System.currentTimeMillis();


[17/34] accumulo git commit: ACCUMULO-3423 more updates based on [~kturner]'s review

Posted by ec...@apache.org.
ACCUMULO-3423 more updates based on [~kturner]'s review


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

Branch: refs/heads/master
Commit: affff422ddd91cd96c8ca84dc824d931a1b72f70
Parents: c71fc12
Author: Eric C. Newton <er...@gmail.com>
Authored: Mon Mar 30 16:05:59 2015 -0400
Committer: Eric C. Newton <er...@gmail.com>
Committed: Mon Mar 30 16:05:59 2015 -0400

----------------------------------------------------------------------
 .../accumulo/server/util/MetadataTableUtil.java |  1 +
 .../gc/GarbageCollectWriteAheadLogs.java        | 34 ++++++++
 .../apache/accumulo/tserver/TabletServer.java   | 24 ++++--
 .../tserver/log/TabletServerLogger.java         | 40 ++++------
 .../accumulo/test/GarbageCollectWALIT.java      | 81 ++++++++++++++++++++
 .../accumulo/test/functional/WALSunnyDayIT.java | 10 +++
 6 files changed, 159 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/affff422/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java
----------------------------------------------------------------------
diff --git 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
index f5326bf..edc6189 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
@@ -1112,6 +1112,7 @@ public class MetadataTableUtil {
   }
 
   public static void markLogUnused(ClientContext context, ZooLock lock, TServerInstance tabletSession, Set<Path> all) throws AccumuloException {
+    // There could be a marker at the meta and/or root level, mark them both as unused
     try {
       BatchWriter root = null;
       BatchWriter meta = null;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/affff422/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 d523706..59612ab 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
@@ -50,9 +50,12 @@ import org.apache.accumulo.core.replication.ReplicationTableOfflineException;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.tabletserver.log.LogEntry;
 import org.apache.accumulo.core.trace.Trace;
+import org.apache.accumulo.core.volume.Volume;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.server.AccumuloServerContext;
+import org.apache.accumulo.server.ServerConstants;
 import org.apache.accumulo.server.fs.VolumeManager;
+import org.apache.accumulo.server.fs.VolumeManagerImpl;
 import org.apache.accumulo.server.master.LiveTServerSet;
 import org.apache.accumulo.server.master.LiveTServerSet.Listener;
 import org.apache.accumulo.server.master.state.MetaDataStateStore;
@@ -63,7 +66,9 @@ import org.apache.accumulo.server.master.state.TabletState;
 import org.apache.accumulo.server.replication.StatusUtil;
 import org.apache.accumulo.server.replication.proto.Replication.Status;
 import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
+import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.io.Text;
 import org.apache.htrace.Span;
 import org.apache.zookeeper.KeeperException;
@@ -406,6 +411,35 @@ public class GarbageCollectWriteAheadLogs {
       }
     }
 
+    // scan HDFS for logs for dead servers
+    for (Volume volume : VolumeManagerImpl.get().getVolumes()) {
+      RemoteIterator<LocatedFileStatus> iter =  volume.getFileSystem().listFiles(volume.prefixChild(ServerConstants.WAL_DIR), true);
+      while (iter.hasNext()) {
+        LocatedFileStatus next = iter.next();
+        // recursive listing returns directories, too
+        if (next.isDirectory()) {
+          continue;
+        }
+        // make sure we've waited long enough for zookeeper propagation
+        if (System.currentTimeMillis() - next.getModificationTime() < context.getConnector().getInstance().getZooKeepersSessionTimeOut()) {
+          continue;
+        }
+        Path path = next.getPath();
+        String hostPlusPort = path.getParent().getName();
+        // server is still alive, or has a replacement
+        TServerInstance instance = liveServers.find(hostPlusPort);
+        if (instance != null) {
+          continue;
+        }
+        TServerInstance fake = new TServerInstance(hostPlusPort, 0L);
+        Set<Path> paths = unusedLogs.get(fake);
+        if (paths == null) {
+          paths = new HashSet<>();
+        }
+        paths.add(path);
+        unusedLogs.put(fake, paths);
+      }
+    }
     return count;
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/affff422/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
index 436fb12..e28d472 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
@@ -2988,7 +2988,14 @@ public class TabletServer extends AccumuloServerContext implements Runnable {
   }
 
   // avoid unnecessary redundant markings to meta
-  ConcurrentHashMap<DfsLogger, EnumSet<TabletLevel>> metadataTableLogs = new ConcurrentHashMap<>();
+  final ConcurrentHashMap<DfsLogger, EnumSet<TabletLevel>> metadataTableLogs = new ConcurrentHashMap<>();
+  final Object levelLocks[] = new Object[TabletLevel.values().length];
+  {
+    for (int i = 0; i < levelLocks.length; i++) {
+      levelLocks[i] = new Object();
+    }
+  }
+
 
   // remove any meta entries after a rolled log is no longer referenced
   Set<DfsLogger> closedLogs = new HashSet<>();
@@ -3018,14 +3025,19 @@ public class TabletServer extends AccumuloServerContext implements Runnable {
   public void addLoggersToMetadata(DfsLogger copy, KeyExtent extent) {
     TabletLevel level = TabletLevel.getLevel(extent);
     // serialize the updates to the metadata per level: avoids updating the level more than once
-    synchronized (level) {
-      EnumSet<TabletLevel> set = metadataTableLogs.putIfAbsent(copy, EnumSet.of(level));
+    // updating one level, may cause updates to other levels, so we need to release the lock on metadataTableLogs
+    synchronized (levelLocks[level.ordinal()]) {
+      EnumSet<TabletLevel> set = null;
+      synchronized (metadataTableLogs) {
+        set = metadataTableLogs.putIfAbsent(copy, EnumSet.of(level));
+      }
       if (set == null || !set.contains(level) || level == TabletLevel.ROOT) {
         log.info("Writing log marker for level " + level + " " + copy.getFileName());
         MetadataTableUtil.addNewLogMarker(this, this.getLock(), this.getTabletSession(), copy.getPath(), extent);
-        if (set != null) {
-          set.add(level);
-        }
+      }
+      synchronized (metadataTableLogs) {
+        set = metadataTableLogs.get(copy);
+        set.add(level);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/affff422/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
index 0f3f642..04e7a83 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
@@ -25,6 +25,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
+import java.util.concurrent.SynchronousQueue;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -77,7 +78,7 @@ public class TabletServerLogger {
 
   // The current logger
   private DfsLogger currentLog = null;
-  private final AtomicReference<DfsLogger> nextLog = new AtomicReference<>(null);
+  private final SynchronousQueue<DfsLogger> nextLog = new SynchronousQueue<>();
   private final ThreadPoolExecutor nextLogMaker = new SimpleThreadPool(1, "WALog creator");
 
   // The current generation of logs.
@@ -149,6 +150,7 @@ public class TabletServerLogger {
     this.maxSize = maxSize;
     this.syncCounter = syncCounter;
     this.flushCounter = flushCounter;
+    startLogMaker();
   }
 
   private DfsLogger initializeLoggers(final AtomicInteger logIdOut) throws IOException {
@@ -198,18 +200,9 @@ public class TabletServerLogger {
     }
 
     try {
-      DfsLogger next = nextLog.getAndSet(null);
-      if (next != null) {
-        log.info("Using next log " + next.getFileName());
-        currentLog = next;
-      } else {
-        DfsLogger alog = new DfsLogger(tserver.getServerConfig(), syncCounter, flushCounter);
-        alog.open(tserver.getClientAddressString());
-        currentLog = alog;
-      }
-      if (nextLog.get() == null) {
-        createNextLog();
-      }
+      DfsLogger next = nextLog.take();
+      log.info("Using next log " + next.getFileName());
+      currentLog = next;
       logId.incrementAndGet();
       return;
     } catch (Exception t) {
@@ -221,12 +214,11 @@ public class TabletServerLogger {
     }
   }
 
-  // callers are synchronized already
-  private void createNextLog() {
-    if (nextLogMaker.getActiveCount() == 0) {
-      nextLogMaker.submit(new Runnable() {
-        @Override
-        public void run() {
+  private void startLogMaker() {
+    nextLogMaker.submit(new Runnable() {
+      @Override
+      public void run() {
+        while (!nextLogMaker.isShutdown()) {
           try {
             log.debug("Creating next WAL");
             DfsLogger alog = new DfsLogger(tserver.getServerConfig(), syncCounter, flushCounter);
@@ -235,17 +227,15 @@ public class TabletServerLogger {
               tserver.addLoggersToMetadata(alog, tablet.getExtent());
             }
             log.debug("Created next WAL " + alog.getFileName());
-            alog = nextLog.getAndSet(alog);
-            if (alog != null) {
-              log.debug("closing unused next log: " + alog.getFileName());
-              alog.close();
+            while (!nextLog.offer(alog, 12, TimeUnit.HOURS)) {
+              log.info("Our WAL was not used for 12 hours: " + alog.getFileName());
             }
           } catch (Exception t) {
             log.error("{}", t.getMessage(), t);
           }
         }
-      });
-    }
+      }
+    });
   }
 
   public void resetLoggers() throws IOException {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/affff422/test/src/test/java/org/apache/accumulo/test/GarbageCollectWALIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/GarbageCollectWALIT.java b/test/src/test/java/org/apache/accumulo/test/GarbageCollectWALIT.java
new file mode 100644
index 0000000..0324e4a
--- /dev/null
+++ b/test/src/test/java/org/apache/accumulo/test/GarbageCollectWALIT.java
@@ -0,0 +1,81 @@
+/*
+ * 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.test;
+
+import static org.junit.Assert.assertEquals;
+
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.fate.util.UtilWaitThread;
+import org.apache.accumulo.minicluster.ServerType;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloClusterImpl;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.test.functional.ConfigurableMacIT;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RawLocalFileSystem;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.junit.Test;
+
+import com.google.common.collect.Iterators;
+
+public class GarbageCollectWALIT extends ConfigurableMacIT {
+
+  @Override
+  protected void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
+    cfg.setProperty(Property.INSTANCE_ZK_HOST, "5s");
+    cfg.setProperty(Property.GC_CYCLE_START, "1s");
+    cfg.setProperty(Property.GC_CYCLE_DELAY, "1s");
+    cfg.setNumTservers(1);
+    hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
+  }
+
+  @Test(timeout = 2 * 60 * 1000)
+  public void test() throws Exception {
+    // not yet, please
+    String tableName = getUniqueNames(1)[0];
+    cluster.getClusterControl().stop(ServerType.GARBAGE_COLLECTOR);
+    Connector c = getConnector();
+    c.tableOperations().create(tableName);
+    // count the number of WALs in the filesystem
+    assertEquals(2, countWALsInFS(cluster));
+    cluster.getClusterControl().stop(ServerType.TABLET_SERVER);
+    cluster.getClusterControl().start(ServerType.GARBAGE_COLLECTOR);
+    cluster.getClusterControl().start(ServerType.TABLET_SERVER);
+    Iterators.size(c.createScanner(MetadataTable.NAME, Authorizations.EMPTY).iterator());
+    // let GC run
+    UtilWaitThread.sleep(3 * 5 * 1000);
+    assertEquals(2, countWALsInFS(cluster));
+  }
+
+  private int countWALsInFS(MiniAccumuloClusterImpl cluster) throws Exception {
+    FileSystem fs = cluster.getFileSystem();
+    RemoteIterator<LocatedFileStatus> iterator = fs.listFiles(new Path(cluster.getConfig().getAccumuloDir() + "/wal"), true);
+    int result = 0;
+    while (iterator.hasNext()) {
+      LocatedFileStatus next = iterator.next();
+      if (!next.isDirectory()) {
+        result++;
+      }
+    }
+    return result;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/accumulo/blob/affff422/test/src/test/java/org/apache/accumulo/test/functional/WALSunnyDayIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/WALSunnyDayIT.java b/test/src/test/java/org/apache/accumulo/test/functional/WALSunnyDayIT.java
index b8e36bc..3f51c8d 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/WALSunnyDayIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/WALSunnyDayIT.java
@@ -60,6 +60,7 @@ import org.apache.hadoop.io.Text;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.ZooKeeper;
+import org.junit.Assert;
 import org.junit.Test;
 
 import com.google.common.collect.Iterators;
@@ -122,6 +123,7 @@ public class WALSunnyDayIT extends ConfigurableMacIT {
     for (String table: new String[] { tableName, MetadataTable.NAME, RootTable.NAME} ) {
       c.tableOperations().flush(table, null, null, true);
     }
+    UtilWaitThread.sleep(1000);
     // rolled WAL is no longer in use, but needs to be GC'd
     Map<String,Boolean> walsAfterflush = getWals(c, zoo);
     assertEquals(walsAfterflush.toString(), 3, walsAfterflush.size());
@@ -151,6 +153,7 @@ public class WALSunnyDayIT extends ConfigurableMacIT {
 
     // put some data in the WAL
     assertEquals(0, cluster.exec(SetGoalState.class, "NORMAL").waitFor());
+    verifySomeData(c, tableName, 1000 * 50 + 1);
     writeSomeData(c, tableName, 100, 100);
 
     Map<String,Boolean> walsAfterRestart = getWals(c, zoo);
@@ -163,6 +166,13 @@ public class WALSunnyDayIT extends ConfigurableMacIT {
     assertEquals("logs in use should be 2", 2, countTrue(walsAfterRestartAndGC.values()));
   }
 
+  private void verifySomeData(Connector c, String tableName, int expected) throws Exception {
+    Scanner scan = c.createScanner(tableName, EMPTY);
+    int result = Iterators.size(scan.iterator());
+    scan.close();
+    Assert.assertEquals(expected, result);
+  }
+
   private void writeSomeData(Connector conn, String tableName, int row, int col) throws Exception {
     Random rand = new Random();
     BatchWriter bw = conn.createBatchWriter(tableName, null);


[18/34] accumulo git commit: ACCUMULO-3423 updates from reviews

Posted by ec...@apache.org.
ACCUMULO-3423 updates from reviews


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

Branch: refs/heads/master
Commit: 9c2ca7a5cce8f7d0bacb100fed6405c86bde2a2d
Parents: affff42
Author: Eric C. Newton <er...@gmail.com>
Authored: Tue Apr 14 14:52:07 2015 -0400
Committer: Eric C. Newton <er...@gmail.com>
Committed: Tue Apr 14 14:52:07 2015 -0400

----------------------------------------------------------------------
 .../core/tabletserver/log/LogEntry.java         |  4 +++
 .../server/master/state/MetaDataStateStore.java |  3 ++
 .../accumulo/server/replication/StatusUtil.java | 12 +++++---
 .../gc/GarbageCollectWriteAheadLogs.java        | 30 ++++++++++++++------
 .../apache/accumulo/tserver/TabletServer.java   |  4 +--
 .../apache/accumulo/tserver/log/DfsLogger.java  |  1 +
 .../tserver/log/TabletServerLogger.java         | 10 +++++--
 7 files changed, 46 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/9c2ca7a5/core/src/main/java/org/apache/accumulo/core/tabletserver/log/LogEntry.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/tabletserver/log/LogEntry.java b/core/src/main/java/org/apache/accumulo/core/tabletserver/log/LogEntry.java
index 964e3b3..90ce692 100644
--- a/core/src/main/java/org/apache/accumulo/core/tabletserver/log/LogEntry.java
+++ b/core/src/main/java/org/apache/accumulo/core/tabletserver/log/LogEntry.java
@@ -81,6 +81,10 @@ public class LogEntry {
   static private final Text EMPTY_TEXT = new Text();
 
   public static LogEntry fromKeyValue(Key key, Value value) {
+    String qualifier = key.getColumnQualifier().toString();
+    if (qualifier.indexOf('/') < 1) {
+      throw new IllegalArgumentException("Bad key for log entry: " + key);
+    }
     KeyExtent extent = new KeyExtent(key.getRow(), EMPTY_TEXT);
     String[] parts = key.getColumnQualifier().toString().split("/", 2);
     String server = parts[0];

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9c2ca7a5/server/base/src/main/java/org/apache/accumulo/server/master/state/MetaDataStateStore.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/state/MetaDataStateStore.java b/server/base/src/main/java/org/apache/accumulo/server/master/state/MetaDataStateStore.java
index adcf04d..c154bd0 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/state/MetaDataStateStore.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/state/MetaDataStateStore.java
@@ -168,6 +168,9 @@ public class MetaDataStateStore extends TabletStateStore {
     BatchWriter writer = createBatchWriter();
     try {
       for (Entry<TServerInstance,List<Path>> entry : logs.entrySet()) {
+        if (entry.getValue().isEmpty()) {
+          continue;
+        }
         Mutation m = new Mutation(MetadataSchema.CurrentLogsSection.getRowPrefix() + entry.getKey().toString());
         for (Path log : entry.getValue()) {
           m.put(MetadataSchema.CurrentLogsSection.COLF, new Text(log.toString()), MetadataSchema.CurrentLogsSection.UNUSED);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9c2ca7a5/server/base/src/main/java/org/apache/accumulo/server/replication/StatusUtil.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/replication/StatusUtil.java b/server/base/src/main/java/org/apache/accumulo/server/replication/StatusUtil.java
index e973ebc..d72eea2 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/replication/StatusUtil.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/replication/StatusUtil.java
@@ -32,7 +32,6 @@ public class StatusUtil {
   private static final Value INF_END_REPLICATION_STATUS_VALUE, CLOSED_STATUS_VALUE;
 
   private static final Status.Builder CREATED_STATUS_BUILDER;
-  private static final Status.Builder INF_END_REPLICATION_STATUS_BUILDER;
 
   static {
     CREATED_STATUS_BUILDER = Status.newBuilder();
@@ -46,7 +45,6 @@ public class StatusUtil {
     builder.setEnd(0);
     builder.setInfiniteEnd(true);
     builder.setClosed(false);
-    INF_END_REPLICATION_STATUS_BUILDER = builder;
     INF_END_REPLICATION_STATUS = builder.build();
     INF_END_REPLICATION_STATUS_VALUE = ProtobufUtil.toValue(INF_END_REPLICATION_STATUS);
 
@@ -155,8 +153,14 @@ public class StatusUtil {
   /**
    * @return A {@link Status} for an open file of unspecified length, all of which needs replicating.
    */
-  public static synchronized Status openWithUnknownLength(long timeCreated) {
-    return INF_END_REPLICATION_STATUS_BUILDER.setCreatedTime(timeCreated).build();
+  public static Status openWithUnknownLength(long timeCreated) {
+    Builder builder = Status.newBuilder();
+    builder.setBegin(0);
+    builder.setEnd(0);
+    builder.setInfiniteEnd(true);
+    builder.setClosed(false);
+    builder.setCreatedTime(timeCreated);
+    return builder.build();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9c2ca7a5/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 59612ab..9f537af 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
@@ -26,6 +26,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
+import java.util.UUID;
 
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -130,9 +131,9 @@ public class GarbageCollectWriteAheadLogs {
       status.currentLog.candidates = count;
       span.stop();
 
-      span = Trace.start("removeMetadataEntries");
+      span = Trace.start("removeEntriesInUse");
       try {
-        count = removeMetadataEntries(candidates, status, currentServers);
+        count = removeEntriesInUse(candidates, status, currentServers);
       } catch (Exception ex) {
         log.error("Unable to scan metadata table", ex);
         return;
@@ -165,7 +166,7 @@ public class GarbageCollectWriteAheadLogs {
       span.stop();
 
       span = Trace.start("removeMarkers");
-      count = removeMarkers(candidates);
+      count = removeTabletServerMarkers(candidates);
       long removeMarkersStop = System.currentTimeMillis();
       log.info(String.format("%d markers removed in %.2f seconds", count, (removeMarkersStop - removeStop) / 1000.));
       span.stop();
@@ -182,7 +183,7 @@ public class GarbageCollectWriteAheadLogs {
     }
   }
 
-  private long removeMarkers(Map<TServerInstance,Set<Path>> candidates) {
+  private long removeTabletServerMarkers(Map<TServerInstance,Set<Path>> candidates) {
     long result = 0;
     try {
       BatchWriter root = null;
@@ -231,15 +232,19 @@ public class GarbageCollectWriteAheadLogs {
     return status.currentLog.deleted;
   }
 
-  private long removeMetadataEntries(Map<TServerInstance, Set<Path> > candidates, GCStatus status, Set<TServerInstance> liveServers) throws IOException, KeeperException,
+  private UUID path2uuid(Path path) {
+    return UUID.fromString(path.getName());
+  }
+
+  private long removeEntriesInUse(Map<TServerInstance, Set<Path> > candidates, GCStatus status, Set<TServerInstance> liveServers) throws IOException, KeeperException,
       InterruptedException {
 
     // remove any entries if there's a log reference, or a tablet is still assigned to the dead server
 
-    Map<Path, TServerInstance> walToDeadServer = new HashMap<>();
+    Map<UUID, TServerInstance> walToDeadServer = new HashMap<>();
     for (Entry<TServerInstance,Set<Path>> entry : candidates.entrySet()) {
       for (Path file : entry.getValue()) {
-        walToDeadServer.put(file, entry.getKey());
+        walToDeadServer.put(path2uuid(file), entry.getKey());
       }
     }
     long count = 0;
@@ -254,9 +259,16 @@ public class GarbageCollectWriteAheadLogs {
       }
       for (Collection<String> wals : state.walogs) {
         for (String wal : wals) {
-          TServerInstance dead = walToDeadServer.get(new Path(wal));
+          UUID walUUID = path2uuid(new Path(wal));
+          TServerInstance dead = walToDeadServer.get(walUUID);
           if (dead != null) {
-            candidates.get(dead).remove(wal);
+            Iterator<Path> iter = candidates.get(dead).iterator();
+            while (iter.hasNext()) {
+              if (path2uuid(iter.next()).equals(walUUID)) {
+                iter.remove();
+                break;
+              }
+            }
           }
         }
       }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9c2ca7a5/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
index e28d472..af45c14 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
@@ -2893,7 +2893,7 @@ public class TabletServer extends AccumuloServerContext implements Runnable {
   public void minorCompactionFinished(CommitSession tablet, String newDatafile, int walogSeq) throws IOException {
     totalMinorCompactions.incrementAndGet();
     logger.minorCompactionFinished(tablet, newDatafile, walogSeq);
-    removeUnusedWALs();
+    markUnusedWALs();
   }
 
   public void minorCompactionStarted(CommitSession tablet, int lastUpdateSequence, String newMapfileLocation) throws IOException {
@@ -3000,7 +3000,7 @@ public class TabletServer extends AccumuloServerContext implements Runnable {
   // remove any meta entries after a rolled log is no longer referenced
   Set<DfsLogger> closedLogs = new HashSet<>();
 
-  private void removeUnusedWALs() {
+  private void markUnusedWALs() {
     Set<DfsLogger> candidates;
     synchronized (closedLogs) {
       candidates = new HashSet<>(closedLogs);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9c2ca7a5/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
index e256604..6e9cdf9 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
@@ -383,6 +383,7 @@ public class DfsLogger implements Comparable<DfsLogger> {
 
   public synchronized void open(String address) throws IOException {
     String filename = UUID.randomUUID().toString();
+    log.debug("Address is " + address);
     String logger = Joiner.on("+").join(address.split(":"));
 
     log.debug("DfsLogger.open() begin");

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9c2ca7a5/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
index 04e7a83..dd54798 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
@@ -79,7 +79,7 @@ public class TabletServerLogger {
   // The current logger
   private DfsLogger currentLog = null;
   private final SynchronousQueue<DfsLogger> nextLog = new SynchronousQueue<>();
-  private final ThreadPoolExecutor nextLogMaker = new SimpleThreadPool(1, "WALog creator");
+  private ThreadPoolExecutor nextLogMaker;
 
   // The current generation of logs.
   // Because multiple threads can be using a log at one time, a log
@@ -150,7 +150,6 @@ public class TabletServerLogger {
     this.maxSize = maxSize;
     this.syncCounter = syncCounter;
     this.flushCounter = flushCounter;
-    startLogMaker();
   }
 
   private DfsLogger initializeLoggers(final AtomicInteger logIdOut) throws IOException {
@@ -200,6 +199,7 @@ public class TabletServerLogger {
     }
 
     try {
+      startLogMaker();
       DfsLogger next = nextLog.take();
       log.info("Using next log " + next.getFileName());
       currentLog = next;
@@ -214,7 +214,11 @@ public class TabletServerLogger {
     }
   }
 
-  private void startLogMaker() {
+  private synchronized void startLogMaker() {
+    if (nextLogMaker != null) {
+      return;
+    }
+    nextLogMaker = new SimpleThreadPool(1, "WALog creator");
     nextLogMaker.submit(new Runnable() {
       @Override
       public void run() {


[04/34] accumulo git commit: ACCUMULO-3625 use log markers against tservers, not tablets

Posted by ec...@apache.org.
ACCUMULO-3625 use log markers against tservers, not tablets


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

Branch: refs/heads/master
Commit: b2539fb1640e3a3967d4cdb21ca2d8b3fb4f0c38
Parents: aac52a8
Author: Eric C. Newton <er...@gmail.com>
Authored: Wed Feb 25 16:42:55 2015 -0500
Committer: Eric C. Newton <er...@gmail.com>
Committed: Wed Feb 25 16:42:55 2015 -0500

----------------------------------------------------------------------
 .../client/impl/ReplicationOperationsImpl.java  |   4 +-
 .../org/apache/accumulo/core/conf/Property.java |   4 +-
 .../accumulo/core/metadata/RootTable.java       |   1 +
 .../core/metadata/schema/MetadataSchema.java    |  48 ++
 .../core/tabletserver/log/LogEntry.java         |  75 +-
 .../thrift/TabletClientService.java             | 749 +------------------
 core/src/main/thrift/tabletserver.thrift        |   1 -
 .../core/metadata/MetadataTableSchemaTest.java  |  39 +
 .../ReplicationOperationsImplTest.java          |   9 +-
 .../apache/accumulo/server/fs/VolumeUtil.java   |  22 +-
 .../apache/accumulo/server/init/Initialize.java |   1 +
 .../server/master/state/MetaDataStateStore.java |  40 +-
 .../master/state/MetaDataTableScanner.java      |   7 +-
 .../master/state/TabletLocationState.java       |   8 +
 .../server/master/state/TabletStateStore.java   |  12 +-
 .../master/state/ZooTabletStateStore.java       |  33 +-
 .../accumulo/server/util/ListVolumesUsed.java   |   3 -
 .../server/util/MasterMetadataUtil.java         |  18 +-
 .../accumulo/server/util/MetadataTableUtil.java | 218 ++++--
 .../server/util/ReplicationTableUtil.java       |  13 +-
 .../server/util/ReplicationTableUtilTest.java   |   2 +-
 .../gc/GarbageCollectWriteAheadLogs.java        | 447 ++++-------
 .../accumulo/gc/SimpleGarbageCollector.java     |   1 -
 .../CloseWriteAheadLogReferences.java           |  25 +-
 .../gc/GarbageCollectWriteAheadLogsTest.java    | 568 --------------
 .../CloseWriteAheadLogReferencesTest.java       | 152 +---
 .../java/org/apache/accumulo/master/Master.java |   3 +
 .../master/MasterClientServiceHandler.java      |   4 +-
 .../accumulo/master/TabletGroupWatcher.java     |  24 +-
 .../accumulo/master/replication/WorkMaker.java  |   1 +
 .../accumulo/master/state/MergeStats.java       |   3 +-
 .../apache/accumulo/master/TestMergeState.java  |   2 +-
 .../master/state/RootTabletStateStoreTest.java  |   4 +-
 .../server/GarbageCollectionLogger.java         |   5 +-
 .../apache/accumulo/tserver/TabletLevel.java    |  34 +
 .../apache/accumulo/tserver/TabletServer.java   | 166 ++--
 .../apache/accumulo/tserver/log/DfsLogger.java  |   7 +-
 .../accumulo/tserver/log/SortedLogRecovery.java |   8 +-
 .../tserver/log/TabletServerLogger.java         | 183 +++--
 .../accumulo/tserver/tablet/CommitSession.java  |   3 +-
 .../tserver/tablet/DatafileManager.java         |   4 +-
 .../apache/accumulo/tserver/tablet/Tablet.java  |  58 +-
 .../tserver/tablet/TabletCommitter.java         |   3 +-
 .../test/performance/thrift/NullTserver.java    |   3 -
 .../accumulo/proxy/ProxyDurabilityIT.java       |   5 +
 .../test/BadDeleteMarkersCreatedIT.java         |   2 +-
 .../org/apache/accumulo/test/BalanceIT.java     |  20 +-
 .../org/apache/accumulo/test/CleanWalIT.java    |   1 +
 .../accumulo/test/ConditionalWriterIT.java      |   1 +
 .../MissingWalHeaderCompletesRecoveryIT.java    |  14 +-
 .../accumulo/test/NoMutationRecoveryIT.java     | 132 ----
 .../org/apache/accumulo/test/ShellServerIT.java |   2 +-
 .../accumulo/test/functional/ExamplesIT.java    |   6 +-
 .../test/functional/WatchTheWatchCountIT.java   |   4 +-
 .../test/performance/RollWALPerformanceIT.java  | 126 ++++
 ...bageCollectorCommunicatesWithTServersIT.java |  35 +-
 .../replication/MultiInstanceReplicationIT.java |   2 +-
 .../test/replication/ReplicationIT.java         | 370 +++------
 58 files changed, 1149 insertions(+), 2586 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/b2539fb1/core/src/main/java/org/apache/accumulo/core/client/impl/ReplicationOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ReplicationOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ReplicationOperationsImpl.java
index 6fdf4db..bcdadff 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ReplicationOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ReplicationOperationsImpl.java
@@ -228,9 +228,7 @@ public class ReplicationOperationsImpl implements ReplicationOperations {
     try {
       for (Entry<Key,Value> entry : metaBs) {
         LogEntry logEntry = LogEntry.fromKeyValue(entry.getKey(), entry.getValue());
-        for (String log : logEntry.logSet) {
-          wals.add(new Path(log).toString());
-        }
+        wals.add(new Path(logEntry.filename).toString());
       }
     } finally {
       metaBs.close();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b2539fb1/core/src/main/java/org/apache/accumulo/core/conf/Property.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/Property.java b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
index 01f03cf..5021d48 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/Property.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
@@ -340,8 +340,8 @@ public enum Property {
       + "no longer in use are removed from the filesystem."),
   GC_PORT("gc.port.client", "50091", PropertyType.PORT, "The listening port for the garbage collector's monitor service"),
   GC_DELETE_THREADS("gc.threads.delete", "16", PropertyType.COUNT, "The number of threads used to delete files"),
-  GC_TRASH_IGNORE("gc.trash.ignore", "false", PropertyType.BOOLEAN, "Do not use the Trash, even if it is configured"),
-  GC_FILE_ARCHIVE("gc.file.archive", "false", PropertyType.BOOLEAN, "Archive any files/directories instead of moving to the HDFS trash or deleting"),
+  GC_TRASH_IGNORE("gc.trash.ignore", "false", PropertyType.BOOLEAN, "Do not use the Trash, even if it is configured."),
+  GC_FILE_ARCHIVE("gc.file.archive", "false", PropertyType.BOOLEAN, "Archive any files/directories instead of moving to the HDFS trash or deleting."),
 
   // properties that are specific to the monitor server behavior
   MONITOR_PREFIX("monitor.", null, PropertyType.PREFIX, "Properties in this category affect the behavior of the monitor web server."),

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b2539fb1/core/src/main/java/org/apache/accumulo/core/metadata/RootTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/RootTable.java b/core/src/main/java/org/apache/accumulo/core/metadata/RootTable.java
index 24148b1..03a2995 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/RootTable.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/RootTable.java
@@ -41,6 +41,7 @@ public class RootTable {
   public static final String ZROOT_TABLET_FUTURE_LOCATION = ZROOT_TABLET + "/future_location";
   public static final String ZROOT_TABLET_LAST_LOCATION = ZROOT_TABLET + "/lastlocation";
   public static final String ZROOT_TABLET_WALOGS = ZROOT_TABLET + "/walogs";
+  public static final String ZROOT_TABLET_CURRENT_LOGS = ZROOT_TABLET + "/current_logs";
   public static final String ZROOT_TABLET_PATH = ZROOT_TABLET + "/dir";
 
   public static final KeyExtent EXTENT = new KeyExtent(new Text(ID), null, null);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b2539fb1/core/src/main/java/org/apache/accumulo/core/metadata/schema/MetadataSchema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/schema/MetadataSchema.java b/core/src/main/java/org/apache/accumulo/core/metadata/schema/MetadataSchema.java
index 534dd7f..88e11f4 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/schema/MetadataSchema.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/schema/MetadataSchema.java
@@ -16,11 +16,14 @@
  */
 package org.apache.accumulo.core.metadata.schema;
 
+import static java.nio.charset.StandardCharsets.UTF_8;
+
 import org.apache.accumulo.core.client.admin.TimeType;
 import org.apache.accumulo.core.data.ArrayByteSequence;
 import org.apache.accumulo.core.data.Key;
 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.schema.Section;
 import org.apache.accumulo.core.util.ColumnFQ;
 import org.apache.hadoop.io.Text;
@@ -292,4 +295,49 @@ public class MetadataSchema {
       buff.set(buff.getBytes(), section.getRowPrefix().length(), buff.getLength() - section.getRowPrefix().length());
     }
   }
+
+  /**
+   * Holds references to the WALs in use in a live Tablet Server.
+   * <p>
+   * <code>~wal+tserver:port[sessionId] log:hdfs://localhost:8020/accumulo/wal/tserver+port/WAL  [] -></code>
+   */
+  public static class CurrentLogsSection {
+    private static final Section section = new Section(RESERVED_PREFIX + "wal+", true, RESERVED_PREFIX + "wal,", false);
+    private static byte LEFT_BRACKET = (byte)'[';
+    public static final Text COLF = new Text("log");
+    public static final Value UNUSED = new Value("unused".getBytes(UTF_8));
+
+    public static Range getRange() {
+      return section.getRange();
+    }
+
+    public static String getRowPrefix() {
+      return section.getRowPrefix();
+    }
+
+    public static void getTabletServer(Key k, Text hostPort, Text session) {
+      Preconditions.checkNotNull(k);
+      Preconditions.checkNotNull(hostPort);
+      Preconditions.checkNotNull(session);
+
+      Text row = new Text();
+      k.getRow(row);
+      if (row.getLength() < section.getRowPrefix().length()) {
+        throw new IllegalArgumentException("Bad key " + k.toString());
+      }
+      for (int sessionStart = section.getRowPrefix().length(); sessionStart < row.getLength() - 1; sessionStart++) {
+        if (row.charAt(sessionStart) == LEFT_BRACKET) {
+          hostPort.set(row.getBytes(), section.getRowPrefix().length(), sessionStart - section.getRowPrefix().length());
+          session.set(row.getBytes(), sessionStart + 1, row.getLength() - sessionStart - 2);
+          return;
+        }
+      }
+      throw new IllegalArgumentException("Bad key " + k.toString());
+    }
+
+    public static void getPath(Key k, Text path) {
+      k.getColumnQualifier(path);
+    }
+
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b2539fb1/core/src/main/java/org/apache/accumulo/core/tabletserver/log/LogEntry.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/tabletserver/log/LogEntry.java b/core/src/main/java/org/apache/accumulo/core/tabletserver/log/LogEntry.java
index 25d0f32..964e3b3 100644
--- a/core/src/main/java/org/apache/accumulo/core/tabletserver/log/LogEntry.java
+++ b/core/src/main/java/org/apache/accumulo/core/tabletserver/log/LogEntry.java
@@ -16,10 +16,10 @@
  */
 package org.apache.accumulo.core.tabletserver.log;
 
+import static java.nio.charset.StandardCharsets.UTF_8;
+
 import java.io.IOException;
-import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collection;
 
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.KeyExtent;
@@ -29,29 +29,29 @@ import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.io.Text;
 
-import com.google.common.base.Joiner;
-
 public class LogEntry {
-  public KeyExtent extent;
-  public long timestamp;
-  public String server;
-  public String filename;
-  public int tabletId;
-  public Collection<String> logSet;
-
-  public LogEntry() {}
+  public final KeyExtent extent;
+  public final long timestamp;
+  public final String server;
+  public final String filename;
 
   public LogEntry(LogEntry le) {
     this.extent = le.extent;
     this.timestamp = le.timestamp;
     this.server = le.server;
     this.filename = le.filename;
-    this.tabletId = le.tabletId;
-    this.logSet = new ArrayList<String>(le.logSet);
   }
 
+  public LogEntry(KeyExtent extent, long timestamp, String server, String filename) {
+    this.extent = extent;
+    this.timestamp = timestamp;
+    this.server = server;
+    this.filename = filename;
+  }
+
+  @Override
   public String toString() {
-    return extent.toString() + " " + filename + " (" + tabletId + ")";
+    return extent.toString() + " " + filename;
   }
 
   public String getName() {
@@ -64,43 +64,31 @@ public class LogEntry {
     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 {
+  static public LogEntry fromBytes(byte bytes[]) throws IOException {
     DataInputBuffer inp = new DataInputBuffer();
     inp.reset(bytes, bytes.length);
-    extent = new KeyExtent();
+    KeyExtent 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;
+    long timestamp = inp.readLong();
+    String server = inp.readUTF();
+    String filename = inp.readUTF();
+    return new LogEntry(extent, timestamp, server, filename);
   }
 
   static private final Text EMPTY_TEXT = new Text();
 
   public static LogEntry fromKeyValue(Key key, Value value) {
-    LogEntry result = new LogEntry();
-    result.extent = new KeyExtent(key.getRow(), EMPTY_TEXT);
+    KeyExtent extent = new KeyExtent(key.getRow(), EMPTY_TEXT);
     String[] parts = key.getColumnQualifier().toString().split("/", 2);
-    result.server = parts[0];
-    result.filename = parts[1];
-    parts = value.toString().split("\\|");
-    result.tabletId = Integer.parseInt(parts[1]);
-    result.logSet = Arrays.asList(parts[0].split(";"));
-    result.timestamp = key.getTimestamp();
-    return result;
+    String server = parts[0];
+    // handle old-style log entries that specify log sets
+    parts = value.toString().split("\\|")[0].split(";");
+    String filename = parts[parts.length - 1];
+    long timestamp = key.getTimestamp();
+    return new LogEntry(extent, timestamp, server, filename);
   }
 
   public Text getRow() {
@@ -111,11 +99,16 @@ public class LogEntry {
     return MetadataSchema.TabletsSection.LogColumnFamily.NAME;
   }
 
+  public String getUniqueID() {
+    String parts[] = filename.split("/");
+    return parts[parts.length - 1];
+  }
+
   public Text getColumnQualifier() {
     return new Text(server + "/" + filename);
   }
 
   public Value getValue() {
-    return new Value((Joiner.on(";").join(logSet) + "|" + tabletId).getBytes());
+    return new Value(filename.getBytes(UTF_8));
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b2539fb1/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/TabletClientService.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/TabletClientService.java b/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/TabletClientService.java
index 02bd4e1..d6d4afd 100644
--- a/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/TabletClientService.java
+++ b/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/TabletClientService.java
@@ -110,8 +110,6 @@ import org.slf4j.LoggerFactory;
 
     public List<ActiveCompaction> getActiveCompactions(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException;
 
-    public void removeLogs(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, List<String> filenames) throws org.apache.thrift.TException;
-
     public List<String> getActiveLogs(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials) throws org.apache.thrift.TException;
 
   }
@@ -176,8 +174,6 @@ import org.slf4j.LoggerFactory;
 
     public void getActiveCompactions(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
-    public void removeLogs(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, List<String> filenames, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
-
     public void getActiveLogs(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
   }
@@ -900,20 +896,6 @@ import org.slf4j.LoggerFactory;
       throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getActiveCompactions failed: unknown result");
     }
 
-    public void removeLogs(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, List<String> filenames) throws org.apache.thrift.TException
-    {
-      send_removeLogs(tinfo, credentials, filenames);
-    }
-
-    public void send_removeLogs(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, List<String> filenames) throws org.apache.thrift.TException
-    {
-      removeLogs_args args = new removeLogs_args();
-      args.setTinfo(tinfo);
-      args.setCredentials(credentials);
-      args.setFilenames(filenames);
-      sendBase("removeLogs", args);
-    }
-
     public List<String> getActiveLogs(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials) throws org.apache.thrift.TException
     {
       send_getActiveLogs(tinfo, credentials);
@@ -2117,43 +2099,6 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    public void removeLogs(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, List<String> filenames, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
-      checkReady();
-      removeLogs_call method_call = new removeLogs_call(tinfo, credentials, filenames, resultHandler, this, ___protocolFactory, ___transport);
-      this.___currentMethod = method_call;
-      ___manager.call(method_call);
-    }
-
-    public static class removeLogs_call extends org.apache.thrift.async.TAsyncMethodCall {
-      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
-      private org.apache.accumulo.core.security.thrift.TCredentials credentials;
-      private List<String> filenames;
-      public removeLogs_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, List<String> filenames, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
-        super(client, protocolFactory, transport, resultHandler, true);
-        this.tinfo = tinfo;
-        this.credentials = credentials;
-        this.filenames = filenames;
-      }
-
-      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
-        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("removeLogs", org.apache.thrift.protocol.TMessageType.CALL, 0));
-        removeLogs_args args = new removeLogs_args();
-        args.setTinfo(tinfo);
-        args.setCredentials(credentials);
-        args.setFilenames(filenames);
-        args.write(prot);
-        prot.writeMessageEnd();
-      }
-
-      public void getResult() throws org.apache.thrift.TException {
-        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
-          throw new IllegalStateException("Method call not finished!");
-        }
-        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
-        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
-      }
-    }
-
     public void getActiveLogs(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
       checkReady();
       getActiveLogs_call method_call = new getActiveLogs_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport);
@@ -2231,7 +2176,6 @@ import org.slf4j.LoggerFactory;
       processMap.put("fastHalt", new fastHalt());
       processMap.put("getActiveScans", new getActiveScans());
       processMap.put("getActiveCompactions", new getActiveCompactions());
-      processMap.put("removeLogs", new removeLogs());
       processMap.put("getActiveLogs", new getActiveLogs());
       return processMap;
     }
@@ -2893,25 +2837,6 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    public static class removeLogs<I extends Iface> extends org.apache.thrift.ProcessFunction<I, removeLogs_args> {
-      public removeLogs() {
-        super("removeLogs");
-      }
-
-      public removeLogs_args getEmptyArgsInstance() {
-        return new removeLogs_args();
-      }
-
-      protected boolean isOneway() {
-        return true;
-      }
-
-      public org.apache.thrift.TBase getResult(I iface, removeLogs_args args) throws org.apache.thrift.TException {
-        iface.removeLogs(args.tinfo, args.credentials, args.filenames);
-        return null;
-      }
-    }
-
     public static class getActiveLogs<I extends Iface> extends org.apache.thrift.ProcessFunction<I, getActiveLogs_args> {
       public getActiveLogs() {
         super("getActiveLogs");
@@ -2974,7 +2899,6 @@ import org.slf4j.LoggerFactory;
       processMap.put("fastHalt", new fastHalt());
       processMap.put("getActiveScans", new getActiveScans());
       processMap.put("getActiveCompactions", new getActiveCompactions());
-      processMap.put("removeLogs", new removeLogs());
       processMap.put("getActiveLogs", new getActiveLogs());
       return processMap;
     }
@@ -4367,34 +4291,6 @@ import org.slf4j.LoggerFactory;
       }
     }
 
-    public static class removeLogs<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, removeLogs_args, Void> {
-      public removeLogs() {
-        super("removeLogs");
-      }
-
-      public removeLogs_args getEmptyArgsInstance() {
-        return new removeLogs_args();
-      }
-
-      public AsyncMethodCallback<Void> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
-        final org.apache.thrift.AsyncProcessFunction fcall = this;
-        return new AsyncMethodCallback<Void>() { 
-          public void onComplete(Void o) {
-          }
-          public void onError(Exception e) {
-          }
-        };
-      }
-
-      protected boolean isOneway() {
-        return true;
-      }
-
-      public void start(I iface, removeLogs_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
-        iface.removeLogs(args.tinfo, args.credentials, args.filenames,resultHandler);
-      }
-    }
-
     public static class getActiveLogs<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getActiveLogs_args, List<String>> {
       public getActiveLogs() {
         super("getActiveLogs");
@@ -33173,619 +33069,6 @@ import org.slf4j.LoggerFactory;
 
   }
 
-  public static class removeLogs_args implements org.apache.thrift.TBase<removeLogs_args, removeLogs_args._Fields>, java.io.Serializable, Cloneable, Comparable<removeLogs_args>   {
-    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("removeLogs_args");
-
-    private static final org.apache.thrift.protocol.TField TINFO_FIELD_DESC = new org.apache.thrift.protocol.TField("tinfo", org.apache.thrift.protocol.TType.STRUCT, (short)1);
-    private static final org.apache.thrift.protocol.TField CREDENTIALS_FIELD_DESC = new org.apache.thrift.protocol.TField("credentials", org.apache.thrift.protocol.TType.STRUCT, (short)2);
-    private static final org.apache.thrift.protocol.TField FILENAMES_FIELD_DESC = new org.apache.thrift.protocol.TField("filenames", org.apache.thrift.protocol.TType.LIST, (short)3);
-
-    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
-    static {
-      schemes.put(StandardScheme.class, new removeLogs_argsStandardSchemeFactory());
-      schemes.put(TupleScheme.class, new removeLogs_argsTupleSchemeFactory());
-    }
-
-    public org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
-    public org.apache.accumulo.core.security.thrift.TCredentials credentials; // required
-    public List<String> filenames; // required
-
-    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
-    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
-      TINFO((short)1, "tinfo"),
-      CREDENTIALS((short)2, "credentials"),
-      FILENAMES((short)3, "filenames");
-
-      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
-
-      static {
-        for (_Fields field : EnumSet.allOf(_Fields.class)) {
-          byName.put(field.getFieldName(), field);
-        }
-      }
-
-      /**
-       * Find the _Fields constant that matches fieldId, or null if its not found.
-       */
-      public static _Fields findByThriftId(int fieldId) {
-        switch(fieldId) {
-          case 1: // TINFO
-            return TINFO;
-          case 2: // CREDENTIALS
-            return CREDENTIALS;
-          case 3: // FILENAMES
-            return FILENAMES;
-          default:
-            return null;
-        }
-      }
-
-      /**
-       * Find the _Fields constant that matches fieldId, throwing an exception
-       * if it is not found.
-       */
-      public static _Fields findByThriftIdOrThrow(int fieldId) {
-        _Fields fields = findByThriftId(fieldId);
-        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
-        return fields;
-      }
-
-      /**
-       * Find the _Fields constant that matches name, or null if its not found.
-       */
-      public static _Fields findByName(String name) {
-        return byName.get(name);
-      }
-
-      private final short _thriftId;
-      private final String _fieldName;
-
-      _Fields(short thriftId, String fieldName) {
-        _thriftId = thriftId;
-        _fieldName = fieldName;
-      }
-
-      public short getThriftFieldId() {
-        return _thriftId;
-      }
-
-      public String getFieldName() {
-        return _fieldName;
-      }
-    }
-
-    // isset id assignments
-    public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
-    static {
-      Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
-      tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
-      tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.security.thrift.TCredentials.class)));
-      tmpMap.put(_Fields.FILENAMES, new org.apache.thrift.meta_data.FieldMetaData("filenames", org.apache.thrift.TFieldRequirementType.DEFAULT, 
-          new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
-              new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
-      metaDataMap = Collections.unmodifiableMap(tmpMap);
-      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(removeLogs_args.class, metaDataMap);
-    }
-
-    public removeLogs_args() {
-    }
-
-    public removeLogs_args(
-      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
-      org.apache.accumulo.core.security.thrift.TCredentials credentials,
-      List<String> filenames)
-    {
-      this();
-      this.tinfo = tinfo;
-      this.credentials = credentials;
-      this.filenames = filenames;
-    }
-
-    /**
-     * Performs a deep copy on <i>other</i>.
-     */
-    public removeLogs_args(removeLogs_args other) {
-      if (other.isSetTinfo()) {
-        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
-      }
-      if (other.isSetCredentials()) {
-        this.credentials = new org.apache.accumulo.core.security.thrift.TCredentials(other.credentials);
-      }
-      if (other.isSetFilenames()) {
-        List<String> __this__filenames = new ArrayList<String>(other.filenames);
-        this.filenames = __this__filenames;
-      }
-    }
-
-    public removeLogs_args deepCopy() {
-      return new removeLogs_args(this);
-    }
-
-    @Override
-    public void clear() {
-      this.tinfo = null;
-      this.credentials = null;
-      this.filenames = null;
-    }
-
-    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
-      return this.tinfo;
-    }
-
-    public removeLogs_args setTinfo(org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
-      this.tinfo = tinfo;
-      return this;
-    }
-
-    public void unsetTinfo() {
-      this.tinfo = null;
-    }
-
-    /** Returns true if field tinfo is set (has been assigned a value) and false otherwise */
-    public boolean isSetTinfo() {
-      return this.tinfo != null;
-    }
-
-    public void setTinfoIsSet(boolean value) {
-      if (!value) {
-        this.tinfo = null;
-      }
-    }
-
-    public org.apache.accumulo.core.security.thrift.TCredentials getCredentials() {
-      return this.credentials;
-    }
-
-    public removeLogs_args setCredentials(org.apache.accumulo.core.security.thrift.TCredentials credentials) {
-      this.credentials = credentials;
-      return this;
-    }
-
-    public void unsetCredentials() {
-      this.credentials = null;
-    }
-
-    /** Returns true if field credentials is set (has been assigned a value) and false otherwise */
-    public boolean isSetCredentials() {
-      return this.credentials != null;
-    }
-
-    public void setCredentialsIsSet(boolean value) {
-      if (!value) {
-        this.credentials = null;
-      }
-    }
-
-    public int getFilenamesSize() {
-      return (this.filenames == null) ? 0 : this.filenames.size();
-    }
-
-    public java.util.Iterator<String> getFilenamesIterator() {
-      return (this.filenames == null) ? null : this.filenames.iterator();
-    }
-
-    public void addToFilenames(String elem) {
-      if (this.filenames == null) {
-        this.filenames = new ArrayList<String>();
-      }
-      this.filenames.add(elem);
-    }
-
-    public List<String> getFilenames() {
-      return this.filenames;
-    }
-
-    public removeLogs_args setFilenames(List<String> filenames) {
-      this.filenames = filenames;
-      return this;
-    }
-
-    public void unsetFilenames() {
-      this.filenames = null;
-    }
-
-    /** Returns true if field filenames is set (has been assigned a value) and false otherwise */
-    public boolean isSetFilenames() {
-      return this.filenames != null;
-    }
-
-    public void setFilenamesIsSet(boolean value) {
-      if (!value) {
-        this.filenames = null;
-      }
-    }
-
-    public void setFieldValue(_Fields field, Object value) {
-      switch (field) {
-      case TINFO:
-        if (value == null) {
-          unsetTinfo();
-        } else {
-          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
-        }
-        break;
-
-      case CREDENTIALS:
-        if (value == null) {
-          unsetCredentials();
-        } else {
-          setCredentials((org.apache.accumulo.core.security.thrift.TCredentials)value);
-        }
-        break;
-
-      case FILENAMES:
-        if (value == null) {
-          unsetFilenames();
-        } else {
-          setFilenames((List<String>)value);
-        }
-        break;
-
-      }
-    }
-
-    public Object getFieldValue(_Fields field) {
-      switch (field) {
-      case TINFO:
-        return getTinfo();
-
-      case CREDENTIALS:
-        return getCredentials();
-
-      case FILENAMES:
-        return getFilenames();
-
-      }
-      throw new IllegalStateException();
-    }
-
-    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
-    public boolean isSet(_Fields field) {
-      if (field == null) {
-        throw new IllegalArgumentException();
-      }
-
-      switch (field) {
-      case TINFO:
-        return isSetTinfo();
-      case CREDENTIALS:
-        return isSetCredentials();
-      case FILENAMES:
-        return isSetFilenames();
-      }
-      throw new IllegalStateException();
-    }
-
-    @Override
-    public boolean equals(Object that) {
-      if (that == null)
-        return false;
-      if (that instanceof removeLogs_args)
-        return this.equals((removeLogs_args)that);
-      return false;
-    }
-
-    public boolean equals(removeLogs_args that) {
-      if (that == null)
-        return false;
-
-      boolean this_present_tinfo = true && this.isSetTinfo();
-      boolean that_present_tinfo = true && that.isSetTinfo();
-      if (this_present_tinfo || that_present_tinfo) {
-        if (!(this_present_tinfo && that_present_tinfo))
-          return false;
-        if (!this.tinfo.equals(that.tinfo))
-          return false;
-      }
-
-      boolean this_present_credentials = true && this.isSetCredentials();
-      boolean that_present_credentials = true && that.isSetCredentials();
-      if (this_present_credentials || that_present_credentials) {
-        if (!(this_present_credentials && that_present_credentials))
-          return false;
-        if (!this.credentials.equals(that.credentials))
-          return false;
-      }
-
-      boolean this_present_filenames = true && this.isSetFilenames();
-      boolean that_present_filenames = true && that.isSetFilenames();
-      if (this_present_filenames || that_present_filenames) {
-        if (!(this_present_filenames && that_present_filenames))
-          return false;
-        if (!this.filenames.equals(that.filenames))
-          return false;
-      }
-
-      return true;
-    }
-
-    @Override
-    public int hashCode() {
-      return 0;
-    }
-
-    @Override
-    public int compareTo(removeLogs_args other) {
-      if (!getClass().equals(other.getClass())) {
-        return getClass().getName().compareTo(other.getClass().getName());
-      }
-
-      int lastComparison = 0;
-
-      lastComparison = Boolean.valueOf(isSetTinfo()).compareTo(other.isSetTinfo());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetTinfo()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tinfo, other.tinfo);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      lastComparison = Boolean.valueOf(isSetCredentials()).compareTo(other.isSetCredentials());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetCredentials()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.credentials, other.credentials);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      lastComparison = Boolean.valueOf(isSetFilenames()).compareTo(other.isSetFilenames());
-      if (lastComparison != 0) {
-        return lastComparison;
-      }
-      if (isSetFilenames()) {
-        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.filenames, other.filenames);
-        if (lastComparison != 0) {
-          return lastComparison;
-        }
-      }
-      return 0;
-    }
-
-    public _Fields fieldForId(int fieldId) {
-      return _Fields.findByThriftId(fieldId);
-    }
-
-    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
-      schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
-    }
-
-    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
-      schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
-    }
-
-    @Override
-    public String toString() {
-      StringBuilder sb = new StringBuilder("removeLogs_args(");
-      boolean first = true;
-
-      sb.append("tinfo:");
-      if (this.tinfo == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.tinfo);
-      }
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("credentials:");
-      if (this.credentials == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.credentials);
-      }
-      first = false;
-      if (!first) sb.append(", ");
-      sb.append("filenames:");
-      if (this.filenames == null) {
-        sb.append("null");
-      } else {
-        sb.append(this.filenames);
-      }
-      first = false;
-      sb.append(")");
-      return sb.toString();
-    }
-
-    public void validate() throws org.apache.thrift.TException {
-      // check for required fields
-      // check for sub-struct validity
-      if (tinfo != null) {
-        tinfo.validate();
-      }
-      if (credentials != null) {
-        credentials.validate();
-      }
-    }
-
-    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
-      try {
-        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
-      } catch (org.apache.thrift.TException te) {
-        throw new java.io.IOException(te);
-      }
-    }
-
-    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
-      try {
-        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
-      } catch (org.apache.thrift.TException te) {
-        throw new java.io.IOException(te);
-      }
-    }
-
-    private static class removeLogs_argsStandardSchemeFactory implements SchemeFactory {
-      public removeLogs_argsStandardScheme getScheme() {
-        return new removeLogs_argsStandardScheme();
-      }
-    }
-
-    private static class removeLogs_argsStandardScheme extends StandardScheme<removeLogs_args> {
-
-      public void read(org.apache.thrift.protocol.TProtocol iprot, removeLogs_args struct) throws org.apache.thrift.TException {
-        org.apache.thrift.protocol.TField schemeField;
-        iprot.readStructBegin();
-        while (true)
-        {
-          schemeField = iprot.readFieldBegin();
-          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
-            break;
-          }
-          switch (schemeField.id) {
-            case 1: // TINFO
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
-                struct.tinfo.read(iprot);
-                struct.setTinfoIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 2: // CREDENTIALS
-              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
-                struct.credentials = new org.apache.accumulo.core.security.thrift.TCredentials();
-                struct.credentials.read(iprot);
-                struct.setCredentialsIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            case 3: // FILENAMES
-              if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
-                {
-                  org.apache.thrift.protocol.TList _list314 = iprot.readListBegin();
-                  struct.filenames = new ArrayList<String>(_list314.size);
-                  for (int _i315 = 0; _i315 < _list314.size; ++_i315)
-                  {
-                    String _elem316;
-                    _elem316 = iprot.readString();
-                    struct.filenames.add(_elem316);
-                  }
-                  iprot.readListEnd();
-                }
-                struct.setFilenamesIsSet(true);
-              } else { 
-                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-              }
-              break;
-            default:
-              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
-          }
-          iprot.readFieldEnd();
-        }
-        iprot.readStructEnd();
-
-        // check for required fields of primitive type, which can't be checked in the validate method
-        struct.validate();
-      }
-
-      public void write(org.apache.thrift.protocol.TProtocol oprot, removeLogs_args struct) throws org.apache.thrift.TException {
-        struct.validate();
-
-        oprot.writeStructBegin(STRUCT_DESC);
-        if (struct.tinfo != null) {
-          oprot.writeFieldBegin(TINFO_FIELD_DESC);
-          struct.tinfo.write(oprot);
-          oprot.writeFieldEnd();
-        }
-        if (struct.credentials != null) {
-          oprot.writeFieldBegin(CREDENTIALS_FIELD_DESC);
-          struct.credentials.write(oprot);
-          oprot.writeFieldEnd();
-        }
-        if (struct.filenames != null) {
-          oprot.writeFieldBegin(FILENAMES_FIELD_DESC);
-          {
-            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.filenames.size()));
-            for (String _iter317 : struct.filenames)
-            {
-              oprot.writeString(_iter317);
-            }
-            oprot.writeListEnd();
-          }
-          oprot.writeFieldEnd();
-        }
-        oprot.writeFieldStop();
-        oprot.writeStructEnd();
-      }
-
-    }
-
-    private static class removeLogs_argsTupleSchemeFactory implements SchemeFactory {
-      public removeLogs_argsTupleScheme getScheme() {
-        return new removeLogs_argsTupleScheme();
-      }
-    }
-
-    private static class removeLogs_argsTupleScheme extends TupleScheme<removeLogs_args> {
-
-      @Override
-      public void write(org.apache.thrift.protocol.TProtocol prot, removeLogs_args struct) throws org.apache.thrift.TException {
-        TTupleProtocol oprot = (TTupleProtocol) prot;
-        BitSet optionals = new BitSet();
-        if (struct.isSetTinfo()) {
-          optionals.set(0);
-        }
-        if (struct.isSetCredentials()) {
-          optionals.set(1);
-        }
-        if (struct.isSetFilenames()) {
-          optionals.set(2);
-        }
-        oprot.writeBitSet(optionals, 3);
-        if (struct.isSetTinfo()) {
-          struct.tinfo.write(oprot);
-        }
-        if (struct.isSetCredentials()) {
-          struct.credentials.write(oprot);
-        }
-        if (struct.isSetFilenames()) {
-          {
-            oprot.writeI32(struct.filenames.size());
-            for (String _iter318 : struct.filenames)
-            {
-              oprot.writeString(_iter318);
-            }
-          }
-        }
-      }
-
-      @Override
-      public void read(org.apache.thrift.protocol.TProtocol prot, removeLogs_args struct) throws org.apache.thrift.TException {
-        TTupleProtocol iprot = (TTupleProtocol) prot;
-        BitSet incoming = iprot.readBitSet(3);
-        if (incoming.get(0)) {
-          struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
-          struct.tinfo.read(iprot);
-          struct.setTinfoIsSet(true);
-        }
-        if (incoming.get(1)) {
-          struct.credentials = new org.apache.accumulo.core.security.thrift.TCredentials();
-          struct.credentials.read(iprot);
-          struct.setCredentialsIsSet(true);
-        }
-        if (incoming.get(2)) {
-          {
-            org.apache.thrift.protocol.TList _list319 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.filenames = new ArrayList<String>(_list319.size);
-            for (int _i320 = 0; _i320 < _list319.size; ++_i320)
-            {
-              String _elem321;
-              _elem321 = iprot.readString();
-              struct.filenames.add(_elem321);
-            }
-          }
-          struct.setFilenamesIsSet(true);
-        }
-      }
-    }
-
-  }
-
   public static class getActiveLogs_args implements org.apache.thrift.TBase<getActiveLogs_args, getActiveLogs_args._Fields>, java.io.Serializable, Cloneable, Comparable<getActiveLogs_args>   {
     private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getActiveLogs_args");
 
@@ -34556,13 +33839,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list322 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list322.size);
-                  for (int _i323 = 0; _i323 < _list322.size; ++_i323)
+                  org.apache.thrift.protocol.TList _list314 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list314.size);
+                  for (int _i315 = 0; _i315 < _list314.size; ++_i315)
                   {
-                    String _elem324;
-                    _elem324 = iprot.readString();
-                    struct.success.add(_elem324);
+                    String _elem316;
+                    _elem316 = iprot.readString();
+                    struct.success.add(_elem316);
                   }
                   iprot.readListEnd();
                 }
@@ -34590,9 +33873,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter325 : struct.success)
+            for (String _iter317 : struct.success)
             {
-              oprot.writeString(_iter325);
+              oprot.writeString(_iter317);
             }
             oprot.writeListEnd();
           }
@@ -34623,9 +33906,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter326 : struct.success)
+            for (String _iter318 : struct.success)
             {
-              oprot.writeString(_iter326);
+              oprot.writeString(_iter318);
             }
           }
         }
@@ -34637,13 +33920,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list327 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list327.size);
-            for (int _i328 = 0; _i328 < _list327.size; ++_i328)
+            org.apache.thrift.protocol.TList _list319 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list319.size);
+            for (int _i320 = 0; _i320 < _list319.size; ++_i320)
             {
-              String _elem329;
-              _elem329 = iprot.readString();
-              struct.success.add(_elem329);
+              String _elem321;
+              _elem321 = iprot.readString();
+              struct.success.add(_elem321);
             }
           }
           struct.setSuccessIsSet(true);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b2539fb1/core/src/main/thrift/tabletserver.thrift
----------------------------------------------------------------------
diff --git a/core/src/main/thrift/tabletserver.thrift b/core/src/main/thrift/tabletserver.thrift
index 4a31036..f25a08f 100644
--- a/core/src/main/thrift/tabletserver.thrift
+++ b/core/src/main/thrift/tabletserver.thrift
@@ -205,7 +205,6 @@ service TabletClientService extends client.ClientService {
   
   list<ActiveScan> getActiveScans(2:trace.TInfo tinfo, 1:security.TCredentials credentials) throws (1:client.ThriftSecurityException sec)
   list<ActiveCompaction> getActiveCompactions(2:trace.TInfo tinfo, 1:security.TCredentials credentials) throws (1:client.ThriftSecurityException sec)
-  oneway void removeLogs(1:trace.TInfo tinfo, 2:security.TCredentials credentials, 3:list<string> filenames)
   list<string> getActiveLogs(1:trace.TInfo tinfo, 2:security.TCredentials credentials)
 }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b2539fb1/core/src/test/java/org/apache/accumulo/core/metadata/MetadataTableSchemaTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/metadata/MetadataTableSchemaTest.java b/core/src/test/java/org/apache/accumulo/core/metadata/MetadataTableSchemaTest.java
new file mode 100644
index 0000000..dfc74cf
--- /dev/null
+++ b/core/src/test/java/org/apache/accumulo/core/metadata/MetadataTableSchemaTest.java
@@ -0,0 +1,39 @@
+/*
+ * 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.metadata;
+
+import static org.junit.Assert.assertEquals;
+
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.CurrentLogsSection;
+import org.apache.hadoop.io.Text;
+import org.junit.Test;
+
+public class MetadataTableSchemaTest {
+
+  @Test
+  public void testGetTabletServer() throws Exception {
+    Key key = new Key("~wal:host:43861[14a7df0e6420003]", "log", "hdfs://localhost:50514/accumulo/wal/host:43861/70c27ab3-6662-40ab-80fb-01c1f1a59df3");
+    Text hostPort = new Text();
+    Text session = new Text();
+    CurrentLogsSection.getTabletServer(key, hostPort, session);
+    assertEquals("host:43861", hostPort.toString());
+    assertEquals("14a7df0e6420003", session.toString());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b2539fb1/core/src/test/java/org/apache/accumulo/core/replication/ReplicationOperationsImplTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/replication/ReplicationOperationsImplTest.java b/core/src/test/java/org/apache/accumulo/core/replication/ReplicationOperationsImplTest.java
index df9816e..8542113 100644
--- a/core/src/test/java/org/apache/accumulo/core/replication/ReplicationOperationsImplTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/replication/ReplicationOperationsImplTest.java
@@ -16,7 +16,6 @@
  */
 package org.apache.accumulo.core.replication;
 
-import java.util.Arrays;
 import java.util.Map.Entry;
 import java.util.UUID;
 import java.util.concurrent.atomic.AtomicBoolean;
@@ -263,13 +262,7 @@ public class ReplicationOperationsImplTest {
     bw.addMutation(m);
     bw.close();
 
-    LogEntry logEntry = new LogEntry();
-    logEntry.extent = new KeyExtent(new Text(tableId1), null, null);
-    logEntry.server = "tserver";
-    logEntry.filename = file1;
-    logEntry.tabletId = 1;
-    logEntry.logSet = Arrays.asList(file1);
-    logEntry.timestamp = System.currentTimeMillis();
+    LogEntry logEntry = new LogEntry(new KeyExtent(new Text(tableId1), null, null), System.currentTimeMillis(), "tserver", file1);
 
     bw = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
     m = new Mutation(ReplicationSection.getRowPrefix() + file1);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b2539fb1/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeUtil.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeUtil.java b/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeUtil.java
index 1dcc6aa..7bddbd9 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeUtil.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeUtil.java
@@ -127,15 +127,12 @@ public class VolumeUtil {
       switchedPath = le.filename;
 
     ArrayList<String> switchedLogs = new ArrayList<String>();
-    for (String log : le.logSet) {
-      String switchedLog = switchVolume(le.filename, FileType.WAL, replacements);
-      if (switchedLog != null) {
-        switchedLogs.add(switchedLog);
-        numSwitched++;
-      } else {
-        switchedLogs.add(log);
-      }
-
+    String switchedLog = switchVolume(le.filename, FileType.WAL, replacements);
+    if (switchedLog != null) {
+      switchedLogs.add(switchedLog);
+      numSwitched++;
+    } else {
+      switchedLogs.add(le.filename);
     }
 
     if (numSwitched == 0) {
@@ -143,9 +140,7 @@ public class VolumeUtil {
       return null;
     }
 
-    LogEntry newLogEntry = new LogEntry(le);
-    newLogEntry.filename = switchedPath;
-    newLogEntry.logSet = switchedLogs;
+    LogEntry newLogEntry = new LogEntry(le.extent, le.timestamp, le.server, switchedPath);
 
     log.trace("Switched " + le + " to " + newLogEntry);
 
@@ -243,7 +238,7 @@ public class VolumeUtil {
         log.debug("Tablet directory switched, need to record old log files " + logsToRemove + " " + ProtobufUtil.toString(status));
         // Before deleting these logs, we need to mark them for replication
         for (LogEntry logEntry : logsToRemove) {
-          ReplicationTableUtil.updateFiles(context, extent, logEntry.logSet, status);
+          ReplicationTableUtil.updateFiles(context, extent, logEntry.filename, status);
         }
       }
     }
@@ -252,7 +247,6 @@ public class VolumeUtil {
 
     // method this should return the exact strings that are in the metadata table
     return ret;
-
   }
 
   private static String decommisionedTabletDir(AccumuloServerContext context, ZooLock zooLock, VolumeManager vm, KeyExtent extent, String metaDir)

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b2539fb1/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java b/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java
index 495e0e0..329556d 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java
@@ -527,6 +527,7 @@ public class Initialize implements KeywordExecutable {
     zoo.putPersistentData(zkInstanceRoot + Constants.ZPROBLEMS, EMPTY_BYTE_ARRAY, NodeExistsPolicy.FAIL);
     zoo.putPersistentData(zkInstanceRoot + RootTable.ZROOT_TABLET, EMPTY_BYTE_ARRAY, NodeExistsPolicy.FAIL);
     zoo.putPersistentData(zkInstanceRoot + RootTable.ZROOT_TABLET_WALOGS, EMPTY_BYTE_ARRAY, NodeExistsPolicy.FAIL);
+    zoo.putPersistentData(zkInstanceRoot + RootTable.ZROOT_TABLET_CURRENT_LOGS, EMPTY_BYTE_ARRAY, NodeExistsPolicy.FAIL);
     zoo.putPersistentData(zkInstanceRoot + RootTable.ZROOT_TABLET_PATH, rootTabletDir.getBytes(UTF_8), NodeExistsPolicy.FAIL);
     zoo.putPersistentData(zkInstanceRoot + Constants.ZMASTERS, EMPTY_BYTE_ARRAY, NodeExistsPolicy.FAIL);
     zoo.putPersistentData(zkInstanceRoot + Constants.ZMASTER_LOCK, EMPTY_BYTE_ARRAY, NodeExistsPolicy.FAIL);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b2539fb1/server/base/src/main/java/org/apache/accumulo/server/master/state/MetaDataStateStore.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/state/MetaDataStateStore.java b/server/base/src/main/java/org/apache/accumulo/server/master/state/MetaDataStateStore.java
index 7ee6f0c..270bb31 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/state/MetaDataStateStore.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/state/MetaDataStateStore.java
@@ -17,6 +17,9 @@
 package org.apache.accumulo.server.master.state;
 
 import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.BatchWriter;
@@ -27,7 +30,9 @@ import org.apache.accumulo.core.client.impl.ClientContext;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema;
+import org.apache.accumulo.core.tabletserver.log.LogEntry;
 import org.apache.accumulo.server.AccumuloServerContext;
+import org.apache.hadoop.io.Text;
 
 public class MetaDataStateStore extends TabletStateStore {
 
@@ -59,7 +64,7 @@ public class MetaDataStateStore extends TabletStateStore {
 
   @Override
   public ClosableIterator<TabletLocationState> iterator() {
-    return new MetaDataTableScanner(context, MetadataSchema.TabletsSection.getRange(), state);
+    return new MetaDataTableScanner(context, MetadataSchema.TabletsSection.getRange(), state, targetTableName);
   }
 
   @Override
@@ -116,7 +121,7 @@ public class MetaDataStateStore extends TabletStateStore {
   }
 
   @Override
-  public void unassign(Collection<TabletLocationState> tablets) throws DistributedStoreException {
+  public void unassign(Collection<TabletLocationState> tablets, Map<TServerInstance, List<String>> logsForDeadServers) throws DistributedStoreException {
 
     BatchWriter writer = createBatchWriter();
     try {
@@ -128,6 +133,15 @@ public class MetaDataStateStore extends TabletStateStore {
         if (tls.future != null) {
           tls.future.clearFutureLocation(m);
         }
+        if (logsForDeadServers != null) {
+          List<String> logs = logsForDeadServers.get(tls.futureOrCurrent());
+          if (logs != null) {
+            for (String log : logs) {
+              LogEntry entry = new LogEntry(tls.extent, 0, tls.futureOrCurrent().hostPort(), log);
+              m.put(entry.getColumnFamily(), entry.getColumnQualifier(), entry.getValue());
+            }
+          }
+        }
         writer.addMutation(m);
       }
     } catch (Exception ex) {
@@ -145,4 +159,26 @@ public class MetaDataStateStore extends TabletStateStore {
   public String name() {
     return "Normal Tablets";
   }
+
+  @Override
+  public void markLogsAsUnused(AccumuloServerContext context, Map<TServerInstance,List<String>> logs) throws DistributedStoreException {
+    BatchWriter writer = createBatchWriter();
+    try {
+      for (Entry<TServerInstance,List<String>> entry : logs.entrySet()) {
+        Mutation m = new Mutation(MetadataSchema.CurrentLogsSection.getRowPrefix() + entry.getKey().toString());
+        for (String log : entry.getValue()) {
+          m.put(MetadataSchema.CurrentLogsSection.COLF, new Text(log), MetadataSchema.CurrentLogsSection.UNUSED);
+        }
+        writer.addMutation(m);
+      }
+    } catch (Exception ex) {
+      throw new DistributedStoreException(ex);
+    } finally {
+      try {
+        writer.close();
+      } catch (MutationsRejectedException e) {
+        throw new DistributedStoreException(e);
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b2539fb1/server/base/src/main/java/org/apache/accumulo/server/master/state/MetaDataTableScanner.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/state/MetaDataTableScanner.java b/server/base/src/main/java/org/apache/accumulo/server/master/state/MetaDataTableScanner.java
index 615e5d3..dac7fe6 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/state/MetaDataTableScanner.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/state/MetaDataTableScanner.java
@@ -43,10 +43,9 @@ import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.Lo
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.server.master.state.TabletLocationState.BadLocationStateException;
 import org.apache.hadoop.io.Text;
-import org.apache.log4j.Logger;
 
 public class MetaDataTableScanner implements ClosableIterator<TabletLocationState> {
-  private static final Logger log = Logger.getLogger(MetaDataTableScanner.class);
+  //private static final Logger log = Logger.getLogger(MetaDataTableScanner.class);
 
   BatchScanner mdScanner = null;
   Iterator<Entry<Key,Value>> iter = null;
@@ -140,6 +139,7 @@ public class MetaDataTableScanner implements ClosableIterator<TabletLocationStat
     boolean chopped = false;
 
     for (Entry<Key,Value> entry : decodedRow.entrySet()) {
+
       Key key = entry.getKey();
       Text row = key.getRow();
       Text cf = key.getColumnFamily();
@@ -172,8 +172,7 @@ public class MetaDataTableScanner implements ClosableIterator<TabletLocationStat
       }
     }
     if (extent == null) {
-      log.warn("No prev-row for key extent: " + decodedRow);
-      return null;
+      throw new BadLocationStateException("No prev-row for key extent " + decodedRow, k.getRow());
     }
     return new TabletLocationState(extent, future, current, last, walogs, chopped);
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b2539fb1/server/base/src/main/java/org/apache/accumulo/server/master/state/TabletLocationState.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/state/TabletLocationState.java b/server/base/src/main/java/org/apache/accumulo/server/master/state/TabletLocationState.java
index b24b562..a222532 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/state/TabletLocationState.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/state/TabletLocationState.java
@@ -68,6 +68,14 @@ public class TabletLocationState {
   final public Collection<Collection<String>> walogs;
   final public boolean chopped;
 
+  public TServerInstance futureOrCurrent() {
+    if (current != null) {
+      return current;
+    }
+    return future;
+  }
+
+  @Override
   public String toString() {
     return extent + "@(" + future + "," + current + "," + last + ")" + (chopped ? " chopped" : "");
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b2539fb1/server/base/src/main/java/org/apache/accumulo/server/master/state/TabletStateStore.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/state/TabletStateStore.java b/server/base/src/main/java/org/apache/accumulo/server/master/state/TabletStateStore.java
index 5413e31..de90d98 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/state/TabletStateStore.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/state/TabletStateStore.java
@@ -18,6 +18,8 @@ package org.apache.accumulo.server.master.state;
 
 import java.util.Collection;
 import java.util.Collections;
+import java.util.List;
+import java.util.Map;
 
 import org.apache.accumulo.server.AccumuloServerContext;
 
@@ -56,10 +58,12 @@ public abstract class TabletStateStore implements Iterable<TabletLocationState>
    *
    * @param tablets
    *          the tablets' current information
+   * @param logsForDeadServers
+   *          a cache of logs in use by servers when they died
    */
-  abstract public void unassign(Collection<TabletLocationState> tablets) throws DistributedStoreException;
+  abstract public void unassign(Collection<TabletLocationState> tablets, Map<TServerInstance, List<String>> logsForDeadServers) throws DistributedStoreException;
 
-  public static void unassign(AccumuloServerContext context, TabletLocationState tls) throws DistributedStoreException {
+  public static void unassign(AccumuloServerContext context, TabletLocationState tls, Map<TServerInstance, List<String>> logsForDeadServers) throws DistributedStoreException {
     TabletStateStore store;
     if (tls.extent.isRootTablet()) {
       store = new ZooTabletStateStore();
@@ -68,7 +72,7 @@ public abstract class TabletStateStore implements Iterable<TabletLocationState>
     } else {
       store = new MetaDataStateStore(context);
     }
-    store.unassign(Collections.singletonList(tls));
+    store.unassign(Collections.singletonList(tls), logsForDeadServers);
   }
 
   public static void setLocation(AccumuloServerContext context, Assignment assignment) throws DistributedStoreException {
@@ -83,4 +87,6 @@ public abstract class TabletStateStore implements Iterable<TabletLocationState>
     store.setLocations(Collections.singletonList(assignment));
   }
 
+  abstract public void markLogsAsUnused(AccumuloServerContext context, Map<TServerInstance, List<String>> logs) throws DistributedStoreException;
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b2539fb1/server/base/src/main/java/org/apache/accumulo/server/master/state/ZooTabletStateStore.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/state/ZooTabletStateStore.java b/server/base/src/main/java/org/apache/accumulo/server/master/state/ZooTabletStateStore.java
index 58b8446..a044434 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/state/ZooTabletStateStore.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/state/ZooTabletStateStore.java
@@ -21,11 +21,15 @@ import static java.nio.charset.StandardCharsets.UTF_8;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.accumulo.core.metadata.RootTable;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema;
 import org.apache.accumulo.core.tabletserver.log.LogEntry;
+import org.apache.accumulo.server.AccumuloServerContext;
 import org.apache.commons.lang.NotImplementedException;
 import org.apache.log4j.Logger;
 
@@ -84,10 +88,9 @@ public class ZooTabletStateStore extends TabletStateStore {
           for (String entry : store.getChildren(RootTable.ZROOT_TABLET_WALOGS)) {
             byte[] logInfo = store.get(RootTable.ZROOT_TABLET_WALOGS + "/" + entry);
             if (logInfo != null) {
-              LogEntry logEntry = new LogEntry();
-              logEntry.fromBytes(logInfo);
-              logs.add(logEntry.logSet);
-              log.debug("root tablet logSet " + logEntry.logSet);
+              LogEntry logEntry = LogEntry.fromBytes(logInfo);
+              logs.add(Collections.singleton(logEntry.filename));
+              log.debug("root tablet log " + logEntry.filename);
             }
           }
           TabletLocationState result = new TabletLocationState(RootTable.EXTENT, futureSession, currentSession, lastSession, logs, false);
@@ -160,12 +163,28 @@ public class ZooTabletStateStore extends TabletStateStore {
   }
 
   @Override
-  public void unassign(Collection<TabletLocationState> tablets) throws DistributedStoreException {
+  public void unassign(Collection<TabletLocationState> tablets, Map<TServerInstance, List<String>> logsForDeadServers) throws DistributedStoreException {
     if (tablets.size() != 1)
       throw new IllegalArgumentException("There is only one root tablet");
     TabletLocationState tls = tablets.iterator().next();
     if (tls.extent.compareTo(RootTable.EXTENT) != 0)
       throw new IllegalArgumentException("You can only store the root tablet location");
+    if (logsForDeadServers != null) {
+      List<String> logs = logsForDeadServers.get(tls.futureOrCurrent());
+      if (logs != null) {
+        for (String entry : logs) {
+          LogEntry logEntry = new LogEntry(RootTable.EXTENT, System.currentTimeMillis(), tls.futureOrCurrent().getLocation().toString(), entry);
+          byte[] value;
+          try {
+            value = logEntry.toBytes();
+          } catch (IOException ex) {
+            throw new DistributedStoreException(ex);
+          }
+          store.put(RootTable.ZROOT_TABLET_WALOGS + "/" + logEntry.getUniqueID(), value);
+          store.remove(RootTable.ZROOT_TABLET_CURRENT_LOGS + "/" + MetadataSchema.CurrentLogsSection.getRowPrefix() + tls.current.toString() + logEntry.getUniqueID());
+        }
+      }
+    }
     store.remove(RootTable.ZROOT_TABLET_LOCATION);
     store.remove(RootTable.ZROOT_TABLET_FUTURE_LOCATION);
     log.debug("unassign root tablet location");
@@ -176,4 +195,8 @@ public class ZooTabletStateStore extends TabletStateStore {
     return "Root Table";
   }
 
+  @Override
+  public void markLogsAsUnused(AccumuloServerContext context, Map<TServerInstance,List<String>> logs) {
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b2539fb1/server/base/src/main/java/org/apache/accumulo/server/util/ListVolumesUsed.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/ListVolumesUsed.java b/server/base/src/main/java/org/apache/accumulo/server/util/ListVolumesUsed.java
index e90d1dd..bf812cd 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/ListVolumesUsed.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/ListVolumesUsed.java
@@ -61,9 +61,6 @@ public class ListVolumesUsed {
 
   private static void getLogURIs(TreeSet<String> volumes, LogEntry logEntry) {
     volumes.add(getLogURI(logEntry.filename));
-    for (String logSet : logEntry.logSet) {
-      volumes.add(getLogURI(logSet));
-    }
   }
 
   private static void listZookeeper() throws Exception {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b2539fb1/server/base/src/main/java/org/apache/accumulo/server/util/MasterMetadataUtil.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/MasterMetadataUtil.java b/server/base/src/main/java/org/apache/accumulo/server/util/MasterMetadataUtil.java
index 80a6734..a1184e4 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/MasterMetadataUtil.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/MasterMetadataUtil.java
@@ -247,35 +247,27 @@ public class MasterMetadataUtil {
       if (unusedWalLogs != null) {
         updateRootTabletDataFile(extent, path, mergeFile, dfv, time, filesInUseByScans, address, zooLock, unusedWalLogs, lastLocation, flushId);
       }
-
       return;
     }
-
     Mutation m = getUpdateForTabletDataFile(extent, path, mergeFile, dfv, time, filesInUseByScans, address, zooLock, unusedWalLogs, lastLocation, flushId);
-
     MetadataTableUtil.update(context, zooLock, m, extent);
-
   }
 
   /**
    * Update the data file for the root tablet
    */
-  protected static void updateRootTabletDataFile(KeyExtent extent, FileRef path, FileRef mergeFile, DataFileValue dfv, String time,
+  private static void updateRootTabletDataFile(KeyExtent extent, FileRef path, FileRef mergeFile, DataFileValue dfv, String time,
       Set<FileRef> filesInUseByScans, String address, ZooLock zooLock, Set<String> unusedWalLogs, TServerInstance lastLocation, long flushId) {
     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 = MetadataTableUtil.getZookeeperLogLocation();
-    boolean foundEntry = false;
     for (String entry : unusedWalLogs) {
       String[] parts = entry.split("/");
       String zpath = root + "/" + parts[parts.length - 1];
       while (true) {
         try {
           if (zk.exists(zpath)) {
+            log.debug("Removing " + zpath);
             zk.recursiveDelete(zpath, NodeMissingPolicy.SKIP);
-            foundEntry = true;
           }
           break;
         } catch (KeeperException e) {
@@ -286,16 +278,15 @@ public class MasterMetadataUtil {
         UtilWaitThread.sleep(1000);
       }
     }
-    if (unusedWalLogs.size() > 0 && !foundEntry)
-      log.warn("WALog entry for root tablet did not exist " + unusedWalLogs);
   }
 
+
   /**
    * Create an update that updates a tablet
    *
    * @return A Mutation to update a tablet from the given information
    */
-  protected static Mutation getUpdateForTabletDataFile(KeyExtent extent, FileRef path, FileRef mergeFile, DataFileValue dfv, String time,
+  private static Mutation getUpdateForTabletDataFile(KeyExtent extent, FileRef path, FileRef mergeFile, DataFileValue dfv, String time,
       Set<FileRef> filesInUseByScans, String address, ZooLock zooLock, Set<String> unusedWalLogs, TServerInstance lastLocation, long flushId) {
     Mutation m = new Mutation(extent.getMetadataEntry());
 
@@ -323,6 +314,7 @@ public class MasterMetadataUtil {
 
     TabletsSection.ServerColumnFamily.FLUSH_COLUMN.put(m, new Value(Long.toString(flushId).getBytes(UTF_8)));
 
+
     return m;
   }
 }


[09/34] accumulo git commit: ACCUMULO-3423 fixed replication bugs with recent refactorings in StatusUtil

Posted by ec...@apache.org.
ACCUMULO-3423 fixed replication bugs with recent refactorings in StatusUtil


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

Branch: refs/heads/master
Commit: 31ee26b8ac41844f2a647a5d1484f47da731872a
Parents: 4635de8
Author: Eric C. Newton <er...@gmail.com>
Authored: Wed Mar 11 14:37:39 2015 -0400
Committer: Eric C. Newton <er...@gmail.com>
Committed: Wed Mar 11 14:37:39 2015 -0400

----------------------------------------------------------------------
 .../main/java/org/apache/accumulo/core/replication/StatusUtil.java | 2 +-
 .../java/org/apache/accumulo/tserver/log/TabletServerLogger.java   | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/31ee26b8/core/src/main/java/org/apache/accumulo/core/replication/StatusUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/replication/StatusUtil.java b/core/src/main/java/org/apache/accumulo/core/replication/StatusUtil.java
index d8ec403..cdb6963 100644
--- a/core/src/main/java/org/apache/accumulo/core/replication/StatusUtil.java
+++ b/core/src/main/java/org/apache/accumulo/core/replication/StatusUtil.java
@@ -155,7 +155,7 @@ public class StatusUtil {
   /**
    * @return A {@link Status} for an open file of unspecified length, all of which needs replicating.
    */
-  public static Status openWithUnknownLength(long timeCreated) {
+  public static synchronized Status openWithUnknownLength(long timeCreated) {
     return INF_END_REPLICATION_STATUS_BUILDER.setCreatedTime(timeCreated).build();
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/31ee26b8/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
index 46101c1..498cbdd 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
@@ -319,7 +319,7 @@ public class TabletServerLogger {
               // Need to release
               KeyExtent extent = commitSession.getExtent();
               if (ReplicationConfigurationUtil.isEnabled(extent, tserver.getTableConfiguration(extent))) {
-                Status status = StatusUtil.fileCreated(System.currentTimeMillis());
+                Status status = StatusUtil.openWithUnknownLength(System.currentTimeMillis());
                 log.debug("Writing " + ProtobufUtil.toString(status) + " to metadata table for " + copy.getFileName());
                 // Got some new WALs, note this in the metadata table
                 ReplicationTableUtil.updateFiles(tserver, commitSession.getExtent(), copy.getFileName(), status);


[22/34] accumulo git commit: ACCUMULO-3423 merge apache master, everything is broken

Posted by ec...@apache.org.
ACCUMULO-3423 merge apache master, everything is broken


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

Branch: refs/heads/master
Commit: 0177e3f644943be1e3d3700e1be3b68c64a1b7f1
Parents: bd6dbba 7480eed
Author: Eric C. Newton <er...@gmail.com>
Authored: Fri Apr 17 18:51:26 2015 -0400
Committer: Eric C. Newton <er...@gmail.com>
Committed: Fri Apr 17 18:51:26 2015 -0400

----------------------------------------------------------------------
 core/src/main/findbugs/exclude-filter.xml       |   1 +
 .../accumulo/core/bloomfilter/Filter.java       |  48 --
 .../core/client/impl/ReplicationClient.java     |  41 --
 .../core/client/impl/ScannerIterator.java       |   5 -
 .../core/client/impl/ScannerOptions.java        |   5 -
 .../accumulo/core/client/impl/Tables.java       |   5 -
 .../core/client/mapred/AbstractInputFormat.java | 263 +++++++----
 .../core/client/mapred/AccumuloInputFormat.java |   8 +-
 .../client/mapred/AccumuloOutputFormat.java     |   1 +
 .../core/client/mapred/InputFormatBase.java     |  60 ++-
 .../client/mapred/impl/BatchInputSplit.java     |  42 ++
 .../client/mapreduce/AbstractInputFormat.java   | 249 +++++++----
 .../client/mapreduce/AccumuloInputFormat.java   |  12 +-
 .../AccumuloMultiTableInputFormat.java          |  12 +-
 .../client/mapreduce/AccumuloOutputFormat.java  |   1 +
 .../core/client/mapreduce/InputFormatBase.java  |  65 ++-
 .../core/client/mapreduce/RangeInputSplit.java  | 424 ++----------------
 .../mapreduce/impl/AccumuloInputSplit.java      | 445 +++++++++++++++++++
 .../client/mapreduce/impl/BatchInputSplit.java  | 152 +++++++
 .../mapreduce/lib/impl/InputConfigurator.java   |  36 +-
 .../accumulo/core/client/mock/MockAccumulo.java |   4 -
 .../accumulo/core/client/mock/MockTable.java    |   6 +-
 .../org/apache/accumulo/core/conf/Property.java |  11 -
 .../accumulo/core/conf/SiteConfiguration.java   |  17 -
 .../apache/accumulo/core/data/PartialKey.java   |   2 +
 .../core/file/blockfile/ABlockReader.java       |   2 -
 .../core/file/blockfile/ABlockWriter.java       |   3 -
 .../core/file/blockfile/BlockFileWriter.java    |   2 -
 .../core/file/blockfile/cache/BlockCache.java   |   5 -
 .../core/file/blockfile/cache/CachedBlock.java  |   4 -
 .../core/file/blockfile/cache/ClassSize.java    |  98 ----
 .../file/blockfile/cache/LruBlockCache.java     |   1 -
 .../file/blockfile/impl/CachableBlockFile.java  |  28 --
 .../file/keyfunctor/ColumnFamilyFunctor.java    |   2 +-
 .../accumulo/core/file/rfile/bcfile/BCFile.java |  19 -
 .../core/file/rfile/bcfile/CompareUtils.java    |  36 --
 .../accumulo/core/file/rfile/bcfile/Utils.java  |  82 ----
 .../core/metadata/schema/MetadataSchema.java    |  14 -
 .../apache/accumulo/core/rpc/ThriftUtil.java    |  70 ---
 .../apache/accumulo/core/util/AddressUtil.java  |   4 -
 .../apache/accumulo/core/util/ByteArraySet.java |   4 -
 .../org/apache/accumulo/core/util/Daemon.java   |  20 -
 .../apache/accumulo/core/util/MapCounter.java   |   4 -
 .../apache/accumulo/core/util/StopWatch.java    |  15 -
 .../core/volume/VolumeConfiguration.java        |   5 -
 .../mapreduce/AccumuloInputFormatTest.java      |  30 +-
 .../mapreduce/impl/BatchInputSplitTest.java     | 122 +++++
 .../simple/mapreduce/TeraSortIngest.java        |   2 -
 .../accumulo/fate/zookeeper/ZooCache.java       |   9 -
 .../apache/accumulo/fate/zookeeper/ZooLock.java |  18 -
 .../accumulo/fate/zookeeper/ZooQueueLock.java   |   7 -
 .../accumulo/fate/zookeeper/ZooSession.java     |   4 -
 .../impl/MiniAccumuloClusterImpl.java           |  15 +-
 .../impl/ZooKeeperBindException.java            |   8 -
 .../apache/accumulo/server/ServerConstants.java |   4 -
 .../accumulo/server/client/BulkImporter.java    |   9 -
 .../server/conf/NamespaceConfWatcher.java       |   4 -
 .../accumulo/server/conf/TableConfWatcher.java  |   4 -
 .../server/conf/TableParentConfiguration.java   |  10 -
 .../server/conf/ZooConfigurationFactory.java    |  11 -
 .../accumulo/server/log/SortedLogState.java     |   8 -
 .../accumulo/server/master/LiveTServerSet.java  |   9 -
 .../master/state/DistributedStoreException.java |   3 -
 .../server/master/state/TServerInstance.java    |   4 -
 .../server/master/state/TabletServerState.java  |   5 -
 .../server/metrics/AbstractMetricsImpl.java     |   4 -
 .../server/metrics/MetricsConfiguration.java    |  18 -
 .../server/replication/StatusFormatter.java     |   5 -
 .../server/rpc/TBufferedServerSocket.java       |  71 ---
 .../server/rpc/TNonblockingServerSocket.java    |   7 -
 .../server/security/SecurityOperation.java      |   4 -
 .../security/handler/KerberosAuthorizor.java    |   8 -
 .../handler/KerberosPermissionHandler.java      |   8 -
 .../accumulo/server/tables/TableManager.java    |   4 -
 .../accumulo/server/util/AccumuloStatus.java    |  14 -
 .../accumulo/server/util/CleanZookeeper.java    |   3 -
 .../accumulo/server/util/FileSystemMonitor.java |  12 -
 .../accumulo/server/util/MetadataTableUtil.java |   5 -
 .../accumulo/server/util/RestoreZookeeper.java  |   2 -
 .../accumulo/server/util/time/RelativeTime.java |   4 -
 .../accumulo/server/util/time/SimpleTimer.java  |  10 -
 .../accumulo/server/zookeeper/ZooCache.java     |   5 -
 .../accumulo/gc/SimpleGarbageCollector.java     |   2 -
 .../org/apache/accumulo/monitor/Monitor.java    |  27 --
 .../accumulo/monitor/ZooKeeperStatus.java       |   4 -
 .../apache/accumulo/tracer/ZooTraceClient.java  |   3 +-
 .../tserver/ActiveAssignmentRunnable.java       |   4 +-
 .../accumulo/tserver/CompactionQueue.java       |  70 ++-
 .../apache/accumulo/tserver/FileManager.java    |   2 +-
 .../apache/accumulo/tserver/InMemoryMap.java    |   8 +-
 .../org/apache/accumulo/tserver/MemValue.java   |  11 -
 .../org/apache/accumulo/tserver/NativeMap.java  |   8 +-
 .../accumulo/tserver/RunnableStartedAt.java     |   5 -
 .../org/apache/accumulo/tserver/TLevel.java     |   6 -
 .../apache/accumulo/tserver/TabletServer.java   |   5 -
 .../tserver/TabletServerResourceManager.java    |  52 +--
 .../accumulo/tserver/TabletStatsKeeper.java     |  12 +-
 .../compaction/MajorCompactionRequest.java      |   7 -
 .../apache/accumulo/tserver/log/DfsLogger.java  |  16 +-
 .../accumulo/tserver/logger/LogFileKey.java     |   4 -
 .../accumulo/tserver/logger/LogFileValue.java   |   4 -
 .../mastermessage/SplitReportMessage.java       |  11 +-
 .../accumulo/tserver/session/Session.java       |   6 +-
 .../apache/accumulo/tserver/tablet/Rate.java    |   4 -
 .../apache/accumulo/tserver/tablet/Tablet.java  |   8 -
 .../java/org/apache/accumulo/shell/Shell.java   |  16 -
 .../accumulo/shell/ShellCommandException.java   |   4 -
 .../apache/accumulo/shell/ShellCompletor.java   |   4 -
 .../org/apache/accumulo/shell/ShellOptions.java |   3 -
 .../java/org/apache/accumulo/shell/Token.java   |   6 -
 .../accumulo/shell/commands/CompactCommand.java |   2 -
 .../accumulo/shell/commands/ScanCommand.java    |   9 -
 .../accumulo/test/AccumuloOutputFormatTest.java | 114 +++++
 .../test/functional/AccumuloInputFormatIT.java  |  49 +-
 .../test/replication/ReplicationIT.java         |   2 +-
 .../accumulo/trace/instrument/Tracer.java       |  29 ++
 116 files changed, 1561 insertions(+), 1759 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/0177e3f6/core/src/main/java/org/apache/accumulo/core/conf/Property.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/0177e3f6/core/src/main/java/org/apache/accumulo/core/metadata/schema/MetadataSchema.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/0177e3f6/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/0177e3f6/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/0177e3f6/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/0177e3f6/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/0177e3f6/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/0177e3f6/test/src/test/java/org/apache/accumulo/test/replication/ReplicationIT.java
----------------------------------------------------------------------
diff --cc test/src/test/java/org/apache/accumulo/test/replication/ReplicationIT.java
index b7a518a,46e3ac1..3c1cbeb
--- a/test/src/test/java/org/apache/accumulo/test/replication/ReplicationIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/replication/ReplicationIT.java
@@@ -149,14 -136,12 +149,14 @@@ public class ReplicationIT extends Conf
        if (Thread.interrupted()) {
          return logs;
        }
 -
 -      LogEntry logEntry = LogEntry.fromKeyValue(entry.getKey(), entry.getValue());
 -
 -      for (String log : logEntry.logSet) {
 -        // Need to normalize the log file from LogEntry
 -        logs.put(new Path(log).toString(), logEntry.extent.getTableId().toString());
 +      Text path = new Text();
 +      MetadataSchema.CurrentLogsSection.getPath(entry.getKey(), path);
 +      Text session = new Text();
 +      Text hostPort = new Text();
 +      MetadataSchema.CurrentLogsSection.getTabletServer(entry.getKey(), hostPort , session);
-       TServerInstance server = new TServerInstance(AddressUtil.parseAddress(hostPort.toString()), session.toString());
++      TServerInstance server = new TServerInstance(AddressUtil.parseAddress(hostPort.toString(), false), session.toString());
 +      for (String tableId : serverToTableID.get(server)) {
 +        logs.put(new Path(path.toString()).toString(), tableId);
        }
      }
      return logs;


[06/34] accumulo git commit: ACCUMULO-3638 merge master branch

Posted by ec...@apache.org.
ACCUMULO-3638 merge master branch


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

Branch: refs/heads/master
Commit: 902ee7ddaef13cfd2064fb3c06495955819a3bff
Parents: c8f3b7d 87208e5
Author: Eric C. Newton <er...@gmail.com>
Authored: Wed Mar 4 10:54:07 2015 -0500
Committer: Eric C. Newton <er...@gmail.com>
Committed: Wed Mar 4 10:54:07 2015 -0500

----------------------------------------------------------------------
 .../apache/accumulo/core/cli/ClientOpts.java    |  36 ++-
 .../client/impl/SecurityOperationsImpl.java     |  10 +-
 .../core/client/lexicoder/AbstractEncoder.java  |  60 ++++
 .../client/lexicoder/BigIntegerLexicoder.java   |  21 +-
 .../core/client/lexicoder/BytesLexicoder.java   |  20 +-
 .../core/client/lexicoder/DateLexicoder.java    |   8 +-
 .../core/client/lexicoder/DoubleLexicoder.java  |   8 +-
 .../core/client/lexicoder/IntegerLexicoder.java |   8 +-
 .../core/client/lexicoder/ListLexicoder.java    |  14 +-
 .../core/client/lexicoder/LongLexicoder.java    |   4 +-
 .../core/client/lexicoder/PairLexicoder.java    |  11 +-
 .../core/client/lexicoder/ReverseLexicoder.java |  15 +-
 .../core/client/lexicoder/StringLexicoder.java  |   8 +-
 .../core/client/lexicoder/TextLexicoder.java    |   9 +-
 .../client/lexicoder/UIntegerLexicoder.java     |  16 +-
 .../core/client/lexicoder/ULongLexicoder.java   |  16 +-
 .../core/client/lexicoder/UUIDLexicoder.java    |  13 +-
 .../lexicoder/impl/AbstractLexicoder.java       |  23 ++
 .../core/client/lexicoder/impl/ByteUtils.java   |  13 +-
 .../core/client/mapred/AbstractInputFormat.java |  14 +-
 .../client/mapreduce/AbstractInputFormat.java   |  14 +-
 .../mapreduce/lib/impl/ConfiguratorBase.java    |  38 ++-
 .../client/security/tokens/KerberosToken.java   |  27 ++
 .../org/apache/accumulo/core/conf/Property.java |   2 +
 .../accumulo/core/iterators/LongCombiner.java   |  27 +-
 .../core/iterators/user/BigDecimalCombiner.java |   7 +-
 .../iterators/user/SummingArrayCombiner.java    |  14 +-
 .../lexicoder/BigIntegerLexicoderTest.java      |  12 +-
 .../client/lexicoder/BytesLexicoderTest.java    |  28 ++
 .../client/lexicoder/DateLexicoderTest.java     |  30 ++
 .../client/lexicoder/DoubleLexicoderTest.java   |  14 +-
 .../client/lexicoder/IntegerLexicoderTest.java  |  12 +-
 .../core/client/lexicoder/LexicoderTest.java    |  31 +-
 .../client/lexicoder/ListLexicoderTest.java     |  33 ++-
 .../client/lexicoder/LongLexicoderTest.java     |  13 +-
 .../client/lexicoder/PairLexicoderTest.java     |  12 +-
 .../client/lexicoder/ReverseLexicoderTest.java  |  15 +-
 .../client/lexicoder/StringLexicoderTest.java   |  28 ++
 .../client/lexicoder/TextLexicoderTest.java     |  28 ++
 .../client/lexicoder/UIntegerLexicoderTest.java |  12 +-
 .../client/lexicoder/ULongLexicoderTest.java    |  13 +-
 .../client/lexicoder/UUIDLexicoderTest.java     |   9 +-
 .../lexicoder/impl/AbstractLexicoderTest.java   |  94 ++++++
 .../client/lexicoder/impl/ByteUtilsTest.java    |  71 +++++
 .../core/iterators/user/CombinerTest.java       |   4 +
 .../main/asciidoc/chapters/administration.txt   |  23 +-
 docs/src/main/resources/distributedTracing.html |   6 +-
 .../accumulo/cluster/AccumuloCluster.java       |   6 +
 .../apache/accumulo/cluster/ClusterUser.java    | 141 +++++++++
 .../apache/accumulo/cluster/ClusterUsers.java   |  39 +++
 .../standalone/StandaloneAccumuloCluster.java   |  37 ++-
 .../impl/MiniAccumuloClusterImpl.java           |  11 +
 pom.xml                                         |   4 +-
 .../server/client/ClientServiceHandler.java     |  14 +-
 .../server/replication/StatusCombiner.java      |  24 +-
 .../server/security/SecurityOperation.java      |   2 +
 .../delegation/ZooAuthenticationKeyWatcher.java |  14 +-
 .../ZooAuthenticationKeyWatcherTest.java        |  24 ++
 .../accumulo/monitor/EmbeddedWebServer.java     |   2 +
 .../monitor/servlets/DefaultServlet.java        |   4 +-
 .../accumulo/monitor/servlets/trace/Basic.java  |  20 +-
 .../java/org/apache/accumulo/shell/Shell.java   |   3 +
 .../shell/commands/CreateUserCommand.java       |  36 ++-
 .../accumulo/shell/ShellSetInstanceTest.java    |   4 +
 .../start/classloader/AccumuloClassLoader.java  |  22 +-
 .../org/apache/accumulo/test/TestIngest.java    |   5 +-
 .../accumulo/test/TestMultiTableIngest.java     |   2 +-
 .../accumulo/harness/AccumuloClusterIT.java     | 180 ++++++++----
 .../org/apache/accumulo/harness/AccumuloIT.java |   5 +-
 .../accumulo/harness/MiniClusterHarness.java    |  12 +-
 .../accumulo/harness/SharedMiniClusterIT.java   |  68 ++++-
 .../org/apache/accumulo/harness/TestingKdc.java | 126 +++++---
 .../conf/AccumuloClusterConfiguration.java      |   6 +-
 .../AccumuloClusterPropertyConfiguration.java   |  19 +-
 .../conf/AccumuloMiniClusterConfiguration.java  |  32 ++-
 .../StandaloneAccumuloClusterConfiguration.java | 127 +++++++-
 .../test/ArbitraryTablePropertiesIT.java        |  56 ++--
 .../accumulo/test/BulkImportVolumeIT.java       |   2 +-
 .../org/apache/accumulo/test/CleanWalIT.java    |   4 +-
 .../accumulo/test/ConditionalWriterIT.java      |  43 ++-
 .../apache/accumulo/test/ImportExportIT.java    |  63 ++--
 .../accumulo/test/InterruptibleScannersIT.java  |   2 +-
 .../accumulo/test/MetaConstraintRetryIT.java    |   4 +-
 .../org/apache/accumulo/test/MetaSplitIT.java   |  40 +--
 .../accumulo/test/MultiTableBatchWriterIT.java  |   2 +-
 .../org/apache/accumulo/test/NamespacesIT.java  |  94 +++---
 .../apache/accumulo/test/ScanIteratorIT.java    |  58 ++--
 .../org/apache/accumulo/test/ShellConfigIT.java |  70 ++++-
 .../org/apache/accumulo/test/ShellServerIT.java | 121 ++++++--
 .../apache/accumulo/test/SplitRecoveryIT.java   |   2 +-
 .../apache/accumulo/test/TableOperationsIT.java |   4 +-
 .../accumulo/test/TransportCachingIT.java       |   2 +-
 .../test/functional/AccumuloInputFormatIT.java  |  12 +-
 .../BalanceInPresenceOfOfflineTableIT.java      |  24 +-
 .../test/functional/BigRootTabletIT.java        |   3 +-
 .../test/functional/BinaryStressIT.java         |   2 +-
 .../accumulo/test/functional/BloomFilterIT.java |   3 +-
 .../accumulo/test/functional/BulkFileIT.java    |   2 +-
 .../apache/accumulo/test/functional/BulkIT.java |  62 ++--
 .../functional/BulkSplitOptimizationIT.java     |  21 +-
 .../test/functional/ChaoticBalancerIT.java      |  13 +-
 .../accumulo/test/functional/CleanTmpIT.java    |   4 +-
 .../accumulo/test/functional/CleanUpIT.java     |  39 +--
 .../accumulo/test/functional/CompactionIT.java  |  15 +-
 .../accumulo/test/functional/ConcurrencyIT.java |   6 +-
 .../accumulo/test/functional/CredentialsIT.java |  51 ++--
 .../test/functional/DeleteEverythingIT.java     |   6 +-
 .../accumulo/test/functional/DeleteIT.java      |  51 +++-
 .../test/functional/DeleteRowsSplitIT.java      |  18 +-
 .../test/functional/DynamicThreadPoolsIT.java   |  18 +-
 .../accumulo/test/functional/ExamplesIT.java    | 286 +++++++++++++++----
 .../test/functional/FateStarvationIT.java       |   8 +
 .../test/functional/GarbageCollectorIT.java     |   2 +
 .../test/functional/HalfDeadTServerIT.java      |   1 +
 .../accumulo/test/functional/KerberosIT.java    |  91 +++---
 .../test/functional/KerberosProxyIT.java        |  14 +-
 .../accumulo/test/functional/LargeRowIT.java    |   6 +-
 .../test/functional/MasterAssignmentIT.java     |   2 +-
 .../test/functional/MasterFailoverIT.java       |  19 +-
 .../accumulo/test/functional/MaxOpenIT.java     |  11 +-
 .../accumulo/test/functional/MonitorSslIT.java  | 128 +++++++++
 .../accumulo/test/functional/PermissionsIT.java | 240 ++++++++++------
 .../accumulo/test/functional/ReadWriteIT.java   | 104 +++++--
 .../functional/RecoveryWithEmptyRFileIT.java    |   4 +-
 .../accumulo/test/functional/RenameIT.java      |  17 ++
 .../accumulo/test/functional/RestartIT.java     |  92 +++++-
 .../test/functional/RestartStressIT.java        |  41 ++-
 .../accumulo/test/functional/RowDeleteIT.java   |   4 +-
 .../test/functional/ScanSessionTimeOutIT.java   |   6 +-
 .../functional/SimpleBalancerFairnessIT.java    |   1 +
 .../accumulo/test/functional/SplitIT.java       |  48 +++-
 .../accumulo/test/functional/TableIT.java       |  14 +-
 .../accumulo/test/functional/TabletIT.java      |   3 +-
 .../accumulo/test/functional/VisibilityIT.java  |  22 +-
 .../test/functional/WriteAheadLogIT.java        |  17 +-
 .../accumulo/test/functional/WriteLotsIT.java   |  13 +
 .../test/replication/ReplicationIT.java         |   4 +-
 .../test/replication/StatusCombinerMacIT.java   |   4 +-
 .../UnorderedWorkAssignerReplicationIT.java     |   2 +-
 test/src/test/resources/log4j.properties        |   2 +-
 .../accumulo/trace/instrument/CountSampler.java |   2 +-
 .../accumulo/trace/instrument/Sampler.java      |   2 +-
 142 files changed, 3118 insertions(+), 942 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/902ee7dd/core/src/main/java/org/apache/accumulo/core/conf/Property.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/902ee7dd/test/src/test/java/org/apache/accumulo/test/CleanWalIT.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/902ee7dd/test/src/test/java/org/apache/accumulo/test/ConditionalWriterIT.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/902ee7dd/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/902ee7dd/test/src/test/java/org/apache/accumulo/test/replication/ReplicationIT.java
----------------------------------------------------------------------


[15/34] accumulo git commit: ACCUMULO-3423 updates based on review by [~elserj] and [~kturner]

Posted by ec...@apache.org.
ACCUMULO-3423 updates based on review by [~elserj] and [~kturner]


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

Branch: refs/heads/master
Commit: afa887b6f5f131a06497eaf1d04ba8c55b0d2877
Parents: daa38ce
Author: Eric C. Newton <er...@gmail.com>
Authored: Mon Mar 30 11:25:19 2015 -0400
Committer: Eric C. Newton <er...@gmail.com>
Committed: Mon Mar 30 11:25:19 2015 -0400

----------------------------------------------------------------------
 .../core/metadata/schema/MetadataSchema.java    |   2 +-
 .../core/metadata/MetadataTableSchemaTest.java  |  10 +-
 .../server/master/state/MetaDataStateStore.java |  18 +--
 .../master/state/TabletLocationState.java       |   2 -
 .../accumulo/server/util/ListVolumesUsed.java   |  15 ++
 .../server/util/MasterMetadataUtil.java         |   2 +-
 .../accumulo/server/util/MetadataTableUtil.java |  63 +++++---
 .../gc/GarbageCollectWriteAheadLogs.java        |  33 +++--
 .../CloseWriteAheadLogReferences.java           |   2 -
 .../accumulo/master/TabletGroupWatcher.java     |   2 +
 .../server/GarbageCollectionLogger.java         |   2 +-
 .../apache/accumulo/tserver/TabletServer.java   |   3 +
 .../tserver/log/TabletServerLogger.java         |  22 +--
 .../apache/accumulo/tserver/tablet/Tablet.java  |   4 +-
 .../accumulo/tserver/log/LogEntryTest.java      |  56 ++++++++
 .../org/apache/accumulo/test/UnusedWALIT.java   | 144 +++++++++++++++++++
 .../java/org/apache/accumulo/test/VolumeIT.java |  17 +++
 17 files changed, 337 insertions(+), 60 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/afa887b6/core/src/main/java/org/apache/accumulo/core/metadata/schema/MetadataSchema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/schema/MetadataSchema.java b/core/src/main/java/org/apache/accumulo/core/metadata/schema/MetadataSchema.java
index 88e11f4..d2f7d07 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/schema/MetadataSchema.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/schema/MetadataSchema.java
@@ -322,7 +322,7 @@ public class MetadataSchema {
 
       Text row = new Text();
       k.getRow(row);
-      if (row.getLength() < section.getRowPrefix().length()) {
+      if (!row.toString().startsWith(section.getRowPrefix())) {
         throw new IllegalArgumentException("Bad key " + k.toString());
       }
       for (int sessionStart = section.getRowPrefix().length(); sessionStart < row.getLength() - 1; sessionStart++) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/afa887b6/core/src/test/java/org/apache/accumulo/core/metadata/MetadataTableSchemaTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/metadata/MetadataTableSchemaTest.java b/core/src/test/java/org/apache/accumulo/core/metadata/MetadataTableSchemaTest.java
index dfc74cf..cfe59f2 100644
--- a/core/src/test/java/org/apache/accumulo/core/metadata/MetadataTableSchemaTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/metadata/MetadataTableSchemaTest.java
@@ -18,6 +18,7 @@
 package org.apache.accumulo.core.metadata;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
 
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.CurrentLogsSection;
@@ -28,12 +29,19 @@ public class MetadataTableSchemaTest {
 
   @Test
   public void testGetTabletServer() throws Exception {
-    Key key = new Key("~wal:host:43861[14a7df0e6420003]", "log", "hdfs://localhost:50514/accumulo/wal/host:43861/70c27ab3-6662-40ab-80fb-01c1f1a59df3");
+    Key key = new Key("~wal+host:43861[14a7df0e6420003]", "log", "hdfs://localhost:50514/accumulo/wal/host:43861/70c27ab3-6662-40ab-80fb-01c1f1a59df3");
     Text hostPort = new Text();
     Text session = new Text();
     CurrentLogsSection.getTabletServer(key, hostPort, session);
     assertEquals("host:43861", hostPort.toString());
     assertEquals("14a7df0e6420003", session.toString());
+    try {
+      Key bogus = new Key("~wal/host:43861[14a7df0e6420003]", "log", "hdfs://localhost:50514/accumulo/wal/host:43861/70c27ab3-6662-40ab-80fb-01c1f1a59df3");
+      CurrentLogsSection.getTabletServer(bogus, hostPort, session);
+      fail("bad argument not thrown");
+    } catch (IllegalArgumentException ex) {
+
+    }
   }
 
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/afa887b6/server/base/src/main/java/org/apache/accumulo/server/master/state/MetaDataStateStore.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/state/MetaDataStateStore.java b/server/base/src/main/java/org/apache/accumulo/server/master/state/MetaDataStateStore.java
index decc8c7..adcf04d 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/state/MetaDataStateStore.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/state/MetaDataStateStore.java
@@ -132,19 +132,19 @@ public class MetaDataStateStore extends TabletStateStore {
         Mutation m = new Mutation(tls.extent.getMetadataEntry());
         if (tls.current != null) {
           tls.current.clearLocation(m);
+          if (logsForDeadServers != null) {
+            List<Path> logs = logsForDeadServers.get(tls.current);
+            if (logs != null) {
+              for (Path log : logs) {
+                LogEntry entry = new LogEntry(tls.extent, 0, tls.current.hostPort(), log.toString());
+                m.put(entry.getColumnFamily(), entry.getColumnQualifier(), entry.getValue());
+              }
+            }
+          }
         }
         if (tls.future != null) {
           tls.future.clearFutureLocation(m);
         }
-        if (logsForDeadServers != null) {
-          List<Path> logs = logsForDeadServers.get(tls.futureOrCurrent());
-          if (logs != null) {
-            for (Path log : logs) {
-              LogEntry entry = new LogEntry(tls.extent, 0, tls.futureOrCurrent().hostPort(), log.toString());
-              m.put(entry.getColumnFamily(), entry.getColumnQualifier(), entry.getValue());
-            }
-          }
-        }
         writer.addMutation(m);
       }
     } catch (Exception ex) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/afa887b6/server/base/src/main/java/org/apache/accumulo/server/master/state/TabletLocationState.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/state/TabletLocationState.java b/server/base/src/main/java/org/apache/accumulo/server/master/state/TabletLocationState.java
index ebad2c8..a222532 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/state/TabletLocationState.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/state/TabletLocationState.java
@@ -32,8 +32,6 @@ import org.apache.hadoop.io.Text;
  */
 public class TabletLocationState {
 
-  // private static final Logger log = Logger.getLogger(TabletLocationState.class);
-
   static public class BadLocationStateException extends Exception {
     private static final long serialVersionUID = 1L;
     private Text metadataTableEntry;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/afa887b6/server/base/src/main/java/org/apache/accumulo/server/util/ListVolumesUsed.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/ListVolumesUsed.java b/server/base/src/main/java/org/apache/accumulo/server/util/ListVolumesUsed.java
index bf812cd..9e3fc7d 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/ListVolumesUsed.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/ListVolumesUsed.java
@@ -35,6 +35,7 @@ import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.accumulo.server.conf.ServerConfigurationFactory;
 import org.apache.accumulo.server.fs.VolumeManager.FileType;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
 
 /**
  *
@@ -120,6 +121,20 @@ public class ListVolumesUsed {
 
     for (String volume : volumes)
       System.out.println("\tVolume : " + volume);
+
+    volumes.clear();
+    scanner.clearColumns();
+    scanner.setRange(MetadataSchema.CurrentLogsSection.getRange());
+    Text path = new Text();
+    for (Entry<Key,Value> entry : scanner) {
+      MetadataSchema.CurrentLogsSection.getPath(entry.getKey(), path);
+      volumes.add(getLogURI(path.toString()));
+    }
+
+    System.out.println("Listing volumes referenced in " + name + " current logs section");
+
+    for (String volume : volumes)
+      System.out.println("\tVolume : " + volume);
   }
 
   public static void listVolumes(ClientContext context) throws Exception {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/afa887b6/server/base/src/main/java/org/apache/accumulo/server/util/MasterMetadataUtil.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/MasterMetadataUtil.java b/server/base/src/main/java/org/apache/accumulo/server/util/MasterMetadataUtil.java
index 24092f9..fb6c4ee 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/MasterMetadataUtil.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/MasterMetadataUtil.java
@@ -267,7 +267,7 @@ public class MasterMetadataUtil {
       while (true) {
         try {
           if (zk.exists(zpath)) {
-            log.debug("Removing " + zpath);
+            log.debug("Removing WAL reference for root table " + zpath);
             zk.recursiveDelete(zpath, NodeMissingPolicy.SKIP);
           }
           break;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/afa887b6/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java
----------------------------------------------------------------------
diff --git 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
index db00b9c..f5326bf 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
@@ -1065,8 +1065,12 @@ public class MetadataTableUtil {
         public void run(IZooReaderWriter rw) throws KeeperException, InterruptedException, IOException {
           String root = ZooUtil.getRoot(HdfsZooInstance.getInstance()) + RootTable.ZROOT_TABLET_CURRENT_LOGS;
           String uniqueId = filename.getName();
-          String path = root + "/" + CurrentLogsSection.getRowPrefix() + tabletSession.toString() + uniqueId;
-          rw.putPersistentData(path, filename.toString().getBytes(UTF_8), NodeExistsPolicy.OVERWRITE);
+          StringBuilder path = new StringBuilder(root);
+          path.append("/");
+          path.append(CurrentLogsSection.getRowPrefix());
+          path.append(tabletSession.toString());
+          path.append(uniqueId);
+          rw.putPersistentData(path.toString(), filename.toString().getBytes(UTF_8), NodeExistsPolicy.OVERWRITE);
         }
       });
     } else {
@@ -1076,12 +1080,20 @@ public class MetadataTableUtil {
       if (extent.isMeta()) {
         tableName = RootTable.NAME;
       }
+      BatchWriter bw = null;
       try {
-        BatchWriter bw = context.getConnector().createBatchWriter(tableName, null);
+        bw = context.getConnector().createBatchWriter(tableName, null);
         bw.addMutation(m);
-        bw.close();
       } catch (Exception e) {
         throw new RuntimeException(e);
+      } finally {
+        if (bw != null) {
+          try {
+            bw.close();
+          } catch (Exception e2) {
+            throw new RuntimeException(e2);
+          }
+        }
       }
     }
   }
@@ -1101,21 +1113,30 @@ public class MetadataTableUtil {
 
   public static void markLogUnused(ClientContext context, ZooLock lock, TServerInstance tabletSession, Set<Path> all) throws AccumuloException {
     try {
-      BatchWriter root = context.getConnector().createBatchWriter(RootTable.NAME, null);
-      BatchWriter meta = context.getConnector().createBatchWriter(MetadataTable.NAME, null);
-      for (Path fname : all) {
-        Text tname = new Text(fname.toString());
-        Mutation m = new Mutation(MetadataSchema.CurrentLogsSection.getRowPrefix() + tabletSession.toString());
-        m.putDelete(MetadataSchema.CurrentLogsSection.COLF, tname);
-        root.addMutation(m);
-        log.debug("deleting " + MetadataSchema.CurrentLogsSection.getRowPrefix() + tabletSession.toString() + " log:" + fname);
-        m = new Mutation(MetadataSchema.CurrentLogsSection.getRowPrefix() + tabletSession.toString());
-        m.put(MetadataSchema.CurrentLogsSection.COLF, tname, MetadataSchema.CurrentLogsSection.UNUSED);
-        meta.addMutation(m);
-        removeCurrentRootLogMarker(context, lock, tabletSession, fname);
+      BatchWriter root = null;
+      BatchWriter meta = null;
+      try {
+        root = context.getConnector().createBatchWriter(RootTable.NAME, null);
+        meta = context.getConnector().createBatchWriter(MetadataTable.NAME, null);
+        for (Path fname : all) {
+          Text tname = new Text(fname.toString());
+          Mutation m = new Mutation(MetadataSchema.CurrentLogsSection.getRowPrefix() + tabletSession.toString());
+          m.putDelete(MetadataSchema.CurrentLogsSection.COLF, tname);
+          root.addMutation(m);
+          log.debug("deleting " + MetadataSchema.CurrentLogsSection.getRowPrefix() + tabletSession.toString() + " log:" + fname);
+          m = new Mutation(MetadataSchema.CurrentLogsSection.getRowPrefix() + tabletSession.toString());
+          m.put(MetadataSchema.CurrentLogsSection.COLF, tname, MetadataSchema.CurrentLogsSection.UNUSED);
+          meta.addMutation(m);
+          removeCurrentRootLogMarker(context, lock, tabletSession, fname);
+        }
+      } finally {
+        if (root != null) {
+          root.close();
+        }
+        if (meta != null) {
+          meta.close();
+        }
       }
-      root.close();
-      meta.close();
     } catch (Exception ex) {
       throw new AccumuloException(ex);
     }
@@ -1150,8 +1171,12 @@ public class MetadataTableUtil {
       Scanner scanner = context.getConnector().createScanner(table, Authorizations.EMPTY);
       scanner.setRange(new Range(MetadataSchema.CurrentLogsSection.getRowPrefix() + server.toString()));
       List<Path> logs = new ArrayList<>();
+      Text path = new Text();
       for (Entry<Key,Value> entry : scanner) {
-        logs.add(new Path(entry.getKey().getColumnQualifier().toString()));
+        MetadataSchema.CurrentLogsSection.getPath(entry.getKey(), path);
+        if (!entry.getValue().equals(MetadataSchema.CurrentLogsSection.UNUSED)) {
+          logs.add(new Path(path.toString()));
+        }
       }
       logsForDeadServers.put(server, logs);
     }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/afa887b6/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 cf068ed..d523706 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
@@ -180,19 +180,28 @@ public class GarbageCollectWriteAheadLogs {
   private long removeMarkers(Map<TServerInstance,Set<Path>> candidates) {
     long result = 0;
     try {
-      BatchWriter root = context.getConnector().createBatchWriter(RootTable.NAME, null);
-      BatchWriter meta = context.getConnector().createBatchWriter(MetadataTable.NAME, null);
-      for (Entry<TServerInstance,Set<Path>> entry : candidates.entrySet()) {
-        Mutation m = new Mutation(CurrentLogsSection.getRowPrefix() + entry.getKey().toString());
-        for (Path path : entry.getValue()) {
-          m.putDelete(CurrentLogsSection.COLF, new Text(path.toString()));
-          result++;
+      BatchWriter root = null;
+      BatchWriter meta = null;
+      try {
+        root = context.getConnector().createBatchWriter(RootTable.NAME, null);
+        meta = context.getConnector().createBatchWriter(MetadataTable.NAME, null);
+        for (Entry<TServerInstance,Set<Path>> entry : candidates.entrySet()) {
+          Mutation m = new Mutation(CurrentLogsSection.getRowPrefix() + entry.getKey().toString());
+          for (Path path : entry.getValue()) {
+            m.putDelete(CurrentLogsSection.COLF, new Text(path.toString()));
+            result++;
+          }
+          root.addMutation(m);
+          meta.addMutation(m);
+        }
+      } finally  {
+        if (meta != null) {
+          meta.close();
+        }
+        if (root != null) {
+          root.close();
         }
-        root.addMutation(m);
-        meta.addMutation(m);
       }
-      meta.close();
-      root.close();
     } catch (Exception ex) {
       throw new RuntimeException(ex);
     }
@@ -386,7 +395,7 @@ public class GarbageCollectWriteAheadLogs {
       CurrentLogsSection.getPath(entry.getKey(), filename);
       TServerInstance tsi = new TServerInstance(HostAndPort.fromString(hostAndPort.toString()), sessionId.toString());
       Path path = new Path(filename.toString());
-      if ((!currentServers.contains(tsi) || (entry.getValue().equals(CurrentLogsSection.UNUSED)) && !rootWALs.contains(path))) {
+      if (!currentServers.contains(tsi) || entry.getValue().equals(CurrentLogsSection.UNUSED) && !rootWALs.contains(path)) {
         Set<Path> logs = unusedLogs.get(tsi);
         if (logs == null) {
           unusedLogs.put(tsi, logs = new HashSet<Path>());

http://git-wip-us.apache.org/repos/asf/accumulo/blob/afa887b6/server/gc/src/main/java/org/apache/accumulo/gc/replication/CloseWriteAheadLogReferences.java
----------------------------------------------------------------------
diff --git a/server/gc/src/main/java/org/apache/accumulo/gc/replication/CloseWriteAheadLogReferences.java b/server/gc/src/main/java/org/apache/accumulo/gc/replication/CloseWriteAheadLogReferences.java
index 6686cb8..455aaee 100644
--- a/server/gc/src/main/java/org/apache/accumulo/gc/replication/CloseWriteAheadLogReferences.java
+++ b/server/gc/src/main/java/org/apache/accumulo/gc/replication/CloseWriteAheadLogReferences.java
@@ -247,8 +247,6 @@ public class CloseWriteAheadLogReferences implements Runnable {
         MetadataSchema.ReplicationSection.getFile(entry.getKey(), replFileText);
         String replFile = replFileText.toString();
         boolean isReferenced = referencedWals.contains(replFile);
-        log.debug("replFile " + replFile);
-        log.debug("referencedWals " + referencedWals);
 
         // We only want to clean up WALs (which is everything but rfiles) and only when
         // metadata doesn't have a reference to the given WAL

http://git-wip-us.apache.org/repos/asf/accumulo/blob/afa887b6/server/master/src/main/java/org/apache/accumulo/master/TabletGroupWatcher.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/TabletGroupWatcher.java b/server/master/src/main/java/org/apache/accumulo/master/TabletGroupWatcher.java
index a536e98..9a7c40e 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/TabletGroupWatcher.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/TabletGroupWatcher.java
@@ -313,6 +313,8 @@ class TabletGroupWatcher extends Daemon {
         if (this.master.tserverSet.getCurrentServers().equals(currentTServers.keySet())) {
           Master.log.debug(String.format("[%s] sleeping for %.2f seconds", store.name(), Master.TIME_TO_WAIT_BETWEEN_SCANS / 1000.));
           eventListener.waitForEvents(Master.TIME_TO_WAIT_BETWEEN_SCANS);
+        } else {
+          Master.log.info("Detected change in current tserver set, re-running state machine.");
         }
       } catch (Exception ex) {
         Master.log.error("Error processing table state for store " + store.name(), ex);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/afa887b6/server/tserver/src/main/java/org/apache/accumulo/server/GarbageCollectionLogger.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/server/GarbageCollectionLogger.java b/server/tserver/src/main/java/org/apache/accumulo/server/GarbageCollectionLogger.java
index 5fe2548..57d8da1 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/server/GarbageCollectionLogger.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/server/GarbageCollectionLogger.java
@@ -98,7 +98,7 @@ public class GarbageCollectionLogger {
     final long keepAliveTimeout = conf.getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT);
     if (lastMemoryCheckTime > 0 && lastMemoryCheckTime < now) {
       final long diff = now - lastMemoryCheckTime;
-      if (diff > keepAliveTimeout + 1000) {
+      if (diff > keepAliveTimeout) {
         log.warn(String.format("GC pause checker not called in a timely fashion. Expected every %.1f seconds but was %.1f seconds since last check",
             keepAliveTimeout / 1000., diff / 1000.));
       }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/afa887b6/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
index ffc1c2a..9389776 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
@@ -1719,6 +1719,8 @@ public class TabletServer extends AccumuloServerContext implements Runnable {
     @Override
     public void removeLogs(TInfo tinfo, TCredentials credentials, List<String> filenames) throws TException {
       log.warn("Garbage collector is attempting to remove logs through the tablet server");
+      log.warn("This is probably because your file Garbage Collector is an older version than your tablet servers.\n" +
+          "Restart your file Garbage Collector.");
     }
   }
 
@@ -3015,6 +3017,7 @@ public class TabletServer extends AccumuloServerContext implements Runnable {
 
   public void addLoggersToMetadata(DfsLogger copy, KeyExtent extent) {
     TabletLevel level = TabletLevel.getLevel(extent);
+    // serialize the updates to the metadata per level: avoids updating the level more than once
     synchronized (level) {
       EnumSet<TabletLevel> set = metadataTableLogs.putIfAbsent(copy, EnumSet.of(level));
       if (set == null || !set.contains(level) || level == TabletLevel.ROOT) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/afa887b6/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
index accfc5e..0f3f642 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
@@ -262,17 +262,19 @@ public class TabletServerLogger {
       throw new IllegalStateException("close should be called with write lock held!");
     }
     try {
-      try {
-        currentLog.close();
-      } catch (DfsLogger.LogClosedException ex) {
-        // ignore
-      } catch (Throwable ex) {
-        log.error("Unable to cleanly close log " + currentLog.getFileName() + ": " + ex, ex);
-      } finally {
-        this.tserver.walogClosed(currentLog);
+      if (null != currentLog) {
+        try {
+          currentLog.close();
+        } catch (DfsLogger.LogClosedException ex) {
+          // ignore
+        } catch (Throwable ex) {
+          log.error("Unable to cleanly close log " + currentLog.getFileName() + ": " + ex, ex);
+        } finally {
+          this.tserver.walogClosed(currentLog);
+        }
+        currentLog = null;
+        logSizeEstimate.set(0);
       }
-      currentLog = null;
-      logSizeEstimate.set(0);
     } catch (Throwable t) {
       throw new IOException(t);
     }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/afa887b6/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
index b30578a..fb0adb8 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
@@ -933,8 +933,8 @@ public class Tablet implements TabletCommitter {
 
     long count = 0;
 
+    String oldName = Thread.currentThread().getName();
     try {
-      String oldName = Thread.currentThread().getName();
       Thread.currentThread().setName("Minor compacting " + this.extent);
       Span span = Trace.start("write");
       CompactionStats stats;
@@ -956,7 +956,6 @@ public class Tablet implements TabletCommitter {
             commitSession, flushId);
       } finally {
         span.stop();
-        Thread.currentThread().setName(oldName);
       }
       return new DataFileValue(stats.getFileSize(), stats.getEntriesWritten());
     } catch (Exception e) {
@@ -967,6 +966,7 @@ public class Tablet implements TabletCommitter {
       failed = true;
       throw new RuntimeException(e);
     } finally {
+      Thread.currentThread().setName(oldName);
       try {
         getTabletMemory().finalizeMinC();
       } catch (Throwable t) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/afa887b6/server/tserver/src/test/java/org/apache/accumulo/tserver/log/LogEntryTest.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/log/LogEntryTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/log/LogEntryTest.java
new file mode 100644
index 0000000..44058d3
--- /dev/null
+++ b/server/tserver/src/test/java/org/apache/accumulo/tserver/log/LogEntryTest.java
@@ -0,0 +1,56 @@
+/*
+ * 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.tserver.log;
+
+import static org.junit.Assert.assertEquals;
+
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.KeyExtent;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.tabletserver.log.LogEntry;
+import org.apache.hadoop.io.Text;
+import org.junit.Test;
+
+public class LogEntryTest {
+
+  @Test
+  public void test() throws Exception {
+    KeyExtent extent = new KeyExtent(new Text("1"), null, new Text(""));
+    long ts = 12345678L;
+    String server = "localhost:1234";
+    String filename = "default/foo";
+    LogEntry entry = new LogEntry(extent, ts, server, filename);
+    assertEquals(extent, entry.extent);
+    assertEquals(server, entry.server);
+    assertEquals(filename, entry.filename);
+    assertEquals(ts, entry.timestamp);
+    assertEquals("1<; default/foo", entry.toString());
+    assertEquals(new Text("log"), entry.getColumnFamily());
+    assertEquals(new Text("localhost:1234/default/foo"), entry.getColumnQualifier());
+    LogEntry copy = LogEntry.fromBytes(entry.toBytes());
+    assertEquals(entry.toString(), copy.toString());
+    Key key = new Key(new Text("1<"), new Text("log"), new Text("localhost:1234/default/foo"));
+    key.setTimestamp(ts);
+    LogEntry copy2 = LogEntry.fromKeyValue(key, entry.getValue());
+    assertEquals(entry.toString(), copy2.toString());
+    assertEquals(entry.timestamp, copy2.timestamp);
+    assertEquals("foo", entry.getUniqueID());
+    assertEquals("localhost:1234/default/foo", entry.getName());
+    assertEquals(new Value("default/foo".getBytes()), entry.getValue());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/afa887b6/test/src/main/java/org/apache/accumulo/test/UnusedWALIT.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/UnusedWALIT.java b/test/src/main/java/org/apache/accumulo/test/UnusedWALIT.java
new file mode 100644
index 0000000..3684ee1
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/UnusedWALIT.java
@@ -0,0 +1,144 @@
+/*
+ * 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.test;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.Map.Entry;
+
+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.Scanner;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.CurrentLogsSection;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.minicluster.ServerType;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.test.functional.ConfigurableMacIT;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.RawLocalFileSystem;
+import org.junit.Test;
+
+import com.google.common.collect.Iterators;
+
+// When reviewing the changes for ACCUMULO-3423, kturner suggested
+// "tablets will now have log references that contain no data,
+// so it may be marked with 3 WALs, the first with data, the 2nd without, a 3rd with data.
+// It would be useful to have an IT that will test this situation.
+public class UnusedWALIT extends ConfigurableMacIT {
+
+  @Override
+  protected void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
+    final long logSize = 1024 * 1024 * 10;
+    cfg.setProperty(Property.INSTANCE_ZK_TIMEOUT, "5s");
+    cfg.setProperty(Property.TSERV_WALOG_MAX_SIZE, Long.toString(logSize));
+    cfg.setNumTservers(1);
+    // use raw local file system so walogs sync and flush will work
+    hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
+    hadoopCoreSite.set("fs.namenode.fs-limits.min-block-size", Long.toString(logSize));
+  }
+
+  @Test(timeout = 2 * 60 * 1000)
+  public void test() throws Exception {
+    // don't want this bad boy cleaning up walog entries
+    getCluster().getClusterControl().stop(ServerType.GARBAGE_COLLECTOR);
+
+    // make two tables
+    String[] tableNames = getUniqueNames(2);
+    String bigTable = tableNames[0];
+    String lilTable = tableNames[1];
+    Connector c = getConnector();
+    c.tableOperations().create(bigTable);
+    c.tableOperations().create(lilTable);
+
+    // put some data in a log that should be replayed for both tables
+    writeSomeData(c, bigTable, 0, 10, 0, 10);
+    scanSomeData(c, bigTable, 0, 10, 0, 10);
+    writeSomeData(c, lilTable, 0, 1, 0, 1);
+    scanSomeData(c, lilTable, 0, 1, 0, 1);
+    assertEquals(1, getWALCount(c));
+
+    // roll the logs by pushing data into bigTable
+    writeSomeData(c, bigTable, 0, 3000, 0, 1000);
+    assertEquals(3, getWALCount(c));
+
+    // put some data in the latest log
+    writeSomeData(c, lilTable, 1, 10, 0, 10);
+    scanSomeData(c, lilTable, 1, 10, 0, 10);
+
+    // bounce the tserver
+    getCluster().getClusterControl().stop(ServerType.TABLET_SERVER);
+    getCluster().getClusterControl().start(ServerType.TABLET_SERVER);
+
+    // wait for the metadata table to be online
+    Iterators.size(c.createScanner(MetadataTable.NAME, Authorizations.EMPTY).iterator());
+
+    // check our two sets of data in different logs
+    scanSomeData(c, lilTable, 0, 1, 0, 1);
+    scanSomeData(c, lilTable, 1, 10, 0, 10);
+  }
+
+  private void scanSomeData(Connector c, String table, int startRow, int rowCount, int startCol, int colCount) throws Exception {
+    Scanner s = c.createScanner(table, Authorizations.EMPTY);
+    s.setRange(new Range(Integer.toHexString(startRow), Integer.toHexString(startRow + rowCount)));
+    int row = startRow;
+    int col = startCol;
+    for (Entry<Key,Value> entry : s) {
+      assertEquals(row, Integer.parseInt(entry.getKey().getRow().toString(), 16));
+      assertEquals(col++, Integer.parseInt(entry.getKey().getColumnQualifier().toString(), 16));
+      if (col == startCol + colCount) {
+        col = startCol;
+        row++;
+        if (row == startRow + rowCount) {
+          break;
+        }
+      }
+    }
+    assertEquals(row, startRow + rowCount);
+  }
+
+  private int getWALCount(Connector c) throws Exception {
+    Scanner s = c.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
+    s.setRange(CurrentLogsSection.getRange());
+    try {
+      return Iterators.size(s.iterator());
+    } finally {
+      s.close();
+    }
+  }
+
+  private void writeSomeData(Connector conn, String table, int startRow, int rowCount, int startCol, int colCount) throws Exception {
+    BatchWriterConfig config = new BatchWriterConfig();
+    config.setMaxMemory(10 * 1024 * 1024);
+    BatchWriter bw = conn.createBatchWriter(table, config);
+    for (int r = startRow; r < startRow + rowCount; r++) {
+      Mutation m = new Mutation(Integer.toHexString(r));
+      for (int c = startCol; c < startCol + colCount; c++) {
+        m.put("", Integer.toHexString(c), "");
+      }
+      bw.addMutation(m);
+    }
+    bw.close();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/afa887b6/test/src/test/java/org/apache/accumulo/test/VolumeIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/VolumeIT.java b/test/src/test/java/org/apache/accumulo/test/VolumeIT.java
index 7f1f921..7aeb135 100644
--- a/test/src/test/java/org/apache/accumulo/test/VolumeIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/VolumeIT.java
@@ -103,6 +103,7 @@ public class VolumeIT extends ConfigurableMacIT {
     cfg.setProperty(Property.INSTANCE_DFS_DIR, v1Uri.getPath());
     cfg.setProperty(Property.INSTANCE_DFS_URI, v1Uri.getScheme() + v1Uri.getHost());
     cfg.setProperty(Property.INSTANCE_VOLUMES, v1.toString() + "," + v2.toString());
+    cfg.setProperty(Property.INSTANCE_ZK_TIMEOUT, "5s");
 
     // use raw local file system so walogs sync and flush will work
     hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
@@ -425,6 +426,21 @@ public class VolumeIT extends ConfigurableMacIT {
       Assert.fail("Unexpected volume " + path);
     }
 
+    Text path = new Text();
+    for (String table : new String[]{RootTable.NAME, MetadataTable.NAME}) {
+      Scanner meta = conn.createScanner(table, Authorizations.EMPTY);
+      meta.setRange(MetadataSchema.CurrentLogsSection.getRange());
+      outer: for (Entry<Key,Value> entry : meta) {
+        MetadataSchema.CurrentLogsSection.getPath(entry.getKey(), path);
+        for (int i = 0; i < paths.length; i++) {
+          if (path.toString().startsWith(paths[i].toString())) {
+            continue outer;
+          }
+        }
+        Assert.fail("Unexpected volume " + path);
+      }
+    }
+
     // if a volume is chosen randomly for each tablet, then the probability that a volume will not be chosen for any tablet is ((num_volumes -
     // 1)/num_volumes)^num_tablets. For 100 tablets and 3 volumes the probability that only 2 volumes would be chosen is 2.46e-18
 
@@ -435,6 +451,7 @@ public class VolumeIT extends ConfigurableMacIT {
     }
 
     Assert.assertEquals(200, sum);
+
   }
 
   @Test


[27/34] accumulo git commit: ACCUMULO-3423 optimize WAL metadata table updates

Posted by ec...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/test/src/test/java/org/apache/accumulo/test/functional/WALSunnyDayIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/WALSunnyDayIT.java b/test/src/test/java/org/apache/accumulo/test/functional/WALSunnyDayIT.java
new file mode 100644
index 0000000..490bd7c
--- /dev/null
+++ b/test/src/test/java/org/apache/accumulo/test/functional/WALSunnyDayIT.java
@@ -0,0 +1,250 @@
+/*
+ * 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.test.functional;
+
+import static org.apache.accumulo.core.conf.Property.GC_CYCLE_DELAY;
+import static org.apache.accumulo.core.conf.Property.GC_CYCLE_START;
+import static org.apache.accumulo.core.conf.Property.INSTANCE_ZK_TIMEOUT;
+import static org.apache.accumulo.core.conf.Property.TSERV_WALOG_MAX_SIZE;
+import static org.apache.accumulo.core.conf.Property.TSERV_WAL_REPLICATION;
+import static org.apache.accumulo.core.security.Authorizations.EMPTY;
+import static org.apache.accumulo.minicluster.ServerType.GARBAGE_COLLECTOR;
+import static org.apache.accumulo.minicluster.ServerType.TABLET_SERVER;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Random;
+
+import org.apache.accumulo.core.client.BatchWriter;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.Scanner;
+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.Value;
+import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.RootTable;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.CurrentLogsSection;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.TabletColumnFamily;
+import org.apache.accumulo.core.util.UtilWaitThread;
+import org.apache.accumulo.core.zookeeper.ZooUtil;
+import org.apache.accumulo.master.state.SetGoalState;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloClusterControl;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloClusterImpl;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.RawLocalFileSystem;
+import org.apache.hadoop.io.Text;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.ZooKeeper;
+import org.junit.Assert;
+import org.junit.Test;
+
+import com.google.common.collect.Iterators;
+
+public class WALSunnyDayIT extends ConfigurableMacIT {
+
+  private static final Text CF = new Text(new byte[0]);
+
+  @Override
+  protected void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
+    cfg.setProperty(GC_CYCLE_DELAY, "1s");
+    cfg.setProperty(GC_CYCLE_START, "0s");
+    cfg.setProperty(TSERV_WALOG_MAX_SIZE, "1M");
+    cfg.setProperty(TSERV_WAL_REPLICATION, "1");
+    cfg.setProperty(INSTANCE_ZK_TIMEOUT, "3s");
+    cfg.setNumTservers(1);
+    hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
+  }
+
+  int countTrue(Collection<Boolean> bools) {
+    int result = 0;
+    for (Boolean b : bools) {
+      if (b.booleanValue())
+        result ++;
+    }
+    return result;
+  }
+
+  @Test
+  public void test() throws Exception {
+    MiniAccumuloClusterImpl mac = getCluster();
+    MiniAccumuloClusterControl control = mac.getClusterControl();
+    control.stop(GARBAGE_COLLECTOR);
+    Connector c = getConnector();
+    ZooKeeper zoo = new ZooKeeper(c.getInstance().getZooKeepers(), c.getInstance().getZooKeepersSessionTimeOut(), new Watcher() {
+      @Override
+      public void process(WatchedEvent event) {
+        log.info(event.toString());
+      }
+    });
+    String tableName = getUniqueNames(1)[0];
+    c.tableOperations().create(tableName);
+    writeSomeData(c, tableName, 1, 1);
+
+    // wal markers are added lazily
+    Map<String,Boolean> wals = getWals(c, zoo);
+    assertEquals(wals.toString(), 1, wals.size());
+    for (Boolean b : wals.values()) {
+      assertTrue("logs should be in use", b.booleanValue());
+    }
+
+    // roll log, get a new next
+    writeSomeData(c, tableName, 1000, 50);
+    Map<String,Boolean> walsAfterRoll = getWals(c, zoo);
+    assertEquals("should have 3 WALs after roll", 2, walsAfterRoll.size());
+    assertTrue("new WALs should be a superset of the old WALs", walsAfterRoll.keySet().containsAll(wals.keySet()));
+    assertEquals("all WALs should be in use", 2, countTrue(walsAfterRoll.values()));
+
+    // flush the tables
+    for (String table: new String[] { tableName, MetadataTable.NAME, RootTable.NAME} ) {
+      c.tableOperations().flush(table, null, null, true);
+    }
+    UtilWaitThread.sleep(1000);
+    // rolled WAL is no longer in use, but needs to be GC'd
+    Map<String,Boolean> walsAfterflush = getWals(c, zoo);
+    assertEquals(walsAfterflush.toString(), 2, walsAfterflush.size());
+    assertEquals("inUse should be 1", 1, countTrue(walsAfterflush.values()));
+
+    // let the GC run for a little bit
+    control.start(GARBAGE_COLLECTOR);
+    UtilWaitThread.sleep(5 * 1000);
+    // make sure the unused WAL goes away
+    Map<String,Boolean> walsAfterGC = getWals(c, zoo);
+    assertEquals(walsAfterGC.toString(), 1, walsAfterGC.size());
+    control.stop(GARBAGE_COLLECTOR);
+    // restart the tserver, but don't run recovery on all tablets
+    control.stop(TABLET_SERVER);
+    // this delays recovery on the normal tables
+    assertEquals(0, cluster.exec(SetGoalState.class, "SAFE_MODE").waitFor());
+    control.start(TABLET_SERVER);
+
+    // wait for the metadata table to go back online
+    getRecoveryMarkers(c);
+    // allow a little time for the master to notice ASSIGNED_TO_DEAD_SERVER tablets
+    UtilWaitThread.sleep(5 * 1000);
+    Map<KeyExtent,List<String>> markers = getRecoveryMarkers(c);
+    //log.debug("markers " + markers);
+    assertEquals("one tablet should have markers", 1, markers.keySet().size());
+    assertEquals("tableId of the keyExtent should be 1", markers.keySet().iterator().next().getTableId(), new Text("1"));
+
+    // put some data in the WAL
+    assertEquals(0, cluster.exec(SetGoalState.class, "NORMAL").waitFor());
+    verifySomeData(c, tableName, 1000 * 50 + 1);
+    writeSomeData(c, tableName, 100, 100);
+
+    Map<String,Boolean> walsAfterRestart = getWals(c, zoo);
+    //log.debug("wals after " + walsAfterRestart);
+    assertEquals("used WALs after restart should be 1", 1, countTrue(walsAfterRestart.values()));
+    control.start(GARBAGE_COLLECTOR);
+    UtilWaitThread.sleep(5 * 1000);
+    Map<String,Boolean> walsAfterRestartAndGC = getWals(c, zoo);
+    assertEquals("wals left should be 1", 1, walsAfterRestartAndGC.size());
+    assertEquals("logs in use should be 1", 1, countTrue(walsAfterRestartAndGC.values()));
+  }
+
+  private void verifySomeData(Connector c, String tableName, int expected) throws Exception {
+    Scanner scan = c.createScanner(tableName, EMPTY);
+    int result = Iterators.size(scan.iterator());
+    scan.close();
+    Assert.assertEquals(expected, result);
+  }
+
+  private void writeSomeData(Connector conn, String tableName, int row, int col) throws Exception {
+    Random rand = new Random();
+    BatchWriter bw = conn.createBatchWriter(tableName, null);
+    byte[] rowData = new byte[10];
+    byte[] cq = new byte[10];
+    byte[] value = new byte[10];
+
+    for (int r = 0; r < row; r++) {
+      rand.nextBytes(rowData);
+      Mutation m = new Mutation(rowData);
+      for (int c = 0; c < col; c++) {
+        rand.nextBytes(cq);
+        rand.nextBytes(value);
+        m.put(CF, new Text(cq), new Value(value));
+      }
+      bw.addMutation(m);
+      if (r % 100 == 0) {
+        bw.flush();
+      }
+    }
+    bw.close();
+  }
+
+  private Map<String, Boolean> getWals(Connector c, ZooKeeper zoo) throws Exception {
+    Map<String, Boolean> result = new HashMap<>();
+    Scanner root = c.createScanner(RootTable.NAME, EMPTY);
+    root.setRange(CurrentLogsSection.getRange());
+    Scanner meta = c.createScanner(MetadataTable.NAME, EMPTY);
+    meta.setRange(root.getRange());
+    Iterator<Entry<Key,Value>> both = Iterators.concat(root.iterator(), meta.iterator());
+    while (both.hasNext()) {
+      Entry<Key,Value> entry = both.next();
+      Text path = new Text();
+      CurrentLogsSection.getPath(entry.getKey(), path);
+      result.put(path.toString(), entry.getValue().get().length == 0);
+    }
+    String zpath = ZooUtil.getRoot(c.getInstance()) + RootTable.ZROOT_TABLET_CURRENT_LOGS;
+    List<String> children = zoo.getChildren(zpath, null);
+    for (String child : children) {
+      byte[] data = zoo.getData(zpath + "/" + child, null, null);
+      result.put(new String(data), true);
+    }
+    return result;
+  }
+
+  private Map<KeyExtent, List<String>> getRecoveryMarkers(Connector c) throws Exception {
+    Map<KeyExtent, List<String>> result = new HashMap<>();
+    Scanner root = c.createScanner(RootTable.NAME, EMPTY);
+    root.setRange(TabletsSection.getRange());
+    root.fetchColumnFamily(TabletsSection.LogColumnFamily.NAME);
+    TabletColumnFamily.PREV_ROW_COLUMN.fetch(root);
+
+    Scanner meta = c.createScanner(MetadataTable.NAME, EMPTY);
+    meta.setRange(TabletsSection.getRange());
+    meta.fetchColumnFamily(TabletsSection.LogColumnFamily.NAME);
+    TabletColumnFamily.PREV_ROW_COLUMN.fetch(meta);
+
+    List<String> logs = new ArrayList<>();
+    Iterator<Entry<Key,Value>> both = Iterators.concat(root.iterator(), meta.iterator());
+    while (both.hasNext()) {
+      Entry<Key,Value> entry = both.next();
+      Key key = entry.getKey();
+      if (key.getColumnFamily().equals(TabletsSection.LogColumnFamily.NAME)) {
+        logs.add(key.getColumnQualifier().toString());
+      }
+      if (TabletColumnFamily.PREV_ROW_COLUMN.hasColumns(key) && !logs.isEmpty()) {
+        KeyExtent extent = new KeyExtent(key.getRow(), entry.getValue());
+        result.put(extent, logs);
+        logs = new ArrayList<String>();
+      }
+    }
+    return result;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/test/src/test/java/org/apache/accumulo/test/functional/WatchTheWatchCountIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/WatchTheWatchCountIT.java b/test/src/test/java/org/apache/accumulo/test/functional/WatchTheWatchCountIT.java
index 3b1dd2f..140fd59 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/WatchTheWatchCountIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/WatchTheWatchCountIT.java
@@ -52,7 +52,7 @@ public class WatchTheWatchCountIT extends ConfigurableMacIT {
       String response = new String(buffer, 0, n);
       long total = Long.parseLong(response.split(":")[1].trim());
       assertTrue("Total watches was not greater than 500, but was " + total, total > 500);
-      assertTrue("Total watches was not less than 650, but was " + total, total < 600);
+      assertTrue("Total watches was not less than 675, but was " + total, total < 675);
     } finally {
       socket.close();
     }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/test/src/test/java/org/apache/accumulo/test/performance/RollWALPerformanceIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/performance/RollWALPerformanceIT.java b/test/src/test/java/org/apache/accumulo/test/performance/RollWALPerformanceIT.java
new file mode 100644
index 0000000..fcd1fd7
--- /dev/null
+++ b/test/src/test/java/org/apache/accumulo/test/performance/RollWALPerformanceIT.java
@@ -0,0 +1,126 @@
+/*
+ * 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.test.performance;
+
+import static org.junit.Assert.assertTrue;
+
+import java.util.SortedSet;
+import java.util.TreeSet;
+
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.RootTable;
+import org.apache.accumulo.minicluster.ServerType;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.minicluster.impl.ProcessReference;
+import org.apache.accumulo.test.continuous.ContinuousIngest;
+import org.apache.accumulo.test.functional.ConfigurableMacIT;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Text;
+import org.junit.Test;
+
+public class RollWALPerformanceIT extends ConfigurableMacIT {
+
+  @Override
+  protected void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
+    cfg.setProperty(Property.TSERV_WAL_REPLICATION, "1");
+    cfg.setProperty(Property.TSERV_WALOG_MAX_SIZE, "10M");
+    cfg.setProperty(Property.TABLE_MINC_LOGS_MAX, "100");
+    cfg.setProperty(Property.GC_FILE_ARCHIVE, "false");
+    cfg.setProperty(Property.GC_CYCLE_START, "1s");
+    cfg.setProperty(Property.GC_CYCLE_DELAY, "1s");
+    cfg.useMiniDFS(true);
+  }
+
+  private long ingest() throws Exception {
+    final Connector c = getConnector();
+    final String tableName = getUniqueNames(1)[0];
+
+    log.info("Creating the table");
+    c.tableOperations().create(tableName);
+
+    log.info("Splitting the table");
+    final long SPLIT_COUNT = 100;
+    final long distance = Long.MAX_VALUE / SPLIT_COUNT;
+    final SortedSet<Text> splits = new TreeSet<Text>();
+    for (int i = 1; i < SPLIT_COUNT; i++) {
+      splits.add(new Text(String.format("%016x", i * distance)));
+    }
+    c.tableOperations().addSplits(tableName, splits);
+
+    log.info("Waiting for balance");
+    c.instanceOperations().waitForBalance();
+
+    final Instance inst = c.getInstance();
+
+    log.info("Starting ingest");
+    final long start = System.currentTimeMillis();
+    final String args[] = {
+        "-i", inst.getInstanceName(),
+        "-z", inst.getZooKeepers(),
+        "-u", "root",
+        "-p", ROOT_PASSWORD,
+        "--batchThreads", "2",
+        "--table", tableName,
+        "--num", Long.toString(1000*1000),  // 1M 100 byte entries
+    };
+
+    ContinuousIngest.main(args);
+    final long result = System.currentTimeMillis() - start;
+    log.debug(String.format("Finished in %,d ms", result));
+    log.debug("Dropping table");
+    c.tableOperations().delete(tableName);
+    return result;
+  }
+
+  private long getAverage() throws Exception {
+    final int REPEAT = 3;
+    long totalTime = 0;
+    for (int i = 0; i < REPEAT; i++) {
+      totalTime += ingest();
+    }
+    return totalTime / REPEAT;
+  }
+
+  private void testWalPerformanceOnce() throws Exception {
+    // get time with a small WAL, which will cause many WAL roll-overs
+    long avg1 = getAverage();
+    // use a bigger WAL max size to eliminate WAL roll-overs
+    Connector c = getConnector();
+    c.instanceOperations().setProperty(Property.TSERV_WALOG_MAX_SIZE.getKey(), "1G");
+    c.tableOperations().flush(MetadataTable.NAME, null, null, true);
+    c.tableOperations().flush(RootTable.NAME, null, null, true);
+    for (ProcessReference  tserver : getCluster().getProcesses().get(ServerType.TABLET_SERVER)) {
+      getCluster().killProcess(ServerType.TABLET_SERVER, tserver);
+    }
+    getCluster().start();
+    long avg2 = getAverage();
+    log.info(String.format("Average run time with small WAL %,d with large WAL %,d", avg1, avg2));
+    assertTrue(avg1 > avg2);
+    double percent = (100. * avg1) / avg2;
+    log.info(String.format("Percent of large log: %.2f%%", percent));
+    assertTrue(percent < 125.);
+  }
+
+  @Test(timeout= 20 * 60 * 1000)
+  public void testWalPerformance() throws Exception {
+    testWalPerformanceOnce();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/test/src/test/java/org/apache/accumulo/test/replication/GarbageCollectorCommunicatesWithTServersIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/replication/GarbageCollectorCommunicatesWithTServersIT.java b/test/src/test/java/org/apache/accumulo/test/replication/GarbageCollectorCommunicatesWithTServersIT.java
index 75f61f1..62ed9c2 100644
--- a/test/src/test/java/org/apache/accumulo/test/replication/GarbageCollectorCommunicatesWithTServersIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/replication/GarbageCollectorCommunicatesWithTServersIT.java
@@ -39,6 +39,7 @@ import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.master.thrift.MasterClientService;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.CurrentLogsSection;
 import org.apache.accumulo.core.protobuf.ProtobufUtil;
 import org.apache.accumulo.core.replication.ReplicationTable;
 import org.apache.accumulo.core.rpc.ThriftUtil;
@@ -78,6 +79,7 @@ public class GarbageCollectorCommunicatesWithTServersIT extends ConfigurableMacI
   @Override
   public void configure(MiniAccumuloConfigImpl cfg, Configuration coreSite) {
     cfg.setNumTservers(1);
+    cfg.setProperty(Property.INSTANCE_ZK_TIMEOUT, "5s");
     cfg.setProperty(Property.GC_CYCLE_DELAY, GC_PERIOD_SECONDS + "s");
     // Wait longer to try to let the replication table come online before a cycle runs
     cfg.setProperty(Property.GC_CYCLE_START, "10s");
@@ -102,18 +104,14 @@ public class GarbageCollectorCommunicatesWithTServersIT extends ConfigurableMacI
     Assert.assertNotNull("Could not determine table ID for " + tableName, tableId);
 
     Scanner s = conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
-    Range r = MetadataSchema.TabletsSection.getRange(tableId);
-    s.setRange(r);
-    s.fetchColumnFamily(MetadataSchema.TabletsSection.LogColumnFamily.NAME);
+    s.setRange(CurrentLogsSection.getRange());
+    s.fetchColumnFamily(CurrentLogsSection.COLF);
 
     Set<String> wals = new HashSet<String>();
     for (Entry<Key,Value> entry : s) {
       log.debug("Reading WALs: {}={}", entry.getKey().toStringNoTruncate(), entry.getValue());
       // hostname:port/uri://path/to/wal
-      String cq = entry.getKey().getColumnQualifier().toString();
-      int index = cq.indexOf('/');
-      // Normalize the path
-      String path = new Path(cq.substring(index + 1)).toString();
+      String path = new Path(entry.getKey().getColumnQualifier().toString()).toString();
       log.debug("Extracted file: " + path);
       wals.add(path);
     }
@@ -228,11 +226,6 @@ public class GarbageCollectorCommunicatesWithTServersIT extends ConfigurableMacI
 
     Assert.assertEquals("Expected Status for file to not be closed", false, status.getClosed());
 
-    log.info("Checking to see that log entries are removed from tablet section after MinC");
-    // After compaction, the log column should be gone from the tablet
-    Set<String> walsAfterMinc = getWalsForTable(table);
-    Assert.assertEquals("Expected to find no WALs for tablet", 0, walsAfterMinc.size());
-
     Set<String> filesForTable = getFilesForTable(table);
     Assert.assertEquals("Expected to only find one rfile for table", 1, filesForTable.size());
     log.info("Files for table before MajC: {}", filesForTable);
@@ -258,14 +251,6 @@ public class GarbageCollectorCommunicatesWithTServersIT extends ConfigurableMacI
       fileExists = fs.exists(fileToBeDeleted);
     }
 
-    // At this point in time, we *know* that the GarbageCollector has run which means that the Status
-    // for our WAL should not be altered.
-
-    log.info("Re-checking that WALs are still not referenced for our table");
-
-    Set<String> walsAfterMajc = getWalsForTable(table);
-    Assert.assertEquals("Expected to find no WALs in tablets section: " + walsAfterMajc, 0, walsAfterMajc.size());
-
     Map<String,Status> fileToStatusAfterMinc = getMetadataStatusForTable(table);
     Assert.assertEquals("Expected to still find only one replication status message: " + fileToStatusAfterMinc, 1, fileToStatusAfterMinc.size());
 
@@ -326,11 +311,6 @@ public class GarbageCollectorCommunicatesWithTServersIT extends ConfigurableMacI
 
     Assert.assertEquals("Expected Status for file to not be closed", false, status.getClosed());
 
-    log.info("Checking to see that log entries are removed from tablet section after MinC");
-    // After compaction, the log column should be gone from the tablet
-    Set<String> walsAfterMinc = getWalsForTable(table);
-    Assert.assertEquals("Expected to find no WALs for tablet", 0, walsAfterMinc.size());
-
     Set<String> filesForTable = getFilesForTable(table);
     Assert.assertEquals("Expected to only find one rfile for table", 1, filesForTable.size());
     log.info("Files for table before MajC: {}", filesForTable);
@@ -359,11 +339,6 @@ public class GarbageCollectorCommunicatesWithTServersIT extends ConfigurableMacI
     // At this point in time, we *know* that the GarbageCollector has run which means that the Status
     // for our WAL should not be altered.
 
-    log.info("Re-checking that WALs are still not referenced for our table");
-
-    Set<String> walsAfterMajc = getWalsForTable(table);
-    Assert.assertEquals("Expected to find no WALs in tablets section: " + walsAfterMajc, 0, walsAfterMajc.size());
-
     Map<String,Status> fileToStatusAfterMinc = getMetadataStatusForTable(table);
     Assert.assertEquals("Expected to still find only one replication status message: " + fileToStatusAfterMinc, 1, fileToStatusAfterMinc.size());
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/test/src/test/java/org/apache/accumulo/test/replication/MultiInstanceReplicationIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/replication/MultiInstanceReplicationIT.java b/test/src/test/java/org/apache/accumulo/test/replication/MultiInstanceReplicationIT.java
index 9dec16e..7a017e1 100644
--- a/test/src/test/java/org/apache/accumulo/test/replication/MultiInstanceReplicationIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/replication/MultiInstanceReplicationIT.java
@@ -146,7 +146,7 @@ public class MultiInstanceReplicationIT extends ConfigurableMacIT {
     }
   }
 
-  @Test
+  @Test(timeout = 10 * 60 * 1000)
   public void dataWasReplicatedToThePeer() throws Exception {
     MiniAccumuloConfigImpl peerCfg = new MiniAccumuloConfigImpl(createTestDir(this.getClass().getName() + "_" + this.testName.getMethodName() + "_peer"),
         ROOT_PASSWORD);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/test/src/test/java/org/apache/accumulo/test/replication/ReplicationIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/replication/ReplicationIT.java b/test/src/test/java/org/apache/accumulo/test/replication/ReplicationIT.java
index 46e3ac1..3c1cbeb 100644
--- a/test/src/test/java/org/apache/accumulo/test/replication/ReplicationIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/replication/ReplicationIT.java
@@ -16,11 +16,12 @@
  */
 package org.apache.accumulo.test.replication;
 
+import static java.nio.charset.StandardCharsets.UTF_8;
+
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collection;
 import java.util.EnumSet;
 import java.util.HashSet;
 import java.util.Iterator;
@@ -45,6 +46,7 @@ import org.apache.accumulo.core.client.ZooKeeperInstance;
 import org.apache.accumulo.core.client.admin.TableOperations;
 import org.apache.accumulo.core.conf.Property;
 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.Range;
 import org.apache.accumulo.core.data.Value;
@@ -63,6 +65,7 @@ import org.apache.accumulo.core.replication.ReplicationTarget;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.TablePermission;
 import org.apache.accumulo.core.tabletserver.log.LogEntry;
+import org.apache.accumulo.core.util.AddressUtil;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.fate.zookeeper.ZooCache;
@@ -71,7 +74,7 @@ import org.apache.accumulo.fate.zookeeper.ZooLock;
 import org.apache.accumulo.gc.SimpleGarbageCollector;
 import org.apache.accumulo.minicluster.ServerType;
 import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
-import org.apache.accumulo.minicluster.impl.ProcessReference;
+import org.apache.accumulo.server.master.state.TServerInstance;
 import org.apache.accumulo.server.replication.ReplicaSystemFactory;
 import org.apache.accumulo.server.replication.StatusCombiner;
 import org.apache.accumulo.server.replication.StatusFormatter;
@@ -79,7 +82,6 @@ import org.apache.accumulo.server.replication.StatusUtil;
 import org.apache.accumulo.server.replication.proto.Replication.Status;
 import org.apache.accumulo.server.util.ReplicationTableUtil;
 import org.apache.accumulo.test.functional.ConfigurableMacIT;
-import org.apache.accumulo.tserver.TabletServer;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -123,25 +125,38 @@ public class ReplicationIT extends ConfigurableMacIT {
     cfg.setProperty(Property.REPLICATION_WORK_PROCESSOR_DELAY, "1s");
     cfg.setProperty(Property.REPLICATION_WORK_PROCESSOR_PERIOD, "1s");
     cfg.setProperty(Property.TSERV_TOTAL_MUTATION_QUEUE_MAX, "1M");
+    cfg.setProperty(Property.INSTANCE_ZK_TIMEOUT, "5s");
     cfg.setNumTservers(1);
     hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
   }
 
   private Multimap<String,String> getLogs(Connector conn) throws TableNotFoundException {
-    Multimap<String,String> logs = HashMultimap.create();
+    // Map of server to tableId
+    Multimap<TServerInstance, String> serverToTableID = HashMultimap.create();
     Scanner scanner = conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
-    scanner.fetchColumnFamily(LogColumnFamily.NAME);
-    scanner.setRange(new Range());
+    scanner.setRange(MetadataSchema.TabletsSection.getRange());
+    scanner.fetchColumnFamily(MetadataSchema.TabletsSection.CurrentLocationColumnFamily.NAME);
+    for (Entry<Key,Value> entry : scanner) {
+      TServerInstance key = new TServerInstance(entry.getValue(), entry.getKey().getColumnQualifier());
+      byte[] tableId = KeyExtent.tableOfMetadataRow(entry.getKey().getRow());
+      serverToTableID.put(key, new String(tableId, UTF_8));
+    }
+    // Map of logs to tableId
+    Multimap<String,String> logs = HashMultimap.create();
+    scanner = conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
+    scanner.setRange(MetadataSchema.CurrentLogsSection.getRange());
     for (Entry<Key,Value> entry : scanner) {
       if (Thread.interrupted()) {
         return logs;
       }
-
-      LogEntry logEntry = LogEntry.fromKeyValue(entry.getKey(), entry.getValue());
-
-      for (String log : logEntry.logSet) {
-        // Need to normalize the log file from LogEntry
-        logs.put(new Path(log).toString(), logEntry.extent.getTableId().toString());
+      Text path = new Text();
+      MetadataSchema.CurrentLogsSection.getPath(entry.getKey(), path);
+      Text session = new Text();
+      Text hostPort = new Text();
+      MetadataSchema.CurrentLogsSection.getTabletServer(entry.getKey(), hostPort , session);
+      TServerInstance server = new TServerInstance(AddressUtil.parseAddress(hostPort.toString(), false), session.toString());
+      for (String tableId : serverToTableID.get(server)) {
+        logs.put(new Path(path.toString()).toString(), tableId);
       }
     }
     return logs;
@@ -296,10 +311,12 @@ public class ReplicationIT extends ConfigurableMacIT {
     attempts = 5;
     while (wals.isEmpty() && attempts > 0) {
       s = conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
-      s.fetchColumnFamily(MetadataSchema.TabletsSection.LogColumnFamily.NAME);
+      s.setRange(MetadataSchema.CurrentLogsSection.getRange());
+      s.fetchColumnFamily(MetadataSchema.CurrentLogsSection.COLF);
       for (Entry<Key,Value> entry : s) {
-        LogEntry logEntry = LogEntry.fromKeyValue(entry.getKey(), entry.getValue());
-        wals.add(new Path(logEntry.filename).toString());
+        Text path = new Text();
+        MetadataSchema.CurrentLogsSection.getPath(entry.getKey(), path);
+        wals.add(new Path(path.toString()).toString());
       }
       attempts--;
     }
@@ -330,18 +347,7 @@ public class ReplicationIT extends ConfigurableMacIT {
     Assert.assertFalse(ReplicationTable.isOnline(conn));
 
     for (String table : tables) {
-      BatchWriter bw = conn.createBatchWriter(table, new BatchWriterConfig());
-
-      for (int j = 0; j < 5; j++) {
-        Mutation m = new Mutation(Integer.toString(j));
-        for (int k = 0; k < 5; k++) {
-          String value = Integer.toString(k);
-          m.put(value, "", value);
-        }
-        bw.addMutation(m);
-      }
-
-      bw.close();
+      writeSomeData(conn, table, 5, 5);
     }
 
     // After writing data, still no replication table
@@ -381,18 +387,7 @@ public class ReplicationIT extends ConfigurableMacIT {
     Assert.assertFalse(ReplicationTable.isOnline(conn));
 
     // Write some data to table1
-    BatchWriter bw = conn.createBatchWriter(table1, new BatchWriterConfig());
-
-    for (int rows = 0; rows < 50; rows++) {
-      Mutation m = new Mutation(Integer.toString(rows));
-      for (int cols = 0; cols < 50; cols++) {
-        String value = Integer.toString(cols);
-        m.put(value, "", value);
-      }
-      bw.addMutation(m);
-    }
-
-    bw.close();
+    writeSomeData(conn, table1, 50, 50);
 
     // After the commit for these mutations finishes, we'll get a replication entry in accumulo.metadata for table1
     // Don't want to compact table1 as it ultimately cause the entry in accumulo.metadata to be removed before we can verify it's there
@@ -439,18 +434,7 @@ public class ReplicationIT extends ConfigurableMacIT {
     conn.tableOperations().setProperty(table2, Property.TABLE_REPLICATION.getKey(), "true");
 
     // Write some data to table2
-    bw = conn.createBatchWriter(table2, new BatchWriterConfig());
-
-    for (int rows = 0; rows < 50; rows++) {
-      Mutation m = new Mutation(Integer.toString(rows));
-      for (int cols = 0; cols < 50; cols++) {
-        String value = Integer.toString(cols);
-        m.put(value, "", value);
-      }
-      bw.addMutation(m);
-    }
-
-    bw.close();
+    writeSomeData(conn, table2, 50, 50);
 
     // After the commit on these mutations, we'll get a replication entry in accumulo.metadata for table2
     // Don't want to compact table2 as it ultimately cause the entry in accumulo.metadata to be removed before we can verify it's there
@@ -498,6 +482,19 @@ public class ReplicationIT extends ConfigurableMacIT {
     Assert.assertFalse("Expected to only find two elements in replication table", iter.hasNext());
   }
 
+  private void writeSomeData(Connector conn, String table, int rows, int cols) throws Exception {
+    BatchWriter bw = conn.createBatchWriter(table, new BatchWriterConfig());
+    for (int row = 0; row < rows; row++) {
+      Mutation m = new Mutation(Integer.toString(row));
+      for (int col = 0; col < cols; col++) {
+        String value = Integer.toString(col);
+        m.put(value, "", value);
+      }
+      bw.addMutation(m);
+    }
+    bw.close();
+  }
+
   @Test
   public void replicationEntriesPrecludeWalDeletion() throws Exception {
     final Connector conn = getConnector();
@@ -529,53 +526,21 @@ public class ReplicationIT extends ConfigurableMacIT {
     Thread.sleep(2000);
 
     // Write some data to table1
-    BatchWriter bw = conn.createBatchWriter(table1, new BatchWriterConfig());
-    for (int rows = 0; rows < 200; rows++) {
-      Mutation m = new Mutation(Integer.toString(rows));
-      for (int cols = 0; cols < 500; cols++) {
-        String value = Integer.toString(cols);
-        m.put(value, "", value);
-      }
-      bw.addMutation(m);
-    }
-
-    bw.close();
+    writeSomeData(conn, table1, 200, 500);
 
     conn.tableOperations().create(table2);
     conn.tableOperations().setProperty(table2, Property.TABLE_REPLICATION.getKey(), "true");
     conn.tableOperations().setProperty(table2, Property.TABLE_REPLICATION_TARGET.getKey() + "cluster1", "1");
     Thread.sleep(2000);
 
-    // Write some data to table2
-    bw = conn.createBatchWriter(table2, new BatchWriterConfig());
-    for (int rows = 0; rows < 200; rows++) {
-      Mutation m = new Mutation(Integer.toString(rows));
-      for (int cols = 0; cols < 500; cols++) {
-        String value = Integer.toString(cols);
-        m.put(value, "", value);
-      }
-      bw.addMutation(m);
-    }
-
-    bw.close();
+    writeSomeData(conn, table2, 200, 500);
 
     conn.tableOperations().create(table3);
     conn.tableOperations().setProperty(table3, Property.TABLE_REPLICATION.getKey(), "true");
     conn.tableOperations().setProperty(table3, Property.TABLE_REPLICATION_TARGET.getKey() + "cluster1", "1");
     Thread.sleep(2000);
 
-    // Write some data to table3
-    bw = conn.createBatchWriter(table3, new BatchWriterConfig());
-    for (int rows = 0; rows < 200; rows++) {
-      Mutation m = new Mutation(Integer.toString(rows));
-      for (int cols = 0; cols < 500; cols++) {
-        String value = Integer.toString(cols);
-        m.put(value, "", value);
-      }
-      bw.addMutation(m);
-    }
-
-    bw.close();
+    writeSomeData(conn, table3, 200, 500);
 
     // Force a write to metadata for the data written
     for (String table : Arrays.asList(table1, table2, table3)) {
@@ -609,7 +574,8 @@ public class ReplicationIT extends ConfigurableMacIT {
 
     // We should have *some* reference to each log that was seen in the metadata table
     // They might not yet all be closed though (might be newfile)
-    Assert.assertEquals("Metadata log distribution: " + logs, logs.keySet(), replFiles);
+    Assert.assertTrue("Metadata log distribution: " + logs + "replFiles " + replFiles, logs.keySet().containsAll(replFiles));
+    Assert.assertTrue("Difference between replication entries and current logs is bigger than one", logs.keySet().size() - replFiles.size() <= 1);
 
     for (String replFile : replFiles) {
       Path p = new Path(replFile);
@@ -697,44 +663,11 @@ public class ReplicationIT extends ConfigurableMacIT {
       conn.tableOperations().setProperty(table3, Property.TABLE_REPLICATION.getKey(), "true");
       conn.tableOperations().setProperty(table3, Property.TABLE_REPLICATION_TARGET.getKey() + "cluster1", "1");
 
-      // Write some data to table1
-      BatchWriter bw = conn.createBatchWriter(table1, new BatchWriterConfig());
-      for (int rows = 0; rows < 200; rows++) {
-        Mutation m = new Mutation(Integer.toString(rows));
-        for (int cols = 0; cols < 500; cols++) {
-          String value = Integer.toString(cols);
-          m.put(value, "", value);
-        }
-        bw.addMutation(m);
-      }
+      writeSomeData(conn, table1, 200, 500);
 
-      bw.close();
+      writeSomeData(conn, table2, 200, 500);
 
-      // Write some data to table2
-      bw = conn.createBatchWriter(table2, new BatchWriterConfig());
-      for (int rows = 0; rows < 200; rows++) {
-        Mutation m = new Mutation(Integer.toString(rows));
-        for (int cols = 0; cols < 500; cols++) {
-          String value = Integer.toString(cols);
-          m.put(value, "", value);
-        }
-        bw.addMutation(m);
-      }
-
-      bw.close();
-
-      // Write some data to table3
-      bw = conn.createBatchWriter(table3, new BatchWriterConfig());
-      for (int rows = 0; rows < 200; rows++) {
-        Mutation m = new Mutation(Integer.toString(rows));
-        for (int cols = 0; cols < 500; cols++) {
-          String value = Integer.toString(cols);
-          m.put(value, "", value);
-        }
-        bw.addMutation(m);
-      }
-
-      bw.close();
+      writeSomeData(conn, table3, 200, 500);
 
       // Flush everything to try to make the replication records
       for (String table : Arrays.asList(table1, table2, table3)) {
@@ -789,10 +722,7 @@ public class ReplicationIT extends ConfigurableMacIT {
     Set<String> wals = new HashSet<>();
     for (Entry<Key,Value> entry : s) {
       LogEntry logEntry = LogEntry.fromKeyValue(entry.getKey(), entry.getValue());
-      for (String file : logEntry.logSet) {
-        Path p = new Path(file);
-        wals.add(p.toString());
-      }
+      wals.add(new Path(logEntry.filename).toString());
     }
 
     log.warn("Found wals {}", wals);
@@ -869,9 +799,7 @@ public class ReplicationIT extends ConfigurableMacIT {
   public void singleTableWithSingleTarget() throws Exception {
     // We want to kill the GC so it doesn't come along and close Status records and mess up the comparisons
     // against expected Status messages.
-    for (ProcessReference proc : cluster.getProcesses().get(ServerType.GARBAGE_COLLECTOR)) {
-      cluster.killProcess(ServerType.GARBAGE_COLLECTOR, proc);
-    }
+    getCluster().getClusterControl().stop(ServerType.GARBAGE_COLLECTOR);
 
     Connector conn = getConnector();
     String table1 = "table1";
@@ -905,17 +833,7 @@ public class ReplicationIT extends ConfigurableMacIT {
     }
 
     // Write some data to table1
-    BatchWriter bw = conn.createBatchWriter(table1, new BatchWriterConfig());
-    for (int rows = 0; rows < 2000; rows++) {
-      Mutation m = new Mutation(Integer.toString(rows));
-      for (int cols = 0; cols < 50; cols++) {
-        String value = Integer.toString(cols);
-        m.put(value, "", value);
-      }
-      bw.addMutation(m);
-    }
-
-    bw.close();
+    writeSomeData(conn, table1, 2000, 50);
 
     // Make sure the replication table is online at this point
     boolean online = ReplicationTable.isOnline(conn);
@@ -1002,17 +920,7 @@ public class ReplicationIT extends ConfigurableMacIT {
     }
 
     // Write some more data so that we over-run the single WAL
-    bw = conn.createBatchWriter(table1, new BatchWriterConfig());
-    for (int rows = 0; rows < 3000; rows++) {
-      Mutation m = new Mutation(Integer.toString(rows));
-      for (int cols = 0; cols < 50; cols++) {
-        String value = Integer.toString(cols);
-        m.put(value, "", value);
-      }
-      bw.addMutation(m);
-    }
-
-    bw.close();
+    writeSomeData(conn, table1, 3000, 50);
 
     log.info("Issued compaction for table");
     conn.tableOperations().compact(table1, null, null, true, true);
@@ -1085,17 +993,7 @@ public class ReplicationIT extends ConfigurableMacIT {
     }
 
     // Write some data to table1
-    BatchWriter bw = conn.createBatchWriter(table1, new BatchWriterConfig());
-    for (int rows = 0; rows < 2000; rows++) {
-      Mutation m = new Mutation(Integer.toString(rows));
-      for (int cols = 0; cols < 50; cols++) {
-        String value = Integer.toString(cols);
-        m.put(value, "", value);
-      }
-      bw.addMutation(m);
-    }
-
-    bw.close();
+    writeSomeData(conn, table1, 2000, 50);
     conn.tableOperations().flush(table1, null, null, true);
 
     String tableId = conn.tableOperations().tableIdMap().get(table1);
@@ -1151,10 +1049,7 @@ public class ReplicationIT extends ConfigurableMacIT {
 
   @Test
   public void replicationRecordsAreClosedAfterGarbageCollection() throws Exception {
-    Collection<ProcessReference> gcProcs = cluster.getProcesses().get(ServerType.GARBAGE_COLLECTOR);
-    for (ProcessReference ref : gcProcs) {
-      cluster.killProcess(ServerType.GARBAGE_COLLECTOR, ref);
-    }
+    getCluster().getClusterControl().stop(ServerType.GARBAGE_COLLECTOR);
 
     final Connector conn = getConnector();
 
@@ -1185,7 +1080,6 @@ public class ReplicationIT extends ConfigurableMacIT {
 
     String table1 = "table1", table2 = "table2", table3 = "table3";
 
-    BatchWriter bw;
     try {
       conn.tableOperations().create(table1);
       conn.tableOperations().setProperty(table1, Property.TABLE_REPLICATION.getKey(), "true");
@@ -1194,51 +1088,19 @@ public class ReplicationIT extends ConfigurableMacIT {
           ReplicaSystemFactory.getPeerConfigurationValue(MockReplicaSystem.class, null));
 
       // Write some data to table1
-      bw = conn.createBatchWriter(table1, new BatchWriterConfig());
-      for (int rows = 0; rows < 200; rows++) {
-        Mutation m = new Mutation(Integer.toString(rows));
-        for (int cols = 0; cols < 500; cols++) {
-          String value = Integer.toString(cols);
-          m.put(value, "", value);
-        }
-        bw.addMutation(m);
-      }
-
-      bw.close();
+      writeSomeData(conn, table1, 200, 500);
 
       conn.tableOperations().create(table2);
       conn.tableOperations().setProperty(table2, Property.TABLE_REPLICATION.getKey(), "true");
       conn.tableOperations().setProperty(table2, Property.TABLE_REPLICATION_TARGET.getKey() + "cluster1", "1");
 
-      // Write some data to table2
-      bw = conn.createBatchWriter(table2, new BatchWriterConfig());
-      for (int rows = 0; rows < 200; rows++) {
-        Mutation m = new Mutation(Integer.toString(rows));
-        for (int cols = 0; cols < 500; cols++) {
-          String value = Integer.toString(cols);
-          m.put(value, "", value);
-        }
-        bw.addMutation(m);
-      }
-
-      bw.close();
+      writeSomeData(conn, table2, 200, 500);
 
       conn.tableOperations().create(table3);
       conn.tableOperations().setProperty(table3, Property.TABLE_REPLICATION.getKey(), "true");
       conn.tableOperations().setProperty(table3, Property.TABLE_REPLICATION_TARGET.getKey() + "cluster1", "1");
 
-      // Write some data to table3
-      bw = conn.createBatchWriter(table3, new BatchWriterConfig());
-      for (int rows = 0; rows < 200; rows++) {
-        Mutation m = new Mutation(Integer.toString(rows));
-        for (int cols = 0; cols < 500; cols++) {
-          String value = Integer.toString(cols);
-          m.put(value, "", value);
-        }
-        bw.addMutation(m);
-      }
-
-      bw.close();
+      writeSomeData(conn, table3, 200, 500);
 
       // Flush everything to try to make the replication records
       for (String table : Arrays.asList(table1, table2, table3)) {
@@ -1252,11 +1114,8 @@ public class ReplicationIT extends ConfigurableMacIT {
 
     // Kill the tserver(s) and restart them
     // to ensure that the WALs we previously observed all move to closed.
-    for (ProcessReference proc : cluster.getProcesses().get(ServerType.TABLET_SERVER)) {
-      cluster.killProcess(ServerType.TABLET_SERVER, proc);
-    }
-
-    cluster.exec(TabletServer.class);
+    cluster.getClusterControl().stop(ServerType.TABLET_SERVER);
+    cluster.getClusterControl().start(ServerType.TABLET_SERVER);
 
     // Make sure we can read all the tables (recovery complete)
     for (String table : Arrays.asList(table1, table2, table3)) {
@@ -1359,9 +1218,7 @@ public class ReplicationIT extends ConfigurableMacIT {
   @Test
   public void replicatedStatusEntriesAreDeleted() throws Exception {
     // Just stop it now, we'll restart it after we restart the tserver
-    for (ProcessReference proc : getCluster().getProcesses().get(ServerType.GARBAGE_COLLECTOR)) {
-      getCluster().killProcess(ServerType.GARBAGE_COLLECTOR, proc);
-    }
+    getCluster().getClusterControl().stop(ServerType.GARBAGE_COLLECTOR);
 
     final Connector conn = getConnector();
     log.info("Got connector to MAC");
@@ -1397,17 +1254,7 @@ public class ReplicationIT extends ConfigurableMacIT {
     Assert.assertNotNull("Could not determine table id for " + table1, tableId);
 
     // Write some data to table1
-    BatchWriter bw = conn.createBatchWriter(table1, new BatchWriterConfig());
-    for (int rows = 0; rows < 2000; rows++) {
-      Mutation m = new Mutation(Integer.toString(rows));
-      for (int cols = 0; cols < 50; cols++) {
-        String value = Integer.toString(cols);
-        m.put(value, "", value);
-      }
-      bw.addMutation(m);
-    }
-
-    bw.close();
+    writeSomeData(conn, table1, 2000, 50);
     conn.tableOperations().flush(table1, null, null, true);
 
     // Make sure the replication table exists at this point
@@ -1425,14 +1272,35 @@ public class ReplicationIT extends ConfigurableMacIT {
     // Grant ourselves the write permission for later
     conn.securityOperations().grantTablePermission("root", ReplicationTable.NAME, TablePermission.WRITE);
 
+    log.info("Checking for replication entries in replication");
+    // Then we need to get those records over to the replication table
+    Scanner s;
+    Set<String> entries = new HashSet<>();
+    for (int i = 0; i < 5; i++) {
+      s = conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
+      s.setRange(ReplicationSection.getRange());
+      entries.clear();
+      for (Entry<Key,Value> entry : s) {
+        entries.add(entry.getKey().getRow().toString());
+        log.info("{}={}", entry.getKey().toStringNoTruncate(), entry.getValue());
+      }
+      if (!entries.isEmpty()) {
+        log.info("Replication entries {}", entries);
+        break;
+      }
+      Thread.sleep(1000);
+    }
+
+    Assert.assertFalse("Did not find any replication entries in the replication table", entries.isEmpty());
+
     // Find the WorkSection record that will be created for that data we ingested
     boolean notFound = true;
-    Scanner s;
     for (int i = 0; i < 10 && notFound; i++) {
       try {
         s = ReplicationTable.getScanner(conn);
         WorkSection.limit(s);
         Entry<Key,Value> e = Iterables.getOnlyElement(s);
+        log.info("Found entry: " + e.getKey().toStringNoTruncate());
         Text expectedColqual = new ReplicationTarget("cluster1", "4", tableId).toText();
         Assert.assertEquals(expectedColqual, e.getKey().getColumnQualifier());
         notFound = false;
@@ -1483,14 +1351,13 @@ public class ReplicationIT extends ConfigurableMacIT {
     log.info("Killing tserver");
     // Kill the tserver(s) and restart them
     // to ensure that the WALs we previously observed all move to closed.
-    for (ProcessReference proc : cluster.getProcesses().get(ServerType.TABLET_SERVER)) {
-      cluster.killProcess(ServerType.TABLET_SERVER, proc);
-    }
+    cluster.getClusterControl().stop(ServerType.TABLET_SERVER);
 
     log.info("Starting tserver");
-    cluster.exec(TabletServer.class);
+    cluster.getClusterControl().start(ServerType.TABLET_SERVER);
 
     log.info("Waiting to read tables");
+    UtilWaitThread.sleep(2 * 3 * 1000);
 
     // Make sure we can read all the tables (recovery complete)
     for (String table : new String[] {MetadataTable.NAME, table1}) {
@@ -1499,55 +1366,48 @@ public class ReplicationIT extends ConfigurableMacIT {
       Entry<Key,Value> entry : s) {}
     }
 
-    log.info("Checking for replication entries in replication");
-    // Then we need to get those records over to the replication table
-    boolean foundResults = false;
-    for (int i = 0; i < 5; i++) {
-      s = ReplicationTable.getScanner(conn);
-      int count = 0;
-      for (Entry<Key,Value> entry : s) {
-        count++;
-        log.info("{}={}", entry.getKey().toStringNoTruncate(), entry.getValue());
-      }
-      if (count > 0) {
-        foundResults = true;
-        break;
-      }
-      Thread.sleep(1000);
+    log.info("Recovered metadata:");
+    s = conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
+    for (Entry<Key,Value> entry : s) {
+      log.info("{}={}", entry.getKey().toStringNoTruncate(), entry.getValue());
     }
 
-    Assert.assertTrue("Did not find any replication entries in the replication table", foundResults);
-
-    getCluster().exec(SimpleGarbageCollector.class);
+    cluster.getClusterControl().start(ServerType.GARBAGE_COLLECTOR);
 
     // Wait for a bit since the GC has to run (should be running after a one second delay)
     waitForGCLock(conn);
 
     Thread.sleep(1000);
 
+    log.info("After GC");
+    s = conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
+    for (Entry<Key,Value> entry : s) {
+      log.info("{}={}", entry.getKey().toStringNoTruncate(), entry.getValue());
+    }
+
     // We expect no records in the metadata table after compaction. We have to poll
     // because we have to wait for the StatusMaker's next iteration which will clean
     // up the dangling *closed* records after we create the record in the replication table.
     // We need the GC to close the file (CloseWriteAheadLogReferences) before we can remove the record
     log.info("Checking metadata table for replication entries");
-    foundResults = true;
+    Set<String> remaining = new HashSet<>();
     for (int i = 0; i < 10; i++) {
       s = conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
       s.setRange(ReplicationSection.getRange());
-      long size = 0;
+      remaining.clear();
       for (Entry<Key,Value> e : s) {
-        size++;
-        log.info("{}={}", e.getKey().toStringNoTruncate(), ProtobufUtil.toString(Status.parseFrom(e.getValue().get())));
+        remaining.add(e.getKey().getRow().toString());
       }
-      if (size == 0) {
-        foundResults = false;
+      remaining.retainAll(entries);
+      if (remaining.isEmpty()) {
         break;
       }
+      log.info("remaining {}", remaining);
       Thread.sleep(2000);
       log.info("");
     }
 
-    Assert.assertFalse("Replication status messages were not cleaned up from metadata table", foundResults);
+    Assert.assertTrue("Replication status messages were not cleaned up from metadata table", remaining.isEmpty());
 
     /**
      * After we close out and subsequently delete the metadata record, this will propagate to the replication table, which will cause those records to be
@@ -1560,10 +1420,10 @@ public class ReplicationIT extends ConfigurableMacIT {
       recordsFound = 0;
       for (Entry<Key,Value> entry : s) {
         recordsFound++;
-        log.info(entry.getKey().toStringNoTruncate() + " " + ProtobufUtil.toString(Status.parseFrom(entry.getValue().get())));
+        log.info("{} {}", entry.getKey().toStringNoTruncate(), ProtobufUtil.toString(Status.parseFrom(entry.getValue().get())));
       }
 
-      if (0 == recordsFound) {
+      if (recordsFound <= 2) {
         break;
       } else {
         Thread.sleep(1000);
@@ -1571,6 +1431,6 @@ public class ReplicationIT extends ConfigurableMacIT {
       }
     }
 
-    Assert.assertEquals("Found unexpected replication records in the replication table", 0, recordsFound);
+    Assert.assertTrue("Found unexpected replication records in the replication table", recordsFound <= 2);
   }
 }


[14/34] accumulo git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/accumulo

Posted by ec...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/accumulo


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

Branch: refs/heads/master
Commit: daa38ce90f41d12aa28607963e8c512f1f049522
Parents: 34af43f 4ca3143
Author: Eric C. Newton <er...@gmail.com>
Authored: Mon Mar 23 12:53:50 2015 -0400
Committer: Eric C. Newton <er...@gmail.com>
Committed: Mon Mar 23 12:53:50 2015 -0400

----------------------------------------------------------------------
 .../accumulo/monitor/servlets/trace/Basic.java  | 85 +++++++++++++++-----
 .../monitor/servlets/trace/ListType.java        | 26 +++++-
 .../monitor/servlets/trace/ShowTrace.java       | 30 +++++--
 .../monitor/servlets/trace/Summary.java         | 34 +++++---
 .../org/apache/accumulo/tracer/TraceServer.java | 31 ++++++-
 5 files changed, 164 insertions(+), 42 deletions(-)
----------------------------------------------------------------------



[03/34] accumulo git commit: ACCUMULO-3625 use log markers against tservers, not tablets

Posted by ec...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/b2539fb1/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java
----------------------------------------------------------------------
diff --git 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
index ed7626e..a95cffa 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
@@ -23,8 +23,6 @@ import static org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSec
 
 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;
@@ -60,6 +58,7 @@ 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.CurrentLogsSection;
 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;
@@ -86,6 +85,7 @@ 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.VolumeManagerImpl;
+import org.apache.accumulo.server.master.state.TServerInstance;
 import org.apache.accumulo.server.tablets.TabletTime;
 import org.apache.accumulo.server.zookeeper.ZooLock;
 import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
@@ -120,7 +120,7 @@ public class MetadataTableUtil {
     return metadataTable;
   }
 
-  private synchronized static Writer getRootTable(ClientContext context) {
+  public synchronized static Writer getRootTable(ClientContext context) {
     Credentials credentials = context.getCredentials();
     Writer rootTable = root_tables.get(credentials);
     if (rootTable == null) {
@@ -227,7 +227,7 @@ public class MetadataTableUtil {
 
       // add before removing in case of process death
       for (LogEntry logEntry : logsToAdd)
-        addLogEntry(context, logEntry, zooLock);
+        addRootLogEntry(context, zooLock, logEntry);
 
       removeUnusedWALEntries(context, extent, logsToRemove, zooLock);
     } else {
@@ -252,6 +252,39 @@ public class MetadataTableUtil {
     }
   }
 
+  private static interface ZooOperation {
+    void run(IZooReaderWriter rw) throws KeeperException, InterruptedException, IOException;
+  }
+
+  private static void retryZooKeeperUpdate(ClientContext context, ZooLock zooLock, ZooOperation op) {
+    while (true) {
+      try {
+        IZooReaderWriter zoo = ZooReaderWriter.getInstance();
+        if (zoo.isLockHeld(zooLock.getLockID())) {
+          op.run(zoo);
+        }
+        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);
+    }
+  }
+
+  private static void addRootLogEntry(AccumuloServerContext context, ZooLock zooLock, final LogEntry entry) {
+    retryZooKeeperUpdate(context, zooLock, new ZooOperation() {
+      @Override
+      public void run(IZooReaderWriter rw) throws KeeperException, InterruptedException, IOException {
+        String root = getZookeeperLogLocation();
+        rw.putPersistentData(root + "/" + entry.getUniqueID(), entry.toBytes(), NodeExistsPolicy.OVERWRITE);
+      }
+    });
+  }
+
   public static SortedMap<FileRef,DataFileValue> getDataFileSizes(KeyExtent extent, ClientContext context) throws IOException {
     TreeMap<FileRef,DataFileValue> sizes = new TreeMap<FileRef,DataFileValue>();
 
@@ -451,34 +484,6 @@ public class MetadataTableUtil {
     return ZooUtil.getRoot(HdfsZooInstance.getInstance()) + RootTable.ZROOT_TABLET_WALOGS;
   }
 
-  public static void addLogEntry(ClientContext context, 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(context, zooLock, m, entry.extent);
-    }
-  }
-
   public static void setRootTabletDir(String dir) throws IOException {
     IZooReaderWriter zoo = ZooReaderWriter.getInstance();
     String zpath = ZooUtil.getRoot(HdfsZooInstance.getInstance()) + RootTable.ZROOT_TABLET_PATH;
@@ -569,22 +574,11 @@ public class MetadataTableUtil {
       }
     }
 
-    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;
   }
 
-  static void getRootLogEntries(ArrayList<LogEntry> result) throws KeeperException, InterruptedException, IOException {
+  static void getRootLogEntries(final 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
@@ -592,11 +586,10 @@ public class MetadataTableUtil {
     while (true) {
       result.clear();
       for (String child : zoo.getChildren(root)) {
-        LogEntry e = new LogEntry();
         try {
-          e.fromBytes(zoo.getData(root + "/" + child, null));
+          LogEntry e = LogEntry.fromBytes(zoo.getData(root + "/" + child, null));
           // upgrade from !0;!0<< -> +r<<
-          e.extent = RootTable.EXTENT;
+          e = new LogEntry(RootTable.EXTENT, 0, e.server, e.filename);
           result.add(e);
         } catch (KeeperException.NoNodeException ex) {
           continue;
@@ -666,28 +659,23 @@ public class MetadataTableUtil {
     return new LogEntryIterator(context);
   }
 
-  public static void removeUnusedWALEntries(AccumuloServerContext context, KeyExtent extent, List<LogEntry> logEntries, ZooLock zooLock) {
+  public static void removeUnusedWALEntries(AccumuloServerContext context, KeyExtent extent, final List<LogEntry> entries, ZooLock zooLock) {
     if (extent.isRootTablet()) {
-      for (LogEntry entry : logEntries) {
-        String root = getZookeeperLogLocation();
-        while (true) {
-          try {
-            IZooReaderWriter zoo = ZooReaderWriter.getInstance();
-            if (zoo.isLockHeld(zooLock.getLockID())) {
-              String parts[] = entry.filename.split("/");
-              zoo.recursiveDelete(root + "/" + parts[parts.length - 1], NodeMissingPolicy.SKIP);
-            }
-            break;
-          } catch (Exception e) {
-            log.error(e, e);
+      retryZooKeeperUpdate(context, zooLock, new ZooOperation() {
+        @Override
+        public void run(IZooReaderWriter rw) throws KeeperException, InterruptedException, IOException {
+          String root = getZookeeperLogLocation();
+          for (LogEntry entry : entries) {
+            String path = root + "/" + entry.getUniqueID();
+            log.debug("Removing " + path + " from zookeeper");
+            rw.recursiveDelete(path, NodeMissingPolicy.SKIP);
           }
-          UtilWaitThread.sleep(1000);
         }
-      }
+      });
     } else {
       Mutation m = new Mutation(extent.getMetadataEntry());
-      for (LogEntry entry : logEntries) {
-        m.putDelete(LogColumnFamily.NAME, new Text(entry.getName()));
+      for (LogEntry entry : entries) {
+        m.putDelete(entry.getColumnFamily(), entry.getColumnQualifier());
       }
       update(context, zooLock, m, extent);
     }
@@ -1072,4 +1060,106 @@ public class MetadataTableUtil {
     return tabletEntries;
   }
 
+  public static void addNewLogMarker(ClientContext context, ZooLock zooLock, final TServerInstance tabletSession, final String filename, KeyExtent extent) {
+    log.debug("Adding log entry " + filename);
+    if (extent.isRootTablet()) {
+      retryZooKeeperUpdate(context, zooLock, new ZooOperation() {
+        @Override
+        public void run(IZooReaderWriter rw) throws KeeperException, InterruptedException, IOException {
+          String root = ZooUtil.getRoot(HdfsZooInstance.getInstance()) + RootTable.ZROOT_TABLET_CURRENT_LOGS;
+          String[] parts = filename.split("/");
+          String uniqueId = parts[parts.length - 1];
+          String path = root + "/" + CurrentLogsSection.getRowPrefix() + tabletSession.toString() + uniqueId;
+          rw.putPersistentData(path, filename.getBytes(UTF_8), NodeExistsPolicy.OVERWRITE);
+        }
+      });
+    } else {
+      Mutation m = new Mutation(CurrentLogsSection.getRowPrefix() + tabletSession.toString());
+      m.put("log", filename, new Value(EMPTY_BYTES));
+      String tableName = MetadataTable.NAME;
+      if (extent.isMeta()) {
+        tableName = RootTable.NAME;
+      }
+      try {
+        BatchWriter bw = context.getConnector().createBatchWriter(tableName, null);
+        bw.addMutation(m);
+        bw.close();
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      }
+    }
+  }
+
+  private static void removeCurrentRootLogMarker(ClientContext context, ZooLock zooLock, final TServerInstance tabletSession, final String filename) {
+    retryZooKeeperUpdate(context, zooLock, new ZooOperation() {
+      @Override
+      public void run(IZooReaderWriter rw) throws KeeperException, InterruptedException, IOException {
+        String root = ZooUtil.getRoot(HdfsZooInstance.getInstance()) + RootTable.ZROOT_TABLET_CURRENT_LOGS;
+        String[] parts = filename.split("/");
+        String uniqueId = parts[parts.length - 1];
+        String path = root + "/" + CurrentLogsSection.getRowPrefix() + tabletSession.toString() + uniqueId;
+        log.debug("Removing entry " + path + " from zookeeper");
+        rw.recursiveDelete(path, NodeMissingPolicy.SKIP);
+      }
+    });
+  }
+
+  public static void markLogUnused(ClientContext context, ZooLock lock, TServerInstance tabletSession, Set<String> all) throws AccumuloException {
+    try {
+      BatchWriter root = context.getConnector().createBatchWriter(RootTable.NAME, null);
+      BatchWriter meta = context.getConnector().createBatchWriter(MetadataTable.NAME, null);
+      for (String fname : all) {
+        Text tname = new Text(fname.getBytes(UTF_8));
+        Mutation m = new Mutation(MetadataSchema.CurrentLogsSection.getRowPrefix() + tabletSession.toString());
+        m.putDelete(MetadataSchema.CurrentLogsSection.COLF, tname);
+        root.addMutation(m);
+        log.debug("deleting " + MetadataSchema.CurrentLogsSection.getRowPrefix() + tabletSession.toString() + " log:" + fname);
+        m = new Mutation(MetadataSchema.CurrentLogsSection.getRowPrefix() + tabletSession.toString());
+        m.put(MetadataSchema.CurrentLogsSection.COLF, tname, MetadataSchema.CurrentLogsSection.UNUSED);
+        meta.addMutation(m);
+        removeCurrentRootLogMarker(context, lock, tabletSession, fname);
+      }
+      root.close();
+      meta.close();
+    } catch (Exception ex) {
+      throw new AccumuloException(ex);
+    }
+  }
+
+  public static void fetchLogsForDeadServer(ClientContext context, ZooLock lock, KeyExtent extent, TServerInstance server, Map<TServerInstance,List<String>> logsForDeadServers)
+      throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
+    // already cached
+    if (logsForDeadServers.containsKey(server)) {
+      return;
+    }
+    if (extent.isRootTablet()) {
+      final List<String> logs = new ArrayList<>();
+      retryZooKeeperUpdate(context, lock, new ZooOperation() {
+        @Override
+        public void run(IZooReaderWriter rw) throws KeeperException, InterruptedException, IOException {
+          String root = ZooUtil.getRoot(HdfsZooInstance.getInstance()) + RootTable.ZROOT_TABLET_CURRENT_LOGS;
+          logs.clear();
+          for (String child : rw.getChildren(root)) {
+            logs.add(new String(rw.getData(root + "/" + child, null), UTF_8));
+          }
+        }
+      });
+      logsForDeadServers.put(server, logs);
+    } else {
+      // use the correct meta table
+      String table = MetadataTable.NAME;
+      if (extent.isMeta()) {
+        table = RootTable.NAME;
+      }
+      // fetch the current logs in use, and put them in the cache
+      Scanner scanner = context.getConnector().createScanner(table, Authorizations.EMPTY);
+      scanner.setRange(new Range(MetadataSchema.CurrentLogsSection.getRowPrefix() + server.toString()));
+      List<String> logs = new ArrayList<>();
+      for (Entry<Key,Value> entry : scanner) {
+        logs.add(entry.getKey().getColumnQualifier().toString());
+      }
+      logsForDeadServers.put(server, logs);
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b2539fb1/server/base/src/main/java/org/apache/accumulo/server/util/ReplicationTableUtil.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/ReplicationTableUtil.java b/server/base/src/main/java/org/apache/accumulo/server/util/ReplicationTableUtil.java
index 344e245..0de0b0e 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/ReplicationTableUtil.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/ReplicationTableUtil.java
@@ -16,7 +16,6 @@
  */
 package org.apache.accumulo.server.util;
 
-import java.util.Collection;
 import java.util.Collections;
 import java.util.EnumSet;
 import java.util.HashMap;
@@ -176,20 +175,14 @@ public class ReplicationTableUtil {
   /**
    * Write replication ingest entries for each provided file with the given {@link Status}.
    */
-  public static void updateFiles(ClientContext context, KeyExtent extent, Collection<String> files, Status stat) {
+  public static void updateFiles(ClientContext context, KeyExtent extent, String file, Status stat) {
     if (log.isDebugEnabled()) {
-      log.debug("Updating replication status for " + extent + " with " + files + " using " + ProtobufUtil.toString(stat));
+      log.debug("Updating replication status for " + extent + " with " + file + " using " + ProtobufUtil.toString(stat));
     }
     // TODO could use batch writer, would need to handle failure and retry like update does - ACCUMULO-1294
-    if (files.isEmpty()) {
-      return;
-    }
 
     Value v = ProtobufUtil.toValue(stat);
-    for (String file : files) {
-      // TODO Can preclude this addition if the extent is for a table we don't need to replicate
-      update(context, createUpdateMutation(new Path(file), v, extent), extent);
-    }
+    update(context, createUpdateMutation(new Path(file), v, extent), extent);
   }
 
   static Mutation createUpdateMutation(Path file, Value v, KeyExtent extent) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b2539fb1/server/base/src/test/java/org/apache/accumulo/server/util/ReplicationTableUtilTest.java
----------------------------------------------------------------------
diff --git a/server/base/src/test/java/org/apache/accumulo/server/util/ReplicationTableUtilTest.java b/server/base/src/test/java/org/apache/accumulo/server/util/ReplicationTableUtilTest.java
index 355fa42..375e263 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/util/ReplicationTableUtilTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/util/ReplicationTableUtilTest.java
@@ -94,7 +94,7 @@ public class ReplicationTableUtilTest {
     String myFile = "file:////home/user/accumulo/wal/server+port/" + uuid;
 
     long createdTime = System.currentTimeMillis();
-    ReplicationTableUtil.updateFiles(context, new KeyExtent(new Text("1"), null, null), Collections.singleton(myFile), StatusUtil.fileCreated(createdTime));
+    ReplicationTableUtil.updateFiles(context, new KeyExtent(new Text("1"), null, null), myFile, StatusUtil.fileCreated(createdTime));
 
     verify(writer);
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b2539fb1/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 35c60d6..2561eec 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
@@ -18,7 +18,7 @@ package org.apache.accumulo.gc;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
-import java.util.ArrayList;
+import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
@@ -26,21 +26,22 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
-import java.util.UUID;
 
-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.Connector;
-import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.gc.thrift.GCStatus;
 import org.apache.accumulo.core.gc.thrift.GcCycleStats;
 import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.RootTable;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.CurrentLogsSection;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.ReplicationSection;
 import org.apache.accumulo.core.protobuf.ProtobufUtil;
 import org.apache.accumulo.core.replication.ReplicationSchema.StatusSection;
@@ -48,29 +49,29 @@ import org.apache.accumulo.core.replication.ReplicationTable;
 import org.apache.accumulo.core.replication.ReplicationTableOfflineException;
 import org.apache.accumulo.core.replication.StatusUtil;
 import org.apache.accumulo.core.replication.proto.Replication.Status;
-import org.apache.accumulo.core.rpc.ThriftUtil;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.tabletserver.log.LogEntry;
-import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
-import org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Client;
-import org.apache.accumulo.core.trace.Span;
 import org.apache.accumulo.core.trace.Trace;
-import org.apache.accumulo.core.trace.Tracer;
-import org.apache.accumulo.core.util.AddressUtil;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.server.AccumuloServerContext;
-import org.apache.accumulo.server.ServerConstants;
 import org.apache.accumulo.server.fs.VolumeManager;
-import org.apache.accumulo.server.util.MetadataTableUtil;
+import org.apache.accumulo.server.master.LiveTServerSet;
+import org.apache.accumulo.server.master.LiveTServerSet.Listener;
+import org.apache.accumulo.server.master.state.MetaDataStateStore;
+import org.apache.accumulo.server.master.state.RootTabletStateStore;
+import org.apache.accumulo.server.master.state.TServerInstance;
+import org.apache.accumulo.server.master.state.TabletLocationState;
+import org.apache.accumulo.server.master.state.TabletState;
 import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
-import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
-import org.apache.thrift.TException;
+import org.apache.hadoop.io.Text;
+import org.apache.htrace.Span;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.google.common.collect.Iterables;
+import com.google.common.collect.Iterators;
 import com.google.common.net.HostAndPort;
 import com.google.protobuf.InvalidProtocolBufferException;
 
@@ -79,8 +80,7 @@ public class GarbageCollectWriteAheadLogs {
 
   private final AccumuloServerContext context;
   private final VolumeManager fs;
-
-  private boolean useTrash;
+  private final boolean useTrash;
 
   /**
    * Creates a new GC WAL object.
@@ -98,54 +98,33 @@ public class GarbageCollectWriteAheadLogs {
     this.useTrash = useTrash;
   }
 
-  /**
-   * Gets the instance used by this object.
-   *
-   * @return instance
-   */
-  Instance getInstance() {
-    return context.getInstance();
-  }
-
-  /**
-   * Gets the volume manager used by this object.
-   *
-   * @return volume manager
-   */
-  VolumeManager getVolumeManager() {
-    return fs;
-  }
-
-  /**
-   * Checks if the volume manager should move files to the trash rather than delete them.
-   *
-   * @return true if trash is used
-   */
-  boolean isUsingTrash() {
-    return useTrash;
-  }
-
   public void collect(GCStatus status) {
 
-    Span span = Trace.start("scanServers");
+    Span span = Trace.start("getCandidates");
     try {
-
-      Map<String,Path> sortedWALogs = getSortedWALogs();
+      LiveTServerSet liveServers = new LiveTServerSet(context, new Listener() {
+        @Override
+        public void update(LiveTServerSet current, Set<TServerInstance> deleted, Set<TServerInstance> added) {
+          log.debug("New tablet server noticed: " + added);
+          log.debug("Tablet server removed: " + deleted);
+        }
+      });
+      Set<TServerInstance> currentServers = liveServers.getCurrentServers();
 
       status.currentLog.started = System.currentTimeMillis();
 
-      Map<Path,String> fileToServerMap = new HashMap<Path,String>();
-      Map<String,Path> nameToFileMap = new HashMap<String,Path>();
-      int count = scanServers(fileToServerMap, nameToFileMap);
+      Map<TServerInstance, Set<String> > candidates = new HashMap<>();
+      long count = getCurrent(candidates, currentServers);
       long fileScanStop = System.currentTimeMillis();
-      log.info(String.format("Fetched %d files from %d servers in %.2f seconds", fileToServerMap.size(), count,
+
+      log.info(String.format("Fetched %d files for %d servers in %.2f seconds", count, candidates.size(),
           (fileScanStop - status.currentLog.started) / 1000.));
-      status.currentLog.candidates = fileToServerMap.size();
+      status.currentLog.candidates = count;
       span.stop();
 
       span = Trace.start("removeMetadataEntries");
       try {
-        count = removeMetadataEntries(nameToFileMap, sortedWALogs, status);
+        count = removeMetadataEntries(candidates, status, currentServers);
       } catch (Exception ex) {
         log.error("Unable to scan metadata table", ex);
         return;
@@ -158,7 +137,7 @@ public class GarbageCollectWriteAheadLogs {
 
       span = Trace.start("removeReplicationEntries");
       try {
-        count = removeReplicationEntries(nameToFileMap, sortedWALogs, status);
+        count = removeReplicationEntries(candidates, status);
       } catch (Exception ex) {
         log.error("Unable to scan replication table", ex);
         return;
@@ -170,16 +149,23 @@ public class GarbageCollectWriteAheadLogs {
       log.info(String.format("%d replication entries scanned in %.2f seconds", count, (replicationEntryScanStop - logEntryScanStop) / 1000.));
 
       span = Trace.start("removeFiles");
-      Map<String,ArrayList<Path>> serverToFileMap = mapServersToFiles(fileToServerMap, nameToFileMap);
 
-      count = removeFiles(nameToFileMap, serverToFileMap, sortedWALogs, status);
+      count = removeFiles(candidates, status);
 
       long removeStop = System.currentTimeMillis();
-      log.info(String.format("%d total logs removed from %d servers in %.2f seconds", count, serverToFileMap.size(), (removeStop - logEntryScanStop) / 1000.));
+      log.info(String.format("%d total logs removed from %d servers in %.2f seconds", count, candidates.size(), (removeStop - logEntryScanStop) / 1000.));
+      span.stop();
+
+      span = Trace.start("removeMarkers");
+      count = removeMarkers(candidates);
+      long removeMarkersStop = System.currentTimeMillis();
+      log.info(String.format("%d markers removed in %.2f seconds", count, (removeMarkersStop - removeStop) / 1000.));
+      span.stop();
+
+
       status.currentLog.finished = removeStop;
       status.lastLog = status.currentLog;
       status.currentLog = new GcCycleStats();
-      span.stop();
 
     } catch (Exception e) {
       log.error("exception occured while garbage collecting write ahead logs", e);
@@ -188,161 +174,82 @@ public class GarbageCollectWriteAheadLogs {
     }
   }
 
-  boolean holdsLock(HostAndPort addr) {
+  private long removeMarkers(Map<TServerInstance,Set<String>> candidates) {
+    long result = 0;
     try {
-      String zpath = ZooUtil.getRoot(context.getInstance()) + Constants.ZTSERVERS + "/" + addr.toString();
-      List<String> children = ZooReaderWriter.getInstance().getChildren(zpath);
-      return !(children == null || children.isEmpty());
-    } catch (KeeperException.NoNodeException ex) {
-      return false;
-    } catch (Exception ex) {
-      log.debug(ex.toString(), ex);
-      return true;
-    }
-  }
-
-  private int removeFiles(Map<String,Path> nameToFileMap, Map<String,ArrayList<Path>> serverToFileMap, Map<String,Path> sortedWALogs, final GCStatus status) {
-    for (Entry<String,ArrayList<Path>> entry : serverToFileMap.entrySet()) {
-      if (entry.getKey().isEmpty()) {
-        // old-style log entry, just remove it
-        for (Path path : entry.getValue()) {
-          log.debug("Removing old-style WAL " + path);
-          try {
-            if (!useTrash || !fs.moveToTrash(path))
-              fs.deleteRecursively(path);
-            status.currentLog.deleted++;
-          } catch (FileNotFoundException ex) {
-            // ignored
-          } catch (IOException ex) {
-            log.error("Unable to delete wal " + path + ": " + ex);
-          }
-        }
-      } else {
-        HostAndPort address = AddressUtil.parseAddress(entry.getKey(), false);
-        if (!holdsLock(address)) {
-          for (Path path : entry.getValue()) {
-            log.debug("Removing WAL for offline server " + path);
-            try {
-              if (!useTrash || !fs.moveToTrash(path))
-                fs.deleteRecursively(path);
-              status.currentLog.deleted++;
-            } catch (FileNotFoundException ex) {
-              // ignored
-            } catch (IOException ex) {
-              log.error("Unable to delete wal " + path + ": " + ex);
-            }
-          }
-          continue;
-        } else {
-          Client tserver = null;
-          try {
-            tserver = ThriftUtil.getClient(new TabletClientService.Client.Factory(), address, context);
-            tserver.removeLogs(Tracer.traceInfo(), context.rpcCreds(), paths2strings(entry.getValue()));
-            log.debug("deleted " + entry.getValue() + " from " + entry.getKey());
-            status.currentLog.deleted += entry.getValue().size();
-          } catch (TException e) {
-            log.warn("Error talking to " + address + ": " + e);
-          } finally {
-            if (tserver != null)
-              ThriftUtil.returnClient(tserver);
-          }
+      BatchWriter root = context.getConnector().createBatchWriter(RootTable.NAME, null);
+      BatchWriter meta = context.getConnector().createBatchWriter(MetadataTable.NAME, null);
+      for (Entry<TServerInstance,Set<String>> entry : candidates.entrySet()) {
+        Mutation m = new Mutation(CurrentLogsSection.getRowPrefix() + entry.toString());
+        for (String wal : entry.getValue()) {
+          m.putDelete(CurrentLogsSection.COLF, new Text(wal));
+          result++;
         }
+        root.addMutation(m);
+        meta.addMutation(m);
       }
+      meta.close();
+      root.close();
+    } catch (Exception ex) {
+      throw new RuntimeException(ex);
     }
+    return result;
+  }
 
-    for (Path swalog : sortedWALogs.values()) {
-      log.debug("Removing sorted WAL " + swalog);
-      try {
-        if (!useTrash || !fs.moveToTrash(swalog)) {
-          fs.deleteRecursively(swalog);
-        }
-      } catch (FileNotFoundException ex) {
-        // ignored
-      } catch (IOException ioe) {
+  private long removeFiles(Map<TServerInstance, Set<String> > candidates, final GCStatus status) {
+    for (Entry<TServerInstance,Set<String>> entry : candidates.entrySet()) {
+      for (String walog : entry.getValue()) {
+        log.debug("Removing WAL for offline server " + entry.getKey() + " log " + walog);
+        Path path = new Path(walog);
         try {
-          if (fs.exists(swalog)) {
-            log.error("Unable to delete sorted walog " + swalog + ": " + ioe);
-          }
+          if (!useTrash || !fs.moveToTrash(path))
+            fs.deleteRecursively(path);
+          status.currentLog.deleted++;
+        } catch (FileNotFoundException ex) {
+          // ignored
         } catch (IOException ex) {
-          log.error("Unable to check for the existence of " + swalog, ex);
+          log.error("Unable to delete wal " + path + ": " + ex);
         }
       }
     }
-
-    return 0;
+    return status.currentLog.deleted;
   }
 
-  /**
-   * Converts a list of paths to their corresponding strings.
-   *
-   * @param paths
-   *          list of paths
-   * @return string forms of paths
-   */
-  static List<String> paths2strings(List<Path> paths) {
-    List<String> result = new ArrayList<String>(paths.size());
-    for (Path path : paths)
-      result.add(path.toString());
-    return result;
-  }
+  private long removeMetadataEntries(Map<TServerInstance, Set<String> > candidates, GCStatus status, Set<TServerInstance> liveServers) throws IOException, KeeperException,
+      InterruptedException {
 
-  /**
-   * Reverses the given mapping of file paths to servers. The returned map provides a list of file paths for each server. Any path whose name is not in the
-   * mapping of file names to paths is skipped.
-   *
-   * @param fileToServerMap
-   *          map of file paths to servers
-   * @param nameToFileMap
-   *          map of file names to paths
-   * @return map of servers to lists of file paths
-   */
-  static Map<String,ArrayList<Path>> mapServersToFiles(Map<Path,String> fileToServerMap, Map<String,Path> nameToFileMap) {
-    Map<String,ArrayList<Path>> result = new HashMap<String,ArrayList<Path>>();
-    for (Entry<Path,String> fileServer : fileToServerMap.entrySet()) {
-      if (!nameToFileMap.containsKey(fileServer.getKey().getName()))
-        continue;
-      ArrayList<Path> files = result.get(fileServer.getValue());
-      if (files == null) {
-        files = new ArrayList<Path>();
-        result.put(fileServer.getValue(), files);
+    // remove any entries if there's a log reference, or a tablet is still assigned to the dead server
+
+    Map<String, TServerInstance> walToDeadServer = new HashMap<>();
+    for (Entry<TServerInstance,Set<String>> entry : candidates.entrySet()) {
+      for (String file : entry.getValue()) {
+        walToDeadServer.put(file, entry.getKey());
       }
-      files.add(fileServer.getKey());
     }
-    return result;
-  }
-
-  protected int removeMetadataEntries(Map<String,Path> nameToFileMap, Map<String,Path> sortedWALogs, GCStatus status) throws IOException, KeeperException,
-      InterruptedException {
-    int count = 0;
-    Iterator<LogEntry> iterator = MetadataTableUtil.getLogEntries(context);
-
-    // For each WAL reference in the metadata table
-    while (iterator.hasNext()) {
-      // Each metadata reference has at least one WAL file
-      for (String entry : iterator.next().logSet) {
-        // old style WALs will have the IP:Port of their logger and new style will either be a Path either absolute or relative, in all cases
-        // the last "/" will mark a UUID file name.
-        String uuid = entry.substring(entry.lastIndexOf("/") + 1);
-        if (!isUUID(uuid)) {
-          // fully expect this to be a uuid, if its not then something is wrong and walog GC should not proceed!
-          throw new IllegalArgumentException("Expected uuid, but got " + uuid + " from " + entry);
-        }
-
-        Path pathFromNN = nameToFileMap.remove(uuid);
-        if (pathFromNN != null) {
-          status.currentLog.inUse++;
-          sortedWALogs.remove(uuid);
+    long count = 0;
+    RootTabletStateStore root = new RootTabletStateStore(context);
+    MetaDataStateStore meta = new MetaDataStateStore(context);
+    Iterator<TabletLocationState> states = Iterators.concat(root.iterator(), meta.iterator());
+    while (states.hasNext()) {
+      count++;
+      TabletLocationState state = states.next();
+      if (state.getState(liveServers) == TabletState.ASSIGNED_TO_DEAD_SERVER) {
+        candidates.remove(state.current);
+      }
+      for (Collection<String> wals : state.walogs) {
+        for (String wal : wals) {
+          TServerInstance dead = walToDeadServer.get(wal);
+          if (dead != null) {
+            candidates.get(dead).remove(wal);
+          }
         }
-
-        count++;
       }
     }
-
     return count;
   }
 
-  protected int removeReplicationEntries(Map<String,Path> nameToFileMap, Map<String,Path> sortedWALogs, GCStatus status) throws IOException, KeeperException,
-      InterruptedException {
+  protected int removeReplicationEntries(Map<TServerInstance, Set<String> > candidates, GCStatus status) throws IOException, KeeperException,
+  InterruptedException {
     Connector conn;
     try {
       conn = context.getConnector();
@@ -353,21 +260,25 @@ public class GarbageCollectWriteAheadLogs {
 
     int count = 0;
 
-    Iterator<Entry<String,Path>> walIter = nameToFileMap.entrySet().iterator();
+    Iterator<Entry<TServerInstance,Set<String>>> walIter = candidates.entrySet().iterator();
 
     while (walIter.hasNext()) {
-      Entry<String,Path> wal = walIter.next();
-      String fullPath = wal.getValue().toString();
-      if (neededByReplication(conn, fullPath)) {
-        log.debug("Removing WAL from candidate deletion as it is still needed for replication: {}", fullPath);
-        // If we haven't already removed it, check to see if this WAL is
-        // "in use" by replication (needed for replication purposes)
-        status.currentLog.inUse++;
-
+      Entry<TServerInstance,Set<String>> wal = walIter.next();
+      Iterator<String> paths = wal.getValue().iterator();
+      while (paths.hasNext()) {
+        String fullPath = paths.next();
+        if (neededByReplication(conn, fullPath)) {
+          log.debug("Removing WAL from candidate deletion as it is still needed for replication: {}", fullPath);
+          // If we haven't already removed it, check to see if this WAL is
+          // "in use" by replication (needed for replication purposes)
+          status.currentLog.inUse++;
+          paths.remove();
+        } else {
+          log.debug("WAL not needed for replication {}", fullPath);
+        }
+      }
+      if (wal.getValue().isEmpty()) {
         walIter.remove();
-        sortedWALogs.remove(wal.getKey());
-      } else {
-        log.debug("WAL not needed for replication {}", fullPath);
       }
       count++;
     }
@@ -375,6 +286,7 @@ public class GarbageCollectWriteAheadLogs {
     return count;
   }
 
+
   /**
    * Determine if the given WAL is needed for replication
    *
@@ -435,107 +347,54 @@ public class GarbageCollectWriteAheadLogs {
     return metaScanner;
   }
 
-  private int scanServers(Map<Path,String> fileToServerMap, Map<String,Path> nameToFileMap) throws Exception {
-    return scanServers(ServerConstants.getWalDirs(), fileToServerMap, nameToFileMap);
-  }
-
-  /**
-   * Scans write-ahead log directories for logs. The maps passed in are populated with scan information.
-   *
-   * @param walDirs
-   *          write-ahead log directories
-   * @param fileToServerMap
-   *          map of file paths to servers
-   * @param nameToFileMap
-   *          map of file names to paths
-   * @return number of servers located (including those with no logs present)
-   */
-  int scanServers(String[] walDirs, Map<Path,String> fileToServerMap, Map<String,Path> nameToFileMap) throws Exception {
-    Set<String> servers = new HashSet<String>();
-    for (String walDir : walDirs) {
-      Path walRoot = new Path(walDir);
-      FileStatus[] listing = null;
-      try {
-        listing = fs.listStatus(walRoot);
-      } catch (FileNotFoundException e) {
-        // ignore dir
-      }
 
-      if (listing == null)
-        continue;
-      for (FileStatus status : listing) {
-        String server = status.getPath().getName();
-        if (status.isDirectory()) {
-          servers.add(server);
-          for (FileStatus file : fs.listStatus(new Path(walRoot, server))) {
-            if (isUUID(file.getPath().getName())) {
-              fileToServerMap.put(file.getPath(), server);
-              nameToFileMap.put(file.getPath().getName(), file.getPath());
-            } else {
-              log.info("Ignoring file " + file.getPath() + " because it doesn't look like a uuid");
-            }
-          }
-        } else if (isUUID(server)) {
-          // old-style WAL are not under a directory
-          servers.add("");
-          fileToServerMap.put(status.getPath(), "");
-          nameToFileMap.put(server, status.getPath());
-        } else {
-          log.info("Ignoring file " + status.getPath() + " because it doesn't look like a uuid");
-        }
-      }
-    }
-    return servers.size();
-  }
 
-  private Map<String,Path> getSortedWALogs() throws IOException {
-    return getSortedWALogs(ServerConstants.getRecoveryDirs());
-  }
 
   /**
-   * Looks for write-ahead logs in recovery directories.
+   * Scans log markers. The map passed in is populated with the logs for dead servers.
    *
-   * @param recoveryDirs
-   *          recovery directories
-   * @return map of log file names to paths
+   * @param logsForDeadServers
+   *          map of dead server to log file entries
+   * @return total number of log files
    */
-  Map<String,Path> getSortedWALogs(String[] recoveryDirs) throws IOException {
-    Map<String,Path> result = new HashMap<String,Path>();
-
-    for (String dir : recoveryDirs) {
-      Path recoveryDir = new Path(dir);
-
-      if (fs.exists(recoveryDir)) {
-        for (FileStatus status : fs.listStatus(recoveryDir)) {
-          String name = status.getPath().getName();
-          if (isUUID(name)) {
-            result.put(name, status.getPath());
-          } else {
-            log.debug("Ignoring file " + status.getPath() + " because it doesn't look like a uuid");
-          }
+  private long getCurrent(Map<TServerInstance, Set<String> > logsForDeadServers, Set<TServerInstance> currentServers) throws Exception {
+    Set<String> rootWALs = new HashSet<String>();
+    // Get entries in zookeeper:
+    String zpath = ZooUtil.getRoot(context.getInstance()) + RootTable.ZROOT_TABLET_WALOGS;
+    ZooReaderWriter zoo = ZooReaderWriter.getInstance();
+    List<String> children = zoo.getChildren(zpath);
+    for (String child : children) {
+      LogEntry entry = LogEntry.fromBytes(zoo.getData(zpath + "/" + child, null));
+      rootWALs.add(entry.filename);
+    }
+    long count = 0;
+
+    // get all the WAL markers that are not in zookeeper for dead servers
+    Scanner rootScanner = context.getConnector().createScanner(RootTable.NAME, Authorizations.EMPTY);
+    rootScanner.setRange(CurrentLogsSection.getRange());
+    Scanner metaScanner = context.getConnector().createScanner(MetadataTable.NAME, Authorizations.EMPTY);
+    metaScanner.setRange(CurrentLogsSection.getRange());
+    Iterator<Entry<Key,Value>> entries = Iterators.concat(rootScanner.iterator(), metaScanner.iterator());
+    Text hostAndPort = new Text();
+    Text sessionId = new Text();
+    Text filename = new Text();
+    while (entries.hasNext()) {
+      Entry<Key,Value> entry = entries.next();
+      CurrentLogsSection.getTabletServer(entry.getKey(), hostAndPort, sessionId);
+      CurrentLogsSection.getPath(entry.getKey(), filename);
+      TServerInstance tsi = new TServerInstance(HostAndPort.fromString(hostAndPort.toString()), sessionId.toString());
+      if ((!currentServers.contains(tsi) || entry.getValue().equals(CurrentLogsSection.UNUSED)) && !rootWALs.contains(filename)) {
+        Set<String> logs = logsForDeadServers.get(tsi);
+        if (logs == null) {
+          logsForDeadServers.put(tsi, logs = new HashSet<String>());
+        }
+        if (logs.add(new Path(filename.toString()).toString())) {
+          count++;
         }
       }
     }
-    return result;
-  }
 
-  /**
-   * Checks if a string is a valid UUID.
-   *
-   * @param name
-   *          string to check
-   * @return true if string is a UUID
-   */
-  static boolean isUUID(String name) {
-    if (name == null || name.length() != 36) {
-      return false;
-    }
-    try {
-      UUID.fromString(name);
-      return true;
-    } catch (IllegalArgumentException ex) {
-      return false;
-    }
+    return count;
   }
 
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b2539fb1/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 35005d8..9328225 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
@@ -568,7 +568,6 @@ public class SimpleGarbageCollector extends AccumuloServerContext implements Ifa
         replSpan.stop();
       }
 
-      // Clean up any unused write-ahead logs
       Span waLogs = Trace.start("walogs");
       try {
         GarbageCollectWriteAheadLogs walogCollector = new GarbageCollectWriteAheadLogs(this, fs, isUsingTrash());

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b2539fb1/server/gc/src/main/java/org/apache/accumulo/gc/replication/CloseWriteAheadLogReferences.java
----------------------------------------------------------------------
diff --git a/server/gc/src/main/java/org/apache/accumulo/gc/replication/CloseWriteAheadLogReferences.java b/server/gc/src/main/java/org/apache/accumulo/gc/replication/CloseWriteAheadLogReferences.java
index 9b60c88..8185f23 100644
--- a/server/gc/src/main/java/org/apache/accumulo/gc/replication/CloseWriteAheadLogReferences.java
+++ b/server/gc/src/main/java/org/apache/accumulo/gc/replication/CloseWriteAheadLogReferences.java
@@ -37,15 +37,13 @@ import org.apache.accumulo.core.file.rfile.RFile;
 import org.apache.accumulo.core.master.thrift.MasterClientService;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.CurrentLogsSection;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.ReplicationSection;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LogColumnFamily;
 import org.apache.accumulo.core.replication.ReplicationTable;
 import org.apache.accumulo.core.replication.StatusUtil;
 import org.apache.accumulo.core.replication.proto.Replication.Status;
 import org.apache.accumulo.core.rpc.ThriftUtil;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.tabletserver.log.LogEntry;
 import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
 import org.apache.accumulo.core.trace.Span;
 import org.apache.accumulo.core.trace.Trace;
@@ -186,20 +184,21 @@ public class CloseWriteAheadLogReferences implements Runnable {
     try {
       // TODO Configurable number of threads
       bs = conn.createBatchScanner(MetadataTable.NAME, Authorizations.EMPTY, 4);
-      bs.setRanges(Collections.singleton(TabletsSection.getRange()));
-      bs.fetchColumnFamily(LogColumnFamily.NAME);
+      bs.setRanges(Collections.singleton(CurrentLogsSection.getRange()));
+      bs.fetchColumnFamily(CurrentLogsSection.COLF);
 
       // For each log key/value in the metadata table
       for (Entry<Key,Value> entry : bs) {
-        // The value may contain multiple WALs
-        LogEntry logEntry = LogEntry.fromKeyValue(entry.getKey(), entry.getValue());
-
-        log.debug("Found WALs for table(" + logEntry.extent.getTableId() + "): " + logEntry.logSet);
+        if (entry.getValue().equals(CurrentLogsSection.UNUSED)) {
+          continue;
+        }
+        Text tpath = new Text();
+        CurrentLogsSection.getPath(entry.getKey(), tpath);
+        String path = new Path(tpath.toString()).toString();
+        log.debug("Found WAL " + path.toString());
 
         // Normalize each log file (using Path) and add it to the set
-        for (String logFile : logEntry.logSet) {
-          referencedWals.add(normalizedWalPaths.get(logFile));
-        }
+        referencedWals.add(normalizedWalPaths.get(path));
       }
     } catch (TableNotFoundException e) {
       // uhhhh
@@ -248,6 +247,8 @@ public class CloseWriteAheadLogReferences implements Runnable {
         MetadataSchema.ReplicationSection.getFile(entry.getKey(), replFileText);
         String replFile = replFileText.toString();
         boolean isReferenced = referencedWals.contains(replFile);
+        log.debug("replFile " + replFile);
+        log.debug("referencedWals " + referencedWals);
 
         // We only want to clean up WALs (which is everything but rfiles) and only when
         // metadata doesn't have a reference to the given WAL

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b2539fb1/server/gc/src/test/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogsTest.java
----------------------------------------------------------------------
diff --git a/server/gc/src/test/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogsTest.java b/server/gc/src/test/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogsTest.java
deleted file mode 100644
index 5224f28..0000000
--- a/server/gc/src/test/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogsTest.java
+++ /dev/null
@@ -1,568 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.gc;
-
-import static org.easymock.EasyMock.createMock;
-import static org.easymock.EasyMock.expect;
-import static org.easymock.EasyMock.replay;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertSame;
-import static org.junit.Assert.assertTrue;
-
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.UUID;
-
-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.mock.MockInstance;
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.ConfigurationCopy;
-import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.conf.SiteConfiguration;
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.gc.thrift.GCStatus;
-import org.apache.accumulo.core.gc.thrift.GcCycleStats;
-import org.apache.accumulo.core.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.ReplicationSection;
-import org.apache.accumulo.core.protobuf.ProtobufUtil;
-import org.apache.accumulo.core.replication.ReplicationSchema.StatusSection;
-import org.apache.accumulo.core.replication.ReplicationTable;
-import org.apache.accumulo.core.replication.StatusUtil;
-import org.apache.accumulo.core.replication.proto.Replication.Status;
-import org.apache.accumulo.server.AccumuloServerContext;
-import org.apache.accumulo.server.conf.ServerConfigurationFactory;
-import org.apache.accumulo.server.fs.VolumeManager;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
-import org.easymock.EasyMock;
-import org.easymock.IAnswer;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
-
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Maps;
-
-public class GarbageCollectWriteAheadLogsTest {
-  private static final long BLOCK_SIZE = 64000000L;
-
-  private static final Path DIR_1_PATH = new Path("/dir1");
-  private static final Path DIR_2_PATH = new Path("/dir2");
-  private static final Path DIR_3_PATH = new Path("/dir3");
-  private static final String UUID1 = UUID.randomUUID().toString();
-  private static final String UUID2 = UUID.randomUUID().toString();
-  private static final String UUID3 = UUID.randomUUID().toString();
-
-  private Instance instance;
-  private AccumuloConfiguration systemConfig;
-  private VolumeManager volMgr;
-  private GarbageCollectWriteAheadLogs gcwal;
-  private AccumuloServerContext context;
-  private long modTime;
-
-  @Rule
-  public TestName testName = new TestName();
-
-  @Before
-  public void setUp() throws Exception {
-    SiteConfiguration siteConfig = EasyMock.createMock(SiteConfiguration.class);
-    instance = createMock(Instance.class);
-    expect(instance.getInstanceID()).andReturn("mock").anyTimes();
-    expect(instance.getZooKeepers()).andReturn("localhost").anyTimes();
-    expect(instance.getZooKeepersSessionTimeOut()).andReturn(30000).anyTimes();
-    systemConfig = new ConfigurationCopy(new HashMap<String,String>());
-    volMgr = createMock(VolumeManager.class);
-    ServerConfigurationFactory factory = createMock(ServerConfigurationFactory.class);
-    expect(factory.getConfiguration()).andReturn(systemConfig).anyTimes();
-    expect(factory.getInstance()).andReturn(instance).anyTimes();
-    expect(factory.getSiteConfiguration()).andReturn(siteConfig).anyTimes();
-
-    // Just make the SiteConfiguration delegate to our AccumuloConfiguration
-    // Presently, we only need get(Property) and iterator().
-    EasyMock.expect(siteConfig.get(EasyMock.anyObject(Property.class))).andAnswer(new IAnswer<String>() {
-      @Override
-      public String answer() {
-        Object[] args = EasyMock.getCurrentArguments();
-        return systemConfig.get((Property) args[0]);
-      }
-    }).anyTimes();
-    EasyMock.expect(siteConfig.getBoolean(EasyMock.anyObject(Property.class))).andAnswer(new IAnswer<Boolean>() {
-      @Override
-      public Boolean answer() {
-        Object[] args = EasyMock.getCurrentArguments();
-        return systemConfig.getBoolean((Property) args[0]);
-      }
-    }).anyTimes();
-
-    EasyMock.expect(siteConfig.iterator()).andAnswer(new IAnswer<Iterator<Entry<String,String>>>() {
-      @Override
-      public Iterator<Entry<String,String>> answer() {
-        return systemConfig.iterator();
-      }
-    }).anyTimes();
-
-    replay(instance, factory, siteConfig);
-    AccumuloServerContext context = new AccumuloServerContext(factory);
-    gcwal = new GarbageCollectWriteAheadLogs(context, volMgr, false);
-    modTime = System.currentTimeMillis();
-  }
-
-  @Test
-  public void testGetters() {
-    assertSame(instance, gcwal.getInstance());
-    assertSame(volMgr, gcwal.getVolumeManager());
-    assertFalse(gcwal.isUsingTrash());
-  }
-
-  @Test
-  public void testPathsToStrings() {
-    ArrayList<Path> paths = new ArrayList<Path>();
-    paths.add(new Path(DIR_1_PATH, "file1"));
-    paths.add(DIR_2_PATH);
-    paths.add(new Path(DIR_3_PATH, "file3"));
-    List<String> strings = GarbageCollectWriteAheadLogs.paths2strings(paths);
-    int len = 3;
-    assertEquals(len, strings.size());
-    for (int i = 0; i < len; i++) {
-      assertEquals(paths.get(i).toString(), strings.get(i));
-    }
-  }
-
-  @Test
-  public void testMapServersToFiles() {
-    // @formatter:off
-    /*
-     * Test fileToServerMap:
-     * /dir1/server1/uuid1 -> server1 (new-style)
-     * /dir1/uuid2 -> "" (old-style)
-     * /dir3/server3/uuid3 -> server3 (new-style)
-     */
-    // @formatter:on
-    Map<Path,String> fileToServerMap = new java.util.HashMap<Path,String>();
-    Path path1 = new Path(new Path(DIR_1_PATH, "server1"), UUID1);
-    fileToServerMap.put(path1, "server1"); // new-style
-    Path path2 = new Path(DIR_1_PATH, UUID2);
-    fileToServerMap.put(path2, ""); // old-style
-    Path path3 = new Path(new Path(DIR_3_PATH, "server3"), UUID3);
-    fileToServerMap.put(path3, "server3"); // old-style
-    // @formatter:off
-    /*
-     * Test nameToFileMap:
-     * uuid1 -> /dir1/server1/uuid1
-     * uuid3 -> /dir3/server3/uuid3
-     */
-    // @formatter:on
-    Map<String,Path> nameToFileMap = new java.util.HashMap<String,Path>();
-    nameToFileMap.put(UUID1, path1);
-    nameToFileMap.put(UUID3, path3);
-
-    // @formatter:off
-    /*
-     * Expected map:
-     * server1 -> [ /dir1/server1/uuid1 ]
-     * server3 -> [ /dir3/server3/uuid3 ]
-     */
-    // @formatter:on
-    Map<String,ArrayList<Path>> result = GarbageCollectWriteAheadLogs.mapServersToFiles(fileToServerMap, nameToFileMap);
-    assertEquals(2, result.size());
-    ArrayList<Path> list1 = result.get("server1");
-    assertEquals(1, list1.size());
-    assertTrue(list1.contains(path1));
-    ArrayList<Path> list3 = result.get("server3");
-    assertEquals(1, list3.size());
-    assertTrue(list3.contains(path3));
-  }
-
-  private FileStatus makeFileStatus(int size, Path path) {
-    boolean isDir = (size == 0);
-    return new FileStatus(size, isDir, 3, BLOCK_SIZE, modTime, path);
-  }
-
-  private void mockListStatus(Path dir, FileStatus... fileStatuses) throws Exception {
-    expect(volMgr.listStatus(dir)).andReturn(fileStatuses);
-  }
-
-  @Test
-  public void testScanServers_NewStyle() throws Exception {
-    String[] walDirs = new String[] {"/dir1", "/dir2", "/dir3"};
-    // @formatter:off
-    /*
-     * Test directory layout:
-     * /dir1/
-     *   server1/
-     *     uuid1
-     *     file2
-     *   subdir2/
-     * /dir2/ missing
-     * /dir3/
-     *   server3/
-     *     uuid3
-     */
-    // @formatter:on
-    Path serverDir1Path = new Path(DIR_1_PATH, "server1");
-    FileStatus serverDir1 = makeFileStatus(0, serverDir1Path);
-    Path subDir2Path = new Path(DIR_1_PATH, "subdir2");
-    FileStatus serverDir2 = makeFileStatus(0, subDir2Path);
-    mockListStatus(DIR_1_PATH, serverDir1, serverDir2);
-    Path path1 = new Path(serverDir1Path, UUID1);
-    FileStatus file1 = makeFileStatus(100, path1);
-    FileStatus file2 = makeFileStatus(200, new Path(serverDir1Path, "file2"));
-    mockListStatus(serverDir1Path, file1, file2);
-    mockListStatus(subDir2Path);
-    expect(volMgr.listStatus(DIR_2_PATH)).andThrow(new FileNotFoundException());
-    Path serverDir3Path = new Path(DIR_3_PATH, "server3");
-    FileStatus serverDir3 = makeFileStatus(0, serverDir3Path);
-    mockListStatus(DIR_3_PATH, serverDir3);
-    Path path3 = new Path(serverDir3Path, UUID3);
-    FileStatus file3 = makeFileStatus(300, path3);
-    mockListStatus(serverDir3Path, file3);
-    replay(volMgr);
-
-    Map<Path,String> fileToServerMap = new java.util.HashMap<Path,String>();
-    Map<String,Path> nameToFileMap = new java.util.HashMap<String,Path>();
-    int count = gcwal.scanServers(walDirs, fileToServerMap, nameToFileMap);
-    assertEquals(3, count);
-    // @formatter:off
-    /*
-     * Expected fileToServerMap:
-     * /dir1/server1/uuid1 -> server1
-     * /dir3/server3/uuid3 -> server3
-     */
-    // @formatter:on
-    assertEquals(2, fileToServerMap.size());
-    assertEquals("server1", fileToServerMap.get(path1));
-    assertEquals("server3", fileToServerMap.get(path3));
-    // @formatter:off
-    /*
-     * Expected nameToFileMap:
-     * uuid1 -> /dir1/server1/uuid1
-     * uuid3 -> /dir3/server3/uuid3
-     */
-    // @formatter:on
-    assertEquals(2, nameToFileMap.size());
-    assertEquals(path1, nameToFileMap.get(UUID1));
-    assertEquals(path3, nameToFileMap.get(UUID3));
-  }
-
-  @Test
-  public void testScanServers_OldStyle() throws Exception {
-    // @formatter:off
-    /*
-     * Test directory layout:
-     * /dir1/
-     *   uuid1
-     * /dir3/
-     *   uuid3
-     */
-    // @formatter:on
-    String[] walDirs = new String[] {"/dir1", "/dir3"};
-    Path serverFile1Path = new Path(DIR_1_PATH, UUID1);
-    FileStatus serverFile1 = makeFileStatus(100, serverFile1Path);
-    mockListStatus(DIR_1_PATH, serverFile1);
-    Path serverFile3Path = new Path(DIR_3_PATH, UUID3);
-    FileStatus serverFile3 = makeFileStatus(300, serverFile3Path);
-    mockListStatus(DIR_3_PATH, serverFile3);
-    replay(volMgr);
-
-    Map<Path,String> fileToServerMap = new java.util.HashMap<Path,String>();
-    Map<String,Path> nameToFileMap = new java.util.HashMap<String,Path>();
-    int count = gcwal.scanServers(walDirs, fileToServerMap, nameToFileMap);
-    /*
-     * Expect only a single server, the non-server entry for upgrade WALs
-     */
-    assertEquals(1, count);
-    // @formatter:off
-    /*
-     * Expected fileToServerMap:
-     * /dir1/uuid1 -> ""
-     * /dir3/uuid3 -> ""
-     */
-    // @formatter:on
-    assertEquals(2, fileToServerMap.size());
-    assertEquals("", fileToServerMap.get(serverFile1Path));
-    assertEquals("", fileToServerMap.get(serverFile3Path));
-    // @formatter:off
-    /*
-     * Expected nameToFileMap:
-     * uuid1 -> /dir1/uuid1
-     * uuid3 -> /dir3/uuid3
-     */
-    // @formatter:on
-    assertEquals(2, nameToFileMap.size());
-    assertEquals(serverFile1Path, nameToFileMap.get(UUID1));
-    assertEquals(serverFile3Path, nameToFileMap.get(UUID3));
-  }
-
-  @Test
-  public void testGetSortedWALogs() throws Exception {
-    String[] recoveryDirs = new String[] {"/dir1", "/dir2", "/dir3"};
-    // @formatter:off
-    /*
-     * Test directory layout:
-     * /dir1/
-     *   uuid1
-     *   file2
-     * /dir2/ missing
-     * /dir3/
-     *   uuid3
-     */
-    // @formatter:on
-    expect(volMgr.exists(DIR_1_PATH)).andReturn(true);
-    expect(volMgr.exists(DIR_2_PATH)).andReturn(false);
-    expect(volMgr.exists(DIR_3_PATH)).andReturn(true);
-    Path path1 = new Path(DIR_1_PATH, UUID1);
-    FileStatus file1 = makeFileStatus(100, path1);
-    FileStatus file2 = makeFileStatus(200, new Path(DIR_1_PATH, "file2"));
-    mockListStatus(DIR_1_PATH, file1, file2);
-    Path path3 = new Path(DIR_3_PATH, UUID3);
-    FileStatus file3 = makeFileStatus(300, path3);
-    mockListStatus(DIR_3_PATH, file3);
-    replay(volMgr);
-
-    Map<String,Path> sortedWalogs = gcwal.getSortedWALogs(recoveryDirs);
-    // @formatter:off
-    /*
-     * Expected map:
-     * uuid1 -> /dir1/uuid1
-     * uuid3 -> /dir3/uuid3
-     */
-    // @formatter:on
-    assertEquals(2, sortedWalogs.size());
-    assertEquals(path1, sortedWalogs.get(UUID1));
-    assertEquals(path3, sortedWalogs.get(UUID3));
-  }
-
-  @Test
-  public void testIsUUID() {
-    assertTrue(GarbageCollectWriteAheadLogs.isUUID(UUID.randomUUID().toString()));
-    assertFalse(GarbageCollectWriteAheadLogs.isUUID("foo"));
-    assertFalse(GarbageCollectWriteAheadLogs.isUUID("0" + UUID.randomUUID().toString()));
-    assertFalse(GarbageCollectWriteAheadLogs.isUUID(null));
-  }
-
-  // It was easier to do this than get the mocking working for me
-  private static class ReplicationGCWAL extends GarbageCollectWriteAheadLogs {
-
-    private List<Entry<Key,Value>> replData;
-
-    ReplicationGCWAL(AccumuloServerContext context, VolumeManager fs, boolean useTrash, List<Entry<Key,Value>> replData) throws IOException {
-      super(context, fs, useTrash);
-      this.replData = replData;
-    }
-
-    @Override
-    protected Iterable<Entry<Key,Value>> getReplicationStatusForFile(Connector conn, String wal) {
-      return this.replData;
-    }
-  }
-
-  @Test
-  public void replicationEntriesAffectGC() throws Exception {
-    String file1 = UUID.randomUUID().toString(), file2 = UUID.randomUUID().toString();
-    Connector conn = createMock(Connector.class);
-
-    // Write a Status record which should prevent file1 from being deleted
-    LinkedList<Entry<Key,Value>> replData = new LinkedList<>();
-    replData.add(Maps.immutableEntry(new Key("/wals/" + file1, StatusSection.NAME.toString(), "1"), StatusUtil.fileCreatedValue(System.currentTimeMillis())));
-
-    ReplicationGCWAL replGC = new ReplicationGCWAL(context, volMgr, false, replData);
-
-    replay(conn);
-
-    // Open (not-closed) file must be retained
-    assertTrue(replGC.neededByReplication(conn, "/wals/" + file1));
-
-    // No replication data, not needed
-    replData.clear();
-    assertFalse(replGC.neededByReplication(conn, "/wals/" + file2));
-
-    // The file is closed but not replicated, must be retained
-    replData.add(Maps.immutableEntry(new Key("/wals/" + file1, StatusSection.NAME.toString(), "1"), StatusUtil.fileClosedValue()));
-    assertTrue(replGC.neededByReplication(conn, "/wals/" + file1));
-
-    // File is closed and fully replicated, can be deleted
-    replData.clear();
-    replData.add(Maps.immutableEntry(new Key("/wals/" + file1, StatusSection.NAME.toString(), "1"),
-        ProtobufUtil.toValue(Status.newBuilder().setInfiniteEnd(true).setBegin(Long.MAX_VALUE).setClosed(true).build())));
-    assertFalse(replGC.neededByReplication(conn, "/wals/" + file1));
-  }
-
-  @Test
-  public void removeReplicationEntries() throws Exception {
-    String file1 = UUID.randomUUID().toString(), file2 = UUID.randomUUID().toString();
-
-    Instance inst = new MockInstance(testName.getMethodName());
-    AccumuloServerContext context = new AccumuloServerContext(new ServerConfigurationFactory(inst));
-
-    GarbageCollectWriteAheadLogs gcWALs = new GarbageCollectWriteAheadLogs(context, volMgr, false);
-
-    long file1CreateTime = System.currentTimeMillis();
-    long file2CreateTime = file1CreateTime + 50;
-    BatchWriter bw = ReplicationTable.getBatchWriter(context.getConnector());
-    Mutation m = new Mutation("/wals/" + file1);
-    StatusSection.add(m, new Text("1"), StatusUtil.fileCreatedValue(file1CreateTime));
-    bw.addMutation(m);
-    m = new Mutation("/wals/" + file2);
-    StatusSection.add(m, new Text("1"), StatusUtil.fileCreatedValue(file2CreateTime));
-    bw.addMutation(m);
-
-    // These WALs are potential candidates for deletion from fs
-    Map<String,Path> nameToFileMap = new HashMap<>();
-    nameToFileMap.put(file1, new Path("/wals/" + file1));
-    nameToFileMap.put(file2, new Path("/wals/" + file2));
-
-    Map<String,Path> sortedWALogs = Collections.emptyMap();
-
-    // Make the GCStatus and GcCycleStats
-    GCStatus status = new GCStatus();
-    GcCycleStats cycleStats = new GcCycleStats();
-    status.currentLog = cycleStats;
-
-    // We should iterate over two entries
-    Assert.assertEquals(2, gcWALs.removeReplicationEntries(nameToFileMap, sortedWALogs, status));
-
-    // We should have noted that two files were still in use
-    Assert.assertEquals(2l, cycleStats.inUse);
-
-    // Both should have been deleted
-    Assert.assertEquals(0, nameToFileMap.size());
-  }
-
-  @Test
-  public void replicationEntriesOnlyInMetaPreventGC() throws Exception {
-    String file1 = UUID.randomUUID().toString(), file2 = UUID.randomUUID().toString();
-
-    Instance inst = new MockInstance(testName.getMethodName());
-    AccumuloServerContext context = new AccumuloServerContext(new ServerConfigurationFactory(inst));
-
-    Connector conn = context.getConnector();
-
-    GarbageCollectWriteAheadLogs gcWALs = new GarbageCollectWriteAheadLogs(context, volMgr, false);
-
-    long file1CreateTime = System.currentTimeMillis();
-    long file2CreateTime = file1CreateTime + 50;
-    // Write some records to the metadata table, we haven't yet written status records to the replication table
-    BatchWriter bw = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
-    Mutation m = new Mutation(ReplicationSection.getRowPrefix() + "/wals/" + file1);
-    m.put(ReplicationSection.COLF, new Text("1"), StatusUtil.fileCreatedValue(file1CreateTime));
-    bw.addMutation(m);
-
-    m = new Mutation(ReplicationSection.getRowPrefix() + "/wals/" + file2);
-    m.put(ReplicationSection.COLF, new Text("1"), StatusUtil.fileCreatedValue(file2CreateTime));
-    bw.addMutation(m);
-
-    // These WALs are potential candidates for deletion from fs
-    Map<String,Path> nameToFileMap = new HashMap<>();
-    nameToFileMap.put(file1, new Path("/wals/" + file1));
-    nameToFileMap.put(file2, new Path("/wals/" + file2));
-
-    Map<String,Path> sortedWALogs = Collections.emptyMap();
-
-    // Make the GCStatus and GcCycleStats objects
-    GCStatus status = new GCStatus();
-    GcCycleStats cycleStats = new GcCycleStats();
-    status.currentLog = cycleStats;
-
-    // We should iterate over two entries
-    Assert.assertEquals(2, gcWALs.removeReplicationEntries(nameToFileMap, sortedWALogs, status));
-
-    // We should have noted that two files were still in use
-    Assert.assertEquals(2l, cycleStats.inUse);
-
-    // Both should have been deleted
-    Assert.assertEquals(0, nameToFileMap.size());
-  }
-
-  @Test
-  public void noReplicationTableDoesntLimitMetatdataResults() throws Exception {
-    Instance inst = new MockInstance(testName.getMethodName());
-    AccumuloServerContext context = new AccumuloServerContext(new ServerConfigurationFactory(inst));
-    Connector conn = context.getConnector();
-
-    String wal = "hdfs://localhost:8020/accumulo/wal/tserver+port/123456-1234-1234-12345678";
-    BatchWriter bw = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
-    Mutation m = new Mutation(ReplicationSection.getRowPrefix() + wal);
-    m.put(ReplicationSection.COLF, new Text("1"), StatusUtil.fileCreatedValue(System.currentTimeMillis()));
-    bw.addMutation(m);
-    bw.close();
-
-    GarbageCollectWriteAheadLogs gcWALs = new GarbageCollectWriteAheadLogs(context, volMgr, false);
-
-    Iterable<Entry<Key,Value>> data = gcWALs.getReplicationStatusForFile(conn, wal);
-    Entry<Key,Value> entry = Iterables.getOnlyElement(data);
-
-    Assert.assertEquals(ReplicationSection.getRowPrefix() + wal, entry.getKey().getRow().toString());
-  }
-
-  @Test
-  public void fetchesReplicationEntriesFromMetadataAndReplicationTables() throws Exception {
-    Instance inst = new MockInstance(testName.getMethodName());
-    AccumuloServerContext context = new AccumuloServerContext(new ServerConfigurationFactory(inst));
-    Connector conn = context.getConnector();
-
-    long walCreateTime = System.currentTimeMillis();
-    String wal = "hdfs://localhost:8020/accumulo/wal/tserver+port/123456-1234-1234-12345678";
-    BatchWriter bw = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
-    Mutation m = new Mutation(ReplicationSection.getRowPrefix() + wal);
-    m.put(ReplicationSection.COLF, new Text("1"), StatusUtil.fileCreatedValue(walCreateTime));
-    bw.addMutation(m);
-    bw.close();
-
-    bw = ReplicationTable.getBatchWriter(conn);
-    m = new Mutation(wal);
-    StatusSection.add(m, new Text("1"), StatusUtil.fileCreatedValue(walCreateTime));
-    bw.addMutation(m);
-    bw.close();
-
-    GarbageCollectWriteAheadLogs gcWALs = new GarbageCollectWriteAheadLogs(context, volMgr, false);
-
-    Iterable<Entry<Key,Value>> iter = gcWALs.getReplicationStatusForFile(conn, wal);
-    Map<Key,Value> data = new HashMap<>();
-    for (Entry<Key,Value> e : iter) {
-      data.put(e.getKey(), e.getValue());
-    }
-
-    Assert.assertEquals(2, data.size());
-
-    // Should get one element from each table (metadata and replication)
-    for (Key k : data.keySet()) {
-      String row = k.getRow().toString();
-      if (row.startsWith(ReplicationSection.getRowPrefix())) {
-        Assert.assertTrue(row.endsWith(wal));
-      } else {
-        Assert.assertEquals(wal, row);
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b2539fb1/server/gc/src/test/java/org/apache/accumulo/gc/replication/CloseWriteAheadLogReferencesTest.java
----------------------------------------------------------------------
diff --git a/server/gc/src/test/java/org/apache/accumulo/gc/replication/CloseWriteAheadLogReferencesTest.java b/server/gc/src/test/java/org/apache/accumulo/gc/replication/CloseWriteAheadLogReferencesTest.java
index ba68890..f47f14b 100644
--- a/server/gc/src/test/java/org/apache/accumulo/gc/replication/CloseWriteAheadLogReferencesTest.java
+++ b/server/gc/src/test/java/org/apache/accumulo/gc/replication/CloseWriteAheadLogReferencesTest.java
@@ -46,20 +46,17 @@ import org.apache.accumulo.core.conf.ConfigurationCopy;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.conf.SiteConfiguration;
 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.Value;
 import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.CurrentLogsSection;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.ReplicationSection;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LogColumnFamily;
 import org.apache.accumulo.core.protobuf.ProtobufUtil;
 import org.apache.accumulo.core.replication.ReplicationSchema.StatusSection;
 import org.apache.accumulo.core.replication.ReplicationTable;
 import org.apache.accumulo.core.replication.StatusUtil;
 import org.apache.accumulo.core.replication.proto.Replication.Status;
 import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.tabletserver.log.LogEntry;
 import org.apache.accumulo.core.trace.thrift.TInfo;
 import org.apache.accumulo.server.AccumuloServerContext;
 import org.apache.accumulo.server.conf.ServerConfigurationFactory;
@@ -130,22 +127,16 @@ public class CloseWriteAheadLogReferencesTest {
   public void findOneWalFromMetadata() throws Exception {
     Connector conn = createMock(Connector.class);
     BatchScanner bs = createMock(BatchScanner.class);
-
     // Fake out some data
     final ArrayList<Entry<Key,Value>> data = new ArrayList<>();
-    LogEntry logEntry = new LogEntry();
-    logEntry.extent = new KeyExtent(new Text("1"), new Text("b"), new Text("a"));
-    logEntry.filename = "hdfs://localhost:8020/accumulo/wal/tserver+port/" + UUID.randomUUID();
-    logEntry.server = "tserver1";
-    logEntry.tabletId = 1;
-    logEntry.logSet = Collections.singleton(logEntry.filename);
-    data.add(Maps.immutableEntry(new Key(logEntry.getRow(), logEntry.getColumnFamily(), logEntry.getColumnQualifier()), new Value(logEntry.getValue())));
+    String file = "hdfs://localhost:8020/accumulo/wal/tserver1+9997/" + UUID.randomUUID();
+    data.add(entry("tserver1:9997[1234567890]", file));
 
     // Get a batchscanner, scan the tablets section, fetch only the logs
     expect(conn.createBatchScanner(MetadataTable.NAME, Authorizations.EMPTY, 4)).andReturn(bs);
-    bs.setRanges(Collections.singleton(TabletsSection.getRange()));
+    bs.setRanges(Collections.singleton(CurrentLogsSection.getRange()));
     expectLastCall().once();
-    bs.fetchColumnFamily(LogColumnFamily.NAME);
+    bs.fetchColumnFamily(CurrentLogsSection.COLF);
     expectLastCall().once();
     expect(bs.iterator()).andAnswer(new IAnswer<Iterator<Entry<Key,Value>>>() {
 
@@ -163,54 +154,12 @@ public class CloseWriteAheadLogReferencesTest {
 
     // Validate
     Set<String> wals = refs.getReferencedWals(conn);
-    Assert.assertEquals(Collections.singleton(logEntry.filename), wals);
-
-    verify(conn, bs);
-  }
-
-  @Test
-  public void findManyWalFromSingleMetadata() throws Exception {
-    Connector conn = createMock(Connector.class);
-    BatchScanner bs = createMock(BatchScanner.class);
-
-    // Fake out some data
-    final ArrayList<Entry<Key,Value>> data = new ArrayList<>();
-    LogEntry logEntry = new LogEntry();
-    logEntry.extent = new KeyExtent(new Text("1"), new Text("b"), new Text("a"));
-    logEntry.filename = "hdfs://localhost:8020/accumulo/wal/tserver+port/" + UUID.randomUUID();
-    logEntry.server = "tserver1";
-    logEntry.tabletId = 1;
-    // Multiple DFSLoggers
-    logEntry.logSet = Sets.newHashSet(logEntry.filename, "hdfs://localhost:8020/accumulo/wal/tserver+port/" + UUID.randomUUID());
-    data.add(Maps.immutableEntry(new Key(logEntry.getRow(), logEntry.getColumnFamily(), logEntry.getColumnQualifier()), new Value(logEntry.getValue())));
-
-    // Get a batchscanner, scan the tablets section, fetch only the logs
-    expect(conn.createBatchScanner(MetadataTable.NAME, Authorizations.EMPTY, 4)).andReturn(bs);
-    bs.setRanges(Collections.singleton(TabletsSection.getRange()));
-    expectLastCall().once();
-    bs.fetchColumnFamily(LogColumnFamily.NAME);
-    expectLastCall().once();
-    expect(bs.iterator()).andAnswer(new IAnswer<Iterator<Entry<Key,Value>>>() {
-
-      @Override
-      public Iterator<Entry<Key,Value>> answer() throws Throwable {
-        return data.iterator();
-      }
-
-    });
-    // Close the bs
-    bs.close();
-    expectLastCall().once();
-
-    replay(conn, bs);
-
-    // Validate
-    Set<String> wals = refs.getReferencedWals(conn);
-    Assert.assertEquals(logEntry.logSet, wals);
+    Assert.assertEquals(Collections.singleton(file), wals);
 
     verify(conn, bs);
   }
 
+  // This is a silly test now
   @Test
   public void findManyRefsToSingleWalFromMetadata() throws Exception {
     Connector conn = createMock(Connector.class);
@@ -220,31 +169,14 @@ public class CloseWriteAheadLogReferencesTest {
 
     // Fake out some data
     final ArrayList<Entry<Key,Value>> data = new ArrayList<>();
-    LogEntry logEntry = new LogEntry();
-    logEntry.extent = new KeyExtent(new Text("1"), new Text("b"), new Text("a"));
-    logEntry.filename = "hdfs://localhost:8020/accumulo/wal/tserver+port/" + uuid;
-    logEntry.server = "tserver1";
-    logEntry.tabletId = 1;
-    logEntry.logSet = Collections.singleton(logEntry.filename);
-    data.add(Maps.immutableEntry(new Key(logEntry.getRow(), logEntry.getColumnFamily(), logEntry.getColumnQualifier()), new Value(logEntry.getValue())));
-
-    logEntry.extent = new KeyExtent(new Text("1"), new Text("c"), new Text("b"));
-    logEntry.server = "tserver1";
-    logEntry.tabletId = 2;
-    logEntry.logSet = Collections.singleton(logEntry.filename);
-    data.add(Maps.immutableEntry(new Key(logEntry.getRow(), logEntry.getColumnFamily(), logEntry.getColumnQualifier()), new Value(logEntry.getValue())));
-
-    logEntry.extent = new KeyExtent(new Text("1"), null, new Text("c"));
-    logEntry.server = "tserver1";
-    logEntry.tabletId = 3;
-    logEntry.logSet = Collections.singleton(logEntry.filename);
-    data.add(Maps.immutableEntry(new Key(logEntry.getRow(), logEntry.getColumnFamily(), logEntry.getColumnQualifier()), new Value(logEntry.getValue())));
+    String filename = "hdfs://localhost:8020/accumulo/wal/tserver+9997/" + uuid;
+    data.add(entry("tserver1:9997[0123456789]", filename));
 
     // Get a batchscanner, scan the tablets section, fetch only the logs
     expect(conn.createBatchScanner(MetadataTable.NAME, Authorizations.EMPTY, 4)).andReturn(bs);
-    bs.setRanges(Collections.singleton(TabletsSection.getRange()));
+    bs.setRanges(Collections.singleton(CurrentLogsSection.getRange()));
     expectLastCall().once();
-    bs.fetchColumnFamily(LogColumnFamily.NAME);
+    bs.fetchColumnFamily(CurrentLogsSection.COLF);
     expectLastCall().once();
     expect(bs.iterator()).andAnswer(new IAnswer<Iterator<Entry<Key,Value>>>() {
 
@@ -262,7 +194,7 @@ public class CloseWriteAheadLogReferencesTest {
 
     // Validate
     Set<String> wals = refs.getReferencedWals(conn);
-    Assert.assertEquals(Collections.singleton(logEntry.filename), wals);
+    Assert.assertEquals(Collections.singleton(filename), wals);
 
     verify(conn, bs);
   }
@@ -272,59 +204,22 @@ public class CloseWriteAheadLogReferencesTest {
     Connector conn = createMock(Connector.class);
     BatchScanner bs = createMock(BatchScanner.class);
 
-    String file1 = "hdfs://localhost:8020/accumulo/wal/tserver1+port/" + UUID.randomUUID(), file2 = "hdfs://localhost:8020/accumulo/wal/tserver2+port/"
-        + UUID.randomUUID(), file3 = "hdfs://localhost:8020/accumulo/wal/tserver3+port/" + UUID.randomUUID();
+    String file1 = "hdfs://localhost:8020/accumulo/wal/tserver1+9997/" + UUID.randomUUID();
+    String file2 = "hdfs://localhost:8020/accumulo/wal/tserver2+9997/" + UUID.randomUUID();
+    String file3 = "hdfs://localhost:8020/accumulo/wal/tserver3+9997/" + UUID.randomUUID();
 
     // Fake out some data
     final ArrayList<Entry<Key,Value>> data = new ArrayList<>();
-    LogEntry logEntry = new LogEntry();
-    logEntry.extent = new KeyExtent(new Text("1"), new Text("b"), new Text("a"));
-    logEntry.filename = file1;
-    logEntry.server = "tserver1";
-    logEntry.tabletId = 1;
-    logEntry.logSet = Collections.singleton(logEntry.filename);
-    data.add(Maps.immutableEntry(new Key(logEntry.getRow(), logEntry.getColumnFamily(), logEntry.getColumnQualifier()), new Value(logEntry.getValue())));
-
-    logEntry.extent = new KeyExtent(new Text("5"), null, null);
-    logEntry.tabletId = 2;
-    data.add(Maps.immutableEntry(new Key(logEntry.getRow(), logEntry.getColumnFamily(), logEntry.getColumnQualifier()), new Value(logEntry.getValue())));
-
-    logEntry.extent = new KeyExtent(new Text("3"), new Text("b"), new Text("a"));
-    logEntry.filename = file2;
-    logEntry.server = "tserver2";
-    logEntry.tabletId = 3;
-    logEntry.logSet = Collections.singleton(logEntry.filename);
-    data.add(Maps.immutableEntry(new Key(logEntry.getRow(), logEntry.getColumnFamily(), logEntry.getColumnQualifier()), new Value(logEntry.getValue())));
-
-    logEntry.extent = new KeyExtent(new Text("3"), new Text("c"), new Text("b"));
-    logEntry.tabletId = 4;
-    logEntry.logSet = Collections.singleton(logEntry.filename);
-    data.add(Maps.immutableEntry(new Key(logEntry.getRow(), logEntry.getColumnFamily(), logEntry.getColumnQualifier()), new Value(logEntry.getValue())));
-
-    logEntry.extent = new KeyExtent(new Text("4"), new Text("5"), new Text("0"));
-    logEntry.filename = file3;
-    logEntry.server = "tserver3";
-    logEntry.tabletId = 5;
-    logEntry.logSet = Collections.singleton(logEntry.filename);
-    data.add(Maps.immutableEntry(new Key(logEntry.getRow(), logEntry.getColumnFamily(), logEntry.getColumnQualifier()), new Value(logEntry.getValue())));
-
-    logEntry.extent = new KeyExtent(new Text("4"), new Text("8"), new Text("5"));
-    logEntry.server = "tserver3";
-    logEntry.tabletId = 7;
-    logEntry.logSet = Collections.singleton(logEntry.filename);
-    data.add(Maps.immutableEntry(new Key(logEntry.getRow(), logEntry.getColumnFamily(), logEntry.getColumnQualifier()), new Value(logEntry.getValue())));
-
-    logEntry.extent = new KeyExtent(new Text("4"), null, new Text("8"));
-    logEntry.server = "tserver3";
-    logEntry.tabletId = 15;
-    logEntry.logSet = Collections.singleton(logEntry.filename);
-    data.add(Maps.immutableEntry(new Key(logEntry.getRow(), logEntry.getColumnFamily(), logEntry.getColumnQualifier()), new Value(logEntry.getValue())));
+
+    data.add(entry("tserver1:9997[1234567890]", file1));
+    data.add(entry("tserver2:9997[1234567891]", file2));
+    data.add(entry("tserver3:9997[1234567891]", file3));
 
     // Get a batchscanner, scan the tablets section, fetch only the logs
     expect(conn.createBatchScanner(MetadataTable.NAME, Authorizations.EMPTY, 4)).andReturn(bs);
-    bs.setRanges(Collections.singleton(TabletsSection.getRange()));
+    bs.setRanges(Collections.singleton(CurrentLogsSection.getRange()));
     expectLastCall().once();
-    bs.fetchColumnFamily(LogColumnFamily.NAME);
+    bs.fetchColumnFamily(CurrentLogsSection.COLF);
     expectLastCall().once();
     expect(bs.iterator()).andAnswer(new IAnswer<Iterator<Entry<Key,Value>>>() {
 
@@ -347,6 +242,11 @@ public class CloseWriteAheadLogReferencesTest {
     verify(conn, bs);
   }
 
+  private static Entry<Key,Value> entry(String session, String file) {
+    Key key = new Key(new Text(CurrentLogsSection.getRowPrefix() + session), CurrentLogsSection.COLF, new Text(file));
+    return Maps.immutableEntry(key, new Value());
+  }
+
   @Test
   public void unusedWalsAreClosed() throws Exception {
     Set<String> wals = Collections.emptySet();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b2539fb1/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 5e6dcfb..2434487 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
@@ -421,6 +421,9 @@ public class Master extends AccumuloServerContext implements LiveTServerSet.List
           perm.grantNamespacePermission(user, Namespaces.ACCUMULO_NAMESPACE_ID, NamespacePermission.READ);
         }
         perm.grantNamespacePermission("root", Namespaces.ACCUMULO_NAMESPACE_ID, NamespacePermission.ALTER_TABLE);
+
+        // add the currlog location for root tablet current logs
+        zoo.putPersistentData(ZooUtil.getRoot(getInstance()) + RootTable.ZROOT_TABLET_CURRENT_LOGS, new byte[0], NodeExistsPolicy.SKIP);
         haveUpgradedZooKeeper = true;
       } catch (Exception ex) {
         log.fatal("Error performing upgrade", ex);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b2539fb1/server/master/src/main/java/org/apache/accumulo/master/MasterClientServiceHandler.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/MasterClientServiceHandler.java b/server/master/src/main/java/org/apache/accumulo/master/MasterClientServiceHandler.java
index 3809a29..43939d2 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/MasterClientServiceHandler.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/MasterClientServiceHandler.java
@@ -44,6 +44,7 @@ import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.KeyExtent;
+import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.data.thrift.TKeyExtent;
 import org.apache.accumulo.core.master.thrift.MasterClientService;
@@ -162,7 +163,8 @@ class MasterClientServiceHandler extends FateServiceHandler implements MasterCli
           scanner.setRange(MetadataSchema.TabletsSection.getRange());
         } else {
           scanner = new IsolatedScanner(conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY));
-          scanner.setRange(new KeyExtent(new Text(tableId), null, ByteBufferUtil.toText(startRow)).toMetadataRange());
+          Range range = new KeyExtent(new Text(tableId), null, ByteBufferUtil.toText(startRow)).toMetadataRange();
+          scanner.setRange(range.clip(MetadataSchema.TabletsSection.getRange()));
         }
         TabletsSection.ServerColumnFamily.FLUSH_COLUMN.fetch(scanner);
         TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN.fetch(scanner);


[19/34] accumulo git commit: ACCUMULO-3423 merge apache master

Posted by ec...@apache.org.
ACCUMULO-3423 merge apache master


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

Branch: refs/heads/master
Commit: 8f5e0021a11d507fa00853ce62bc36b4d6654a53
Parents: 9c2ca7a 85d254e
Author: Eric C. Newton <er...@gmail.com>
Authored: Tue Apr 14 15:14:08 2015 -0400
Committer: Eric C. Newton <er...@gmail.com>
Committed: Tue Apr 14 15:14:08 2015 -0400

----------------------------------------------------------------------
 assemble/bin/accumulo_watcher.sh                | 133 +++++++
 assemble/bin/start-server.sh                    |   9 +-
 assemble/conf/templates/accumulo-env.sh         |  14 +
 assemble/pom.xml                                |  32 ++
 assemble/src/main/assemblies/component.xml      |  11 +
 .../main/scripts/generate-versions-listing.sh   |  27 ++
 core/src/main/findbugs/exclude-filter.xml       |   5 +
 .../accumulo/core/bloomfilter/Filter.java       |   2 +
 .../apache/accumulo/core/cli/ClientOpts.java    |  45 +--
 .../accumulo/core/client/BatchDeleter.java      |   4 +-
 .../accumulo/core/client/BatchScanner.java      |  16 +-
 .../accumulo/core/client/IteratorSetting.java   |   8 +-
 .../accumulo/core/client/ScannerBase.java       |  30 +-
 .../core/client/admin/TableOperations.java      |   2 +-
 .../core/client/impl/ClientContext.java         |  10 +-
 .../client/impl/MultiTableBatchWriterImpl.java  |   4 +-
 .../accumulo/core/client/impl/ScannerImpl.java  |   7 +-
 .../core/client/impl/ScannerIterator.java       |   6 +-
 .../core/client/impl/ScannerOptions.java        |  22 +-
 .../core/client/impl/TableOperationsImpl.java   | 218 ----------
 .../client/impl/TabletServerBatchReader.java    |   4 +-
 .../client/impl/TabletServerBatchWriter.java    |   6 +-
 .../core/client/impl/TimeoutTabletLocator.java  |   4 -
 .../core/client/lexicoder/AbstractEncoder.java  |  24 +-
 .../core/client/lexicoder/BytesLexicoder.java   |   7 +-
 .../core/client/lexicoder/ListLexicoder.java    |  12 +-
 .../core/client/lexicoder/UUIDLexicoder.java    |  10 +-
 .../core/client/mapred/AbstractInputFormat.java |  11 +-
 .../client/mapred/AccumuloOutputFormat.java     |  32 +-
 .../client/mapreduce/AbstractInputFormat.java   |  12 +-
 .../client/mapreduce/AccumuloOutputFormat.java  |  32 +-
 .../core/client/mapreduce/InputFormatBase.java  |   9 -
 .../core/client/mapreduce/InputTableConfig.java |  12 -
 .../core/client/mapreduce/RangeInputSplit.java  |  10 +-
 .../mapreduce/lib/impl/InputConfigurator.java   |   3 +-
 .../core/client/mock/MockConfiguration.java     |  23 +-
 .../client/security/tokens/DelegationToken.java |   4 +-
 .../client/security/tokens/KerberosToken.java   |  13 +-
 .../client/security/tokens/PasswordToken.java   |   6 +-
 .../core/conf/AccumuloConfiguration.java        |  56 ++-
 .../accumulo/core/conf/ConfigurationCopy.java   |   6 +-
 .../accumulo/core/conf/ConfigurationDocGen.java |   2 +-
 .../core/conf/DefaultConfiguration.java         |   6 +-
 .../org/apache/accumulo/core/conf/Property.java |  14 +-
 .../accumulo/core/conf/SiteConfiguration.java   |   7 +-
 .../accumulo/core/data/ArrayByteSequence.java   |   1 +
 .../apache/accumulo/core/data/ByteSequence.java |   9 +-
 .../org/apache/accumulo/core/data/Column.java   |   7 +-
 .../accumulo/core/data/ComparableBytes.java     |   1 +
 .../java/org/apache/accumulo/core/data/Key.java |  13 -
 .../apache/accumulo/core/data/KeyExtent.java    |  12 -
 .../org/apache/accumulo/core/data/Mutation.java |  43 +-
 .../org/apache/accumulo/core/data/Value.java    |   2 +
 .../accumulo/core/file/BloomFilterLayer.java    |   2 +-
 .../file/blockfile/cache/CachedBlockQueue.java  |   1 +
 .../file/blockfile/cache/LruBlockCache.java     |  30 +-
 .../file/blockfile/impl/CachableBlockFile.java  |   9 +-
 .../core/file/rfile/bcfile/CompareUtils.java    |   1 +
 .../core/file/rfile/bcfile/Compression.java     |   1 +
 .../accumulo/core/file/rfile/bcfile/Utils.java  |   6 -
 .../accumulo/core/iterators/DebugIterator.java  |   1 +
 .../core/iterators/SortedKeyValueIterator.java  |   3 +-
 .../core/iterators/TypedValueCombiner.java      |   2 +-
 .../iterators/aggregation/LongSummation.java    |   4 +-
 .../aggregation/NumArraySummation.java          |   3 +
 .../iterators/aggregation/NumSummation.java     |   3 +
 .../core/iterators/aggregation/StringMax.java   |   3 +
 .../core/iterators/aggregation/StringMin.java   |   3 +
 .../iterators/aggregation/StringSummation.java  |   3 +
 .../core/iterators/conf/ColumnUtil.java         |   4 +
 .../iterators/system/ColumnQualifierFilter.java |   1 +
 .../core/iterators/system/CountingIterator.java |   1 +
 .../core/iterators/system/DeletingIterator.java |   1 +
 .../core/iterators/system/MultiIterator.java    |   1 +
 .../iterators/system/SequenceFileIterator.java  |   6 +
 .../system/SourceSwitchingIterator.java         |   2 +-
 .../iterators/user/RowEncodingIterator.java     |   3 -
 .../core/metadata/MetadataLocationObtainer.java |   4 +-
 .../security/crypto/BlockedInputStream.java     |   2 +
 .../CachingHDFSSecretKeyEncryptionStrategy.java |  14 +-
 .../crypto/DiscardCloseOutputStream.java        |   1 +
 .../NonCachingSecretKeyEncryptionStrategy.java  |  14 +-
 .../thrift/TDelegationTokenOptions.java         | 399 -------------------
 .../accumulo/core/trace/CountSampler.java       |   3 +
 .../accumulo/core/trace/DistributedTrace.java   |   3 +-
 .../accumulo/core/trace/ProbabilitySampler.java |  30 ++
 .../org/apache/accumulo/core/util/CleanUp.java  |   2 +-
 .../accumulo/core/util/NamingThreadFactory.java |   1 +
 .../org/apache/accumulo/core/util/Stat.java     |   1 +
 .../accumulo/core/util/UtilWaitThread.java      |   2 +-
 .../core/util/format/BinaryFormatter.java       |   1 +
 .../accumulo/core/client/TestThrift1474.java    |   1 +
 .../core/client/impl/ClientContextTest.java     |   7 +-
 .../core/client/impl/ScannerOptionsTest.java    |  24 ++
 .../client/lexicoder/ListLexicoderTest.java     |  10 +-
 .../mapred/AccumuloFileOutputFormatTest.java    |   6 +-
 .../client/mapred/AccumuloInputFormatTest.java  |   5 +-
 .../AccumuloMultiTableInputFormatTest.java      |   7 +-
 .../client/mapred/AccumuloOutputFormatTest.java |   7 +-
 .../mapred/AccumuloRowInputFormatTest.java      |   7 +-
 .../core/client/mapred/TokenFileTest.java       |   1 +
 .../mapreduce/AccumuloFileOutputFormatTest.java |   6 +-
 .../mapreduce/AccumuloInputFormatTest.java      |   6 +-
 .../AccumuloMultiTableInputFormatTest.java      |   7 +-
 .../mapreduce/AccumuloOutputFormatTest.java     |   7 +-
 .../mapreduce/AccumuloRowInputFormatTest.java   |   7 +-
 .../core/client/mapreduce/TokenFileTest.java    |   1 +
 .../core/conf/AccumuloConfigurationTest.java    |  17 +
 .../core/conf/DefaultConfigurationTest.java     |   7 +-
 .../core/conf/ObservableConfigurationTest.java  |   4 +-
 .../core/conf/SiteConfigurationTest.java        |   7 +-
 .../apache/accumulo/core/data/MutationTest.java |  28 ++
 .../apache/accumulo/core/data/ValueTest.java    |   1 +
 .../core/iterators/IteratorUtilTest.java        |   9 +
 .../iterators/aggregation/NumSummationTest.java |   2 +-
 .../core/iterators/user/FilterTest.java         |   2 +
 .../iterators/user/VersioningIteratorTest.java  |   8 +-
 .../main/asciidoc/chapters/administration.txt   | 245 +++++++++++-
 docs/src/main/asciidoc/chapters/design.txt      |  25 +-
 docs/src/main/asciidoc/chapters/replication.txt |  10 +
 .../asciidoc/chapters/table_configuration.txt   |   8 +-
 .../constraints/AlphaNumKeyConstraint.java      |  26 +-
 .../constraints/NumericValueConstraint.java     |  24 +-
 .../simple/filedata/ChunkInputStream.java       |   2 +-
 .../examples/simple/shell/DebugCommand.java     |   3 +
 .../simple/shell/ExampleShellExtension.java     |  37 ++
 .../simple/shell/MyAppShellExtension.java       |  33 --
 ...ache.accumulo.core.util.shell.ShellExtension |   1 -
 .../constraints/AlphaNumKeyConstraintTest.java  |  53 +++
 .../constraints/NumericValueConstraintTest.java |  51 +++
 .../simple/filedata/ChunkInputFormatTest.java   |   5 +-
 .../simple/filedata/ChunkInputStreamTest.java   |   4 +-
 .../org/apache/accumulo/fate/AdminUtil.java     |   2 +-
 .../java/org/apache/accumulo/fate/TStore.java   |   7 -
 .../java/org/apache/accumulo/fate/ZooStore.java |   1 +
 .../accumulo/fate/util/LoggingRunnable.java     |   4 +-
 .../accumulo/fate/util/UtilWaitThread.java      |   2 +-
 .../fate/zookeeper/IZooReaderWriter.java        |   1 +
 .../apache/accumulo/fate/zookeeper/ZooLock.java |   1 +
 .../fate/zookeeper/ZooReaderWriter.java         |   5 -
 .../apache/accumulo/fate/AgeOffStoreTest.java   |   1 +
 .../maven/plugin/AbstractAccumuloMojo.java      |  10 +
 .../apache/accumulo/maven/plugin/StartMojo.java |  16 +-
 .../apache/accumulo/maven/plugin/StopMojo.java  |   6 +-
 .../apache/accumulo/cluster/ClusterUser.java    |   4 +-
 .../accumulo/cluster/RemoteShellOptions.java    |   2 +-
 .../impl/MiniAccumuloClusterImpl.java           |  10 +-
 pom.xml                                         |  24 +-
 .../accumulo/server/AccumuloServerContext.java  |   2 +-
 .../accumulo/server/client/BulkImporter.java    |   6 +-
 .../server/conf/NamespaceConfiguration.java     |  19 +-
 .../server/conf/TableConfiguration.java         |   7 +-
 .../server/conf/ZooCachePropertyAccessor.java   |   7 +-
 .../accumulo/server/conf/ZooConfiguration.java  |  12 +-
 .../server/constraints/MetadataConstraints.java |   8 +-
 .../server/data/ServerColumnUpdate.java         |   1 +
 .../org/apache/accumulo/server/fs/FileRef.java  |   1 +
 .../server/fs/PreferredVolumeChooser.java       |   7 +-
 .../iterators/MetadataBulkLoadFilter.java       |   2 +-
 .../accumulo/server/master/LiveTServerSet.java  |   2 +-
 .../master/balancer/ChaoticLoadBalancer.java    |   6 +-
 .../server/master/state/DeadServerList.java     |   2 +-
 .../server/master/state/TabletMigration.java    |   1 +
 .../accumulo/server/monitor/LogService.java     |   2 +-
 .../server/problems/ProblemReports.java         |   6 +-
 .../replication/ReplicaSystemFactory.java       |  43 +-
 .../server/replication/ReplicaSystemHelper.java |  18 +-
 .../server/replication/ReplicationUtil.java     |  18 +-
 .../server/replication/StatusCombiner.java      |   3 -
 .../apache/accumulo/server/rpc/RpcWrapper.java  |   4 +-
 .../server/rpc/TNonblockingServerSocket.java    |   7 +-
 .../server/security/SecurityOperation.java      |   5 +-
 .../server/security/handler/Authenticator.java  |   6 +
 .../security/handler/ZKAuthenticator.java       |  26 +-
 .../server/security/handler/ZKAuthorizor.java   |  16 +-
 .../server/security/handler/ZKPermHandler.java  |  44 +-
 .../server/security/handler/ZKSecurityTool.java |   6 +-
 .../server/tablets/UniqueNameAllocator.java     |   1 +
 .../org/apache/accumulo/server/util/Admin.java  |   8 +-
 .../accumulo/server/util/ChangeSecret.java      |   2 +
 .../accumulo/server/util/FileSystemMonitor.java |   1 +
 .../apache/accumulo/server/util/FileUtil.java   |  14 +-
 .../org/apache/accumulo/server/util/Halt.java   |   5 +-
 .../accumulo/server/util/ListInstances.java     |   2 +-
 .../server/util/MasterMetadataUtil.java         |   8 +-
 .../accumulo/server/util/MetadataTableUtil.java |   8 +-
 .../accumulo/server/util/RandomWriter.java      |   2 +-
 .../accumulo/server/util/RandomizeVolumes.java  |   2 +-
 .../org/apache/accumulo/server/util/ZooZap.java |   2 +-
 .../accumulo/server/util/time/SimpleTimer.java  |   1 +
 .../base/src/main/scripts/generate-protobuf.sh  |   4 +-
 .../server/conf/NamespaceConfigurationTest.java |  11 +-
 .../server/conf/TableConfigurationTest.java     |  11 +-
 .../conf/ZooCachePropertyAccessorTest.java      |  24 +-
 .../server/master/state/MergeInfoTest.java      |  50 +++
 .../server/replication/ReplicationUtilTest.java | 113 ++++++
 .../security/handler/ZKAuthenticatorTest.java   |   2 +-
 .../accumulo/server/util/FileUtilTest.java      |   4 +-
 .../server/util/time/BaseRelativeTimeTest.java  |   1 +
 .../server/util/time/SimpleTimerTest.java       |   2 +
 .../accumulo/gc/SimpleGarbageCollector.java     |  20 +-
 .../java/org/apache/accumulo/master/Master.java |   2 +-
 .../master/MasterClientServiceHandler.java      |   6 +-
 .../master/recovery/RecoveryManager.java        |   2 +-
 .../master/replication/ReplicationDriver.java   |   4 +-
 .../accumulo/master/state/MergeStats.java       |   2 +-
 .../accumulo/master/tableOps/BulkImport.java    |   2 +-
 .../accumulo/master/tableOps/CloneTable.java    |   2 +-
 .../master/tableOps/CreateNamespace.java        |   2 +-
 .../accumulo/master/tableOps/CreateTable.java   |   2 +-
 .../master/tableOps/DeleteNamespace.java        |   2 +-
 .../accumulo/master/tableOps/DeleteTable.java   |   2 +-
 .../accumulo/master/tableOps/ImportTable.java   |  10 +-
 .../accumulo/master/state/MergeInfoTest.java    |  76 ----
 .../org/apache/accumulo/monitor/Monitor.java    |   7 +-
 .../monitor/servlets/trace/ListType.java        |   1 +
 .../monitor/servlets/trace/NullScanner.java     |   4 +
 .../monitor/servlets/trace/ShowTrace.java       |  97 ++---
 .../monitor/servlets/trace/Summary.java         |   1 +
 .../accumulo/tracer/AsyncSpanReceiver.java      |  15 +-
 .../accumulo/tracer/SendSpansViaThrift.java     |   2 +-
 .../org/apache/accumulo/tracer/TraceDump.java   |  19 +-
 .../org/apache/accumulo/tracer/TraceServer.java |   2 +-
 .../apache/accumulo/tracer/TraceTableStats.java | 157 ++++++++
 .../org/apache/accumulo/tracer/TracerTest.java  |   1 +
 .../tserver/ConditionalMutationSet.java         |   1 +
 .../apache/accumulo/tserver/FileManager.java    |   8 +-
 .../apache/accumulo/tserver/InMemoryMap.java    |   2 +-
 .../org/apache/accumulo/tserver/MemKey.java     |   1 +
 .../apache/accumulo/tserver/TabletServer.java   |  46 ++-
 .../tserver/TabletServerResourceManager.java    |   2 +-
 .../apache/accumulo/tserver/WriteTracker.java   |   2 +-
 .../ConfigurableCompactionStrategy.java         |   1 +
 .../tserver/constraints/ConstraintChecker.java  |   4 +-
 .../constraints/UnsatisfiableConstraint.java    |   2 +
 .../mastermessage/SplitReportMessage.java       |   1 +
 .../mastermessage/TabletStatusMessage.java      |   1 +
 .../replication/AccumuloReplicaSystem.java      | 171 +++++++-
 .../replication/ReplicationProcessor.java       |  22 +-
 .../tserver/replication/ReplicationWorker.java  |  11 +-
 .../accumulo/tserver/tablet/Compactor.java      |  12 +-
 .../tserver/tablet/MinorCompactionTask.java     |  10 +-
 .../accumulo/tserver/tablet/MinorCompactor.java |   6 +-
 .../apache/accumulo/tserver/tablet/Tablet.java  |  17 +-
 .../tserver/tablet/TabletCommitter.java         |   5 +
 .../accumulo/tserver/tablet/TabletMemory.java   |   2 +-
 .../replication/AccumuloReplicaSystemTest.java  |  31 ++
 .../replication/ReplicationProcessorTest.java   |   8 +-
 .../accumulo/shell/ShellCommandException.java   |   2 +
 .../java/org/apache/accumulo/shell/Token.java   |   1 +
 .../shell/commands/AddSplitsCommand.java        |   1 +
 .../shell/commands/CloneTableCommand.java       |   1 +
 .../accumulo/shell/commands/CompactCommand.java |   1 +
 .../accumulo/shell/commands/DebugCommand.java   |   1 +
 .../shell/commands/DeleteManyCommand.java       |   1 +
 .../shell/commands/ExportTableCommand.java      |   1 +
 .../accumulo/shell/commands/FlushCommand.java   |   1 +
 .../shell/commands/FormatterCommand.java        |   2 +
 .../accumulo/shell/commands/HelpCommand.java    |   3 +
 .../shell/commands/ListShellIterCommand.java    |   1 +
 .../accumulo/shell/commands/MaxRowCommand.java  |   1 +
 .../accumulo/shell/commands/MergeCommand.java   |   1 +
 .../accumulo/shell/commands/OfflineCommand.java |   1 +
 .../accumulo/shell/commands/ScriptCommand.java  |   3 +
 .../ShellPluginConfigurationCommand.java        |   8 +-
 .../accumulo/shell/commands/TraceCommand.java   |   1 +
 .../apache/accumulo/shell/mock/MockShell.java   |   1 +
 .../vfs/AccumuloReloadingVFSClassLoader.java    |   4 +-
 .../vfs/providers/HdfsFileObject.java           |  61 ---
 .../vfs/providers/HdfsFileProvider.java         |  10 -
 .../vfs/providers/HdfsFileSystem.java           |  12 -
 .../vfs/providers/HdfsRandomAccessContent.java  | 102 -----
 .../vfs/providers/VfsClassLoaderTest.java       |   3 +
 .../accumulo/test/EstimateInMemMapOverhead.java |  24 ++
 .../accumulo/test/FaultyConditionalWriter.java  |   2 +
 .../accumulo/test/functional/DropModIter.java   |   1 +
 .../accumulo/test/randomwalk/Environment.java   |  36 +-
 .../test/randomwalk/concurrent/Config.java      |   5 +-
 .../test/randomwalk/conditional/Transfer.java   |   1 +
 .../randomwalk/security/SecurityFixture.java    |   8 +
 .../randomwalk/sequential/MapRedVerify.java     |   3 +
 .../randomwalk/sequential/MapRedVerifyTool.java |  56 ++-
 .../test/stress/random/RandomByteArrays.java    |   1 +
 .../accumulo/test/stress/random/Scan.java       |   2 +
 .../accumulo/harness/AccumuloClusterIT.java     |   2 +-
 .../accumulo/harness/SharedMiniClusterIT.java   |   7 +-
 .../conf/AccumuloMiniClusterConfiguration.java  |   2 +-
 .../StandaloneAccumuloClusterConfiguration.java |   2 +-
 .../accumulo/test/ConditionalWriterIT.java      |   2 +-
 .../apache/accumulo/test/MetaGetsReadersIT.java |   3 +-
 .../accumulo/test/MultiTableRecoveryIT.java     |   2 +-
 .../org/apache/accumulo/test/ShellServerIT.java |  69 ++++
 .../accumulo/test/TabletServerGivesUpIT.java    |   1 +
 .../test/replication/KerberosReplicationIT.java | 232 +++++++++++
 .../test/security/KerberosClientOptsTest.java   |  89 +++++
 .../apache/accumulo/test/util/CertUtils.java    |   5 +-
 .../apache/accumulo/trace/instrument/Span.java  |   7 +
 297 files changed, 2840 insertions(+), 1723 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/8f5e0021/core/src/main/java/org/apache/accumulo/core/conf/Property.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/apache/accumulo/core/conf/Property.java
index 943bf19,08ba3a2..7f8726f
--- a/core/src/main/java/org/apache/accumulo/core/conf/Property.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
@@@ -342,8 -346,9 +346,9 @@@ public enum Property 
        + "no longer in use are removed from the filesystem."),
    GC_PORT("gc.port.client", "50091", PropertyType.PORT, "The listening port for the garbage collector's monitor service"),
    GC_DELETE_THREADS("gc.threads.delete", "16", PropertyType.COUNT, "The number of threads used to delete files"),
 -  GC_TRASH_IGNORE("gc.trash.ignore", "false", PropertyType.BOOLEAN, "Do not use the Trash, even if it is configured"),
 -  GC_FILE_ARCHIVE("gc.file.archive", "false", PropertyType.BOOLEAN, "Archive any files/directories instead of moving to the HDFS trash or deleting"),
 +  GC_TRASH_IGNORE("gc.trash.ignore", "false", PropertyType.BOOLEAN, "Do not use the Trash, even if it is configured."),
 +  GC_FILE_ARCHIVE("gc.file.archive", "false", PropertyType.BOOLEAN, "Archive any files/directories instead of moving to the HDFS trash or deleting."),
+   GC_TRACE_PERCENT("gc.trace.percent", "0.01", PropertyType.FRACTION, "Percent of gc cycles to trace"),
  
    // properties that are specific to the monitor server behavior
    MONITOR_PREFIX("monitor.", null, PropertyType.PREFIX, "Properties in this category affect the behavior of the monitor web server."),

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8f5e0021/server/base/src/main/java/org/apache/accumulo/server/util/MasterMetadataUtil.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8f5e0021/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8f5e0021/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8f5e0021/server/master/src/main/java/org/apache/accumulo/master/Master.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8f5e0021/server/master/src/main/java/org/apache/accumulo/master/MasterClientServiceHandler.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8f5e0021/server/master/src/main/java/org/apache/accumulo/master/state/MergeStats.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8f5e0021/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8f5e0021/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
----------------------------------------------------------------------
diff --cc server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
index fb0adb8,2342789..febd4f7
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
@@@ -2440,9 -2440,9 +2439,9 @@@ public class Tablet implements TabletCo
      return currentLogs.size();
    }
  
-   /* don't release the lock if this method returns true for success; instead, the caller should clean up by calling finishUpdatingLogsUsed() */
+   // don't release the lock if this method returns true for success; instead, the caller should clean up by calling finishUpdatingLogsUsed()
    @Override
 -  public boolean beginUpdatingLogsUsed(InMemoryMap memTable, Collection<DfsLogger> more, boolean mincFinish) {
 +  public boolean beginUpdatingLogsUsed(InMemoryMap memTable, DfsLogger more, boolean mincFinish) {
  
      boolean releaseLock = true;
  

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8f5e0021/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletCommitter.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8f5e0021/test/src/test/java/org/apache/accumulo/test/ConditionalWriterIT.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8f5e0021/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
----------------------------------------------------------------------


[23/34] accumulo git commit: ACCUMULO-3423 got most ITs working again

Posted by ec...@apache.org.
ACCUMULO-3423 got most ITs working again


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

Branch: refs/heads/master
Commit: 35e3f12c031b6ddf2c3936c77c1ccec8f981ce72
Parents: 0177e3f
Author: Eric C. Newton <er...@gmail.com>
Authored: Tue Apr 21 13:26:17 2015 -0400
Committer: Eric C. Newton <er...@gmail.com>
Committed: Tue Apr 21 13:26:17 2015 -0400

----------------------------------------------------------------------
 .../org/apache/accumulo/tserver/log/TabletServerLogger.java | 9 ---------
 .../org/apache/accumulo/test/functional/ReadWriteIT.java    | 8 ++++++++
 2 files changed, 8 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/35e3f12c/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
index 2541e50..cdee51b 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
@@ -20,7 +20,6 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -54,7 +53,6 @@ import org.apache.accumulo.tserver.TabletMutations;
 import org.apache.accumulo.tserver.TabletServer;
 import org.apache.accumulo.tserver.log.DfsLogger.LoggerOperation;
 import org.apache.accumulo.tserver.tablet.CommitSession;
-import org.apache.accumulo.tserver.tablet.Tablet;
 import org.apache.hadoop.fs.Path;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -229,13 +227,6 @@ public class TabletServerLogger {
             log.debug("Creating next WAL");
             DfsLogger alog = new DfsLogger(tserver.getServerConfig(), syncCounter, flushCounter);
             alog.open(tserver.getClientAddressString());
-            EnumSet<TabletLevel> levels = EnumSet.noneOf(TabletLevel.class);
-            for (Tablet tablet : tserver.getOnlineTablets()) {
-              levels.add(TabletLevel.getLevel(tablet.getExtent()));
-            }
-            for (TabletLevel level : levels) {
-              tserver.addLoggersToMetadata(alog, level);
-            }
             log.debug("Created next WAL " + alog.getFileName());
             while (!nextLog.offer(alog, 12, TimeUnit.HOURS)) {
               log.info("Our WAL was not used for 12 hours: " + alog.getFileName());

http://git-wip-us.apache.org/repos/asf/accumulo/blob/35e3f12c/test/src/test/java/org/apache/accumulo/test/functional/ReadWriteIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/ReadWriteIT.java b/test/src/test/java/org/apache/accumulo/test/functional/ReadWriteIT.java
index 099743d..1f3e600 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/ReadWriteIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/ReadWriteIT.java
@@ -57,6 +57,7 @@ import org.apache.accumulo.core.client.admin.TableOperations;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.client.security.tokens.KerberosToken;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Range;
@@ -72,9 +73,11 @@ import org.apache.accumulo.fate.zookeeper.ZooLock;
 import org.apache.accumulo.fate.zookeeper.ZooReader;
 import org.apache.accumulo.harness.AccumuloClusterIT;
 import org.apache.accumulo.minicluster.ServerType;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.test.TestIngest;
 import org.apache.accumulo.test.TestMultiTableIngest;
 import org.apache.accumulo.test.VerifyIngest;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
 import org.junit.Test;
@@ -84,6 +87,11 @@ import org.slf4j.LoggerFactory;
 import com.google.common.base.Charsets;
 
 public class ReadWriteIT extends AccumuloClusterIT {
+  @Override
+  public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
+    cfg.setProperty(Property.INSTANCE_ZK_TIMEOUT, "5s");
+  }
+
   private static final Logger log = LoggerFactory.getLogger(ReadWriteIT.class);
 
   static final int ROWS = 200000;


[24/34] accumulo git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/accumulo

Posted by ec...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/accumulo


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

Branch: refs/heads/master
Commit: ed7c4f6c521af124171c757fb9b81dd7dbfb046d
Parents: 35e3f12 321f573
Author: Eric C. Newton <er...@gmail.com>
Authored: Tue Apr 21 13:26:38 2015 -0400
Committer: Eric C. Newton <er...@gmail.com>
Committed: Tue Apr 21 13:26:38 2015 -0400

----------------------------------------------------------------------
 assemble/pom.xml                                |   2 +-
 core/pom.xml                                    |   2 +-
 .../mapreduce/lib/impl/InputConfigurator.java   |   5 +-
 docs/pom.xml                                    |   2 +-
 examples/simple/pom.xml                         |   2 +-
 fate/pom.xml                                    |   2 +-
 maven-plugin/pom.xml                            |   2 +-
 minicluster/pom.xml                             |   2 +-
 pom.xml                                         |   2 +-
 proxy/pom.xml                                   |   2 +-
 server/base/pom.xml                             |   2 +-
 server/gc/pom.xml                               |   2 +-
 server/master/pom.xml                           |   2 +-
 server/monitor/pom.xml                          |   2 +-
 server/native/pom.xml                           |   2 +-
 server/tracer/pom.xml                           |   2 +-
 server/tserver/pom.xml                          |   2 +-
 .../tserver/log/LocalWALRecoveryTest.java       |   2 +-
 shell/pom.xml                                   |   2 +-
 start/pom.xml                                   |   2 +-
 test/pom.xml                                    |   2 +-
 .../accumulo/test/AccumuloOutputFormatIT.java   | 125 +++++++++++++++++++
 .../accumulo/test/AccumuloOutputFormatTest.java | 114 -----------------
 .../test/functional/AccumuloInputFormatIT.java  |  16 ++-
 trace/pom.xml                                   |   2 +-
 25 files changed, 159 insertions(+), 143 deletions(-)
----------------------------------------------------------------------



[32/34] accumulo git commit: ACCUMULO-3488 ACCUMULO-3423 sync these two changes

Posted by ec...@apache.org.
ACCUMULO-3488 ACCUMULO-3423 sync these two changes


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

Branch: refs/heads/master
Commit: 55981ad881e7b79423799a3473856b4b9d768689
Parents: 3fdd29f
Author: Eric C. Newton <er...@gmail.com>
Authored: Fri Apr 24 19:05:50 2015 -0400
Committer: Eric C. Newton <er...@gmail.com>
Committed: Fri Apr 24 19:05:50 2015 -0400

----------------------------------------------------------------------
 .../base/src/main/java/org/apache/accumulo/server/TabletLevel.java | 2 +-
 .../accumulo/gc/replication/CloseWriteAheadLogReferencesTest.java  | 1 -
 .../test/java/org/apache/accumulo/tserver/log/LogEntryTest.java    | 2 +-
 .../apache/accumulo/test/MissingWalHeaderCompletesRecoveryIT.java  | 2 +-
 .../java/org/apache/accumulo/test/functional/WALSunnyDayIT.java    | 2 +-
 .../java/org/apache/accumulo/test/replication/ReplicationIT.java   | 2 +-
 6 files changed, 5 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/55981ad8/server/base/src/main/java/org/apache/accumulo/server/TabletLevel.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/TabletLevel.java b/server/base/src/main/java/org/apache/accumulo/server/TabletLevel.java
index 91e5ee9..c87c7e1 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/TabletLevel.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/TabletLevel.java
@@ -16,7 +16,7 @@
  */
 package org.apache.accumulo.server;
 
-import org.apache.accumulo.core.data.KeyExtent;
+import org.apache.accumulo.core.data.impl.KeyExtent;
 
 public enum TabletLevel {
   ROOT,

http://git-wip-us.apache.org/repos/asf/accumulo/blob/55981ad8/server/gc/src/test/java/org/apache/accumulo/gc/replication/CloseWriteAheadLogReferencesTest.java
----------------------------------------------------------------------
diff --git a/server/gc/src/test/java/org/apache/accumulo/gc/replication/CloseWriteAheadLogReferencesTest.java b/server/gc/src/test/java/org/apache/accumulo/gc/replication/CloseWriteAheadLogReferencesTest.java
index 78a5bd5..9fcfec9 100644
--- a/server/gc/src/test/java/org/apache/accumulo/gc/replication/CloseWriteAheadLogReferencesTest.java
+++ b/server/gc/src/test/java/org/apache/accumulo/gc/replication/CloseWriteAheadLogReferencesTest.java
@@ -48,7 +48,6 @@ import org.apache.accumulo.core.conf.SiteConfiguration;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
-import org.apache.accumulo.core.data.impl.KeyExtent;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.CurrentLogsSection;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.ReplicationSection;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/55981ad8/server/tserver/src/test/java/org/apache/accumulo/tserver/log/LogEntryTest.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/log/LogEntryTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/log/LogEntryTest.java
index 44058d3..ee3c621 100644
--- a/server/tserver/src/test/java/org/apache/accumulo/tserver/log/LogEntryTest.java
+++ b/server/tserver/src/test/java/org/apache/accumulo/tserver/log/LogEntryTest.java
@@ -19,8 +19,8 @@ package org.apache.accumulo.tserver.log;
 import static org.junit.Assert.assertEquals;
 
 import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.data.impl.KeyExtent;
 import org.apache.accumulo.core.tabletserver.log.LogEntry;
 import org.apache.hadoop.io.Text;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/55981ad8/test/src/test/java/org/apache/accumulo/test/MissingWalHeaderCompletesRecoveryIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/MissingWalHeaderCompletesRecoveryIT.java b/test/src/test/java/org/apache/accumulo/test/MissingWalHeaderCompletesRecoveryIT.java
index 27f1f69..e315841 100644
--- a/test/src/test/java/org/apache/accumulo/test/MissingWalHeaderCompletesRecoveryIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/MissingWalHeaderCompletesRecoveryIT.java
@@ -26,8 +26,8 @@ import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.conf.Property;
-import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.impl.KeyExtent;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema;
 import org.apache.accumulo.core.security.Authorizations;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/55981ad8/test/src/test/java/org/apache/accumulo/test/functional/WALSunnyDayIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/WALSunnyDayIT.java b/test/src/test/java/org/apache/accumulo/test/functional/WALSunnyDayIT.java
index 490bd7c..9f22466 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/WALSunnyDayIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/WALSunnyDayIT.java
@@ -40,9 +40,9 @@ import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Scanner;
 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.Value;
+import org.apache.accumulo.core.data.impl.KeyExtent;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.RootTable;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.CurrentLogsSection;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/55981ad8/test/src/test/java/org/apache/accumulo/test/replication/ReplicationIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/replication/ReplicationIT.java b/test/src/test/java/org/apache/accumulo/test/replication/ReplicationIT.java
index 3c1cbeb..8bd5641 100644
--- a/test/src/test/java/org/apache/accumulo/test/replication/ReplicationIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/replication/ReplicationIT.java
@@ -46,10 +46,10 @@ import org.apache.accumulo.core.client.ZooKeeperInstance;
 import org.apache.accumulo.core.client.admin.TableOperations;
 import org.apache.accumulo.core.conf.Property;
 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.Range;
 import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.data.impl.KeyExtent;
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
 import org.apache.accumulo.core.iterators.conf.ColumnSet;
 import org.apache.accumulo.core.metadata.MetadataTable;


[08/34] accumulo git commit: ACCUMULO-3638 merge master branch

Posted by ec...@apache.org.
ACCUMULO-3638 merge master branch


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

Branch: refs/heads/master
Commit: 4635de8671a62f0a412da775db05519f6831daaa
Parents: 98c3cef 9f108c0
Author: Eric C. Newton <er...@gmail.com>
Authored: Tue Mar 10 16:16:53 2015 -0400
Committer: Eric C. Newton <er...@gmail.com>
Committed: Tue Mar 10 16:16:53 2015 -0400

----------------------------------------------------------------------
 core/pom.xml                                    |  7 --
 core/src/main/findbugs/exclude-filter.xml       | 67 +++++++++++++-----
 .../client/admin/CompactionStrategyConfig.java  |  5 ++
 .../client/impl/MultiTableBatchWriterImpl.java  | 13 ++--
 .../core/client/impl/TabletLocatorImpl.java     | 41 +++++++----
 .../client/impl/TabletServerBatchWriter.java    | 45 ++++++++----
 .../core/client/mock/MockNamespace.java         |  2 +-
 .../client/security/tokens/DelegationToken.java | 16 ++---
 .../client/security/tokens/PasswordToken.java   | 13 ++--
 .../apache/accumulo/core/conf/PropertyType.java |  2 +-
 .../org/apache/accumulo/core/data/Value.java    |  1 +
 .../apache/accumulo/core/volume/VolumeImpl.java |  6 ++
 .../accumulo/core/cli/TestClientOpts.java       |  2 +-
 .../core/file/rfile/RFileMetricsTest.java       | 73 +++++++++-----------
 .../iterators/user/ColumnSliceFilterTest.java   |  2 +-
 .../core/iterators/user/RowFilterTest.java      |  2 +-
 .../simple/src/main/findbugs/exclude-filter.xml | 18 +++++
 .../examples/simple/mapreduce/RowHash.java      |  2 +-
 fate/src/main/findbugs/exclude-filter.xml       | 18 +++++
 .../accumulo/fate/zookeeper/ZooReader.java      | 14 ++--
 .../zookeeper/DistributedReadWriteLockTest.java | 17 +++--
 .../src/main/findbugs/exclude-filter.xml        | 18 +++++
 .../src/main/findbugs/exclude-filter.xml        | 30 ++++++++
 pom.xml                                         |  3 +-
 proxy/pom.xml                                   | 13 ----
 proxy/src/main/findbugs/exclude-filter.xml      |  7 +-
 .../base/src/main/findbugs/exclude-filter.xml   | 26 +++++++
 .../server/master/balancer/GroupBalancer.java   |  6 ++
 .../master/state/TabletLocationState.java       |  3 +-
 .../server/metrics/MetricsConfiguration.java    |  7 +-
 .../TCredentialsUpdatingInvocationHandler.java  |  5 +-
 .../server/security/SecurityOperation.java      |  7 --
 .../security/delegation/AuthenticationKey.java  | 11 +--
 .../security/handler/KerberosAuthenticator.java |  4 +-
 .../server/watcher/Log4jConfiguration.java      |  5 +-
 .../accumulo/server/ServerConstantsTest.java    | 12 +++-
 .../AuthenticationTokenKeyManagerTest.java      | 34 +++++----
 server/gc/src/main/findbugs/exclude-filter.xml  | 18 +++++
 .../gc/GarbageCollectWriteAheadLogs.java        |  2 +-
 .../master/src/main/findbugs/exclude-filter.xml | 23 ++++++
 .../java/org/apache/accumulo/master/Master.java | 24 ++++---
 .../RemoveCompleteReplicationRecords.java       |  3 +
 .../src/main/findbugs/exclude-filter.xml        | 18 +++++
 .../monitor/servlets/OperationServlet.java      | 53 +++++++++-----
 server/tracer/pom.xml                           | 13 ----
 .../tracer/src/main/findbugs/exclude-filter.xml |  7 +-
 .../src/main/findbugs/exclude-filter.xml        | 24 +++++++
 .../accumulo/server/logger/LogFileKey.java      | 25 -------
 .../accumulo/server/logger/LogFileValue.java    | 25 -------
 .../accumulo/tserver/log/LocalWALRecovery.java  |  4 +-
 .../replication/AccumuloReplicaSystem.java      | 19 ++++-
 .../apache/accumulo/tserver/scan/ScanTask.java  | 30 ++++++--
 .../apache/accumulo/tserver/tablet/Tablet.java  | 14 ++--
 .../tserver/tablet/TabletCommitter.java         |  3 +
 shell/src/main/findbugs/exclude-filter.xml      | 18 +++++
 .../apache/accumulo/shell/ShellConfigTest.java  | 29 ++++++--
 .../org/apache/accumulo/shell/ShellTest.java    | 11 ++-
 .../shell/commands/FormatterCommandTest.java    |  6 +-
 start/src/main/findbugs/exclude-filter.xml      | 18 +++++
 .../start/classloader/vfs/MiniDFSUtil.java      |  4 ++
 test/src/main/findbugs/exclude-filter.xml       | 30 ++++++++
 .../server/security/SystemCredentialsIT.java    |  2 +
 .../test/MasterRepairsDualAssignmentIT.java     |  2 +-
 .../accumulo/test/MultiTableRecoveryIT.java     |  3 +-
 .../java/org/apache/accumulo/test/UsersIT.java  | 60 ++++++++++++++++
 .../apache/accumulo/test/functional/BulkIT.java | 13 ++--
 .../test/functional/ConfigurableMacIT.java      |  4 +-
 .../accumulo/test/functional/MonitorSslIT.java  | 40 ++++++-----
 .../accumulo/test/functional/PermissionsIT.java | 43 ++++++++----
 .../apache/accumulo/test/functional/SslIT.java  |  5 +-
 trace/src/main/findbugs/exclude-filter.xml      | 26 +++++++
 71 files changed, 810 insertions(+), 366 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/4635de86/server/base/src/main/java/org/apache/accumulo/server/master/state/TabletLocationState.java
----------------------------------------------------------------------
diff --cc server/base/src/main/java/org/apache/accumulo/server/master/state/TabletLocationState.java
index 3ece3c9,b24b562..ebad2c8
--- a/server/base/src/main/java/org/apache/accumulo/server/master/state/TabletLocationState.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/state/TabletLocationState.java
@@@ -33,8 -32,6 +32,8 @@@ import org.apache.hadoop.io.Text
   */
  public class TabletLocationState {
  
-   private static final Logger log = Logger.getLogger(TabletLocationState.class);
++  // private static final Logger log = Logger.getLogger(TabletLocationState.class);
 +
    static public class BadLocationStateException extends Exception {
      private static final long serialVersionUID = 1L;
      private Text metadataTableEntry;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/4635de86/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogs.java
----------------------------------------------------------------------
diff --cc server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogs.java
index 444789b,35c60d6..f44a9d1
--- a/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogs.java
+++ b/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogs.java
@@@ -214,39 -268,76 +214,39 @@@ public class GarbageCollectWriteAheadLo
          }
        }
      }
 -
 -    return 0;
 +    return status.currentLog.deleted;
    }
  
 -  /**
 -   * Converts a list of paths to their corresponding strings.
 -   *
 -   * @param paths
 -   *          list of paths
 -   * @return string forms of paths
 -   */
 -  static List<String> paths2strings(List<Path> paths) {
 -    List<String> result = new ArrayList<String>(paths.size());
 -    for (Path path : paths)
 -      result.add(path.toString());
 -    return result;
 -  }
 +  private long removeMetadataEntries(Map<TServerInstance, Set<Path> > candidates, GCStatus status, Set<TServerInstance> liveServers) throws IOException, KeeperException,
 +      InterruptedException {
  
 -  /**
 -   * Reverses the given mapping of file paths to servers. The returned map provides a list of file paths for each server. Any path whose name is not in the
 -   * mapping of file names to paths is skipped.
 -   *
 -   * @param fileToServerMap
 -   *          map of file paths to servers
 -   * @param nameToFileMap
 -   *          map of file names to paths
 -   * @return map of servers to lists of file paths
 -   */
 -  static Map<String,ArrayList<Path>> mapServersToFiles(Map<Path,String> fileToServerMap, Map<String,Path> nameToFileMap) {
 -    Map<String,ArrayList<Path>> result = new HashMap<String,ArrayList<Path>>();
 -    for (Entry<Path,String> fileServer : fileToServerMap.entrySet()) {
 -      if (!nameToFileMap.containsKey(fileServer.getKey().getName()))
 -        continue;
 -      ArrayList<Path> files = result.get(fileServer.getValue());
 -      if (files == null) {
 -        files = new ArrayList<Path>();
 -        result.put(fileServer.getValue(), files);
 +    // remove any entries if there's a log reference, or a tablet is still assigned to the dead server
 +
 +    Map<Path, TServerInstance> walToDeadServer = new HashMap<>();
 +    for (Entry<TServerInstance,Set<Path>> entry : candidates.entrySet()) {
 +      for (Path file : entry.getValue()) {
 +        walToDeadServer.put(file, entry.getKey());
        }
 -      files.add(fileServer.getKey());
      }
 -    return result;
 -  }
 -
 -  protected int removeMetadataEntries(Map<String,Path> nameToFileMap, Map<String,Path> sortedWALogs, GCStatus status) throws IOException, KeeperException,
 -      InterruptedException {
 -    int count = 0;
 -    Iterator<LogEntry> iterator = MetadataTableUtil.getLogEntries(context);
 -
 -    // For each WAL reference in the metadata table
 -    while (iterator.hasNext()) {
 -      // Each metadata reference has at least one WAL file
 -      for (String entry : iterator.next().logSet) {
 -        // old style WALs will have the IP:Port of their logger and new style will either be a Path either absolute or relative, in all cases
 -        // the last "/" will mark a UUID file name.
 -        String uuid = entry.substring(entry.lastIndexOf("/") + 1);
 -        if (!isUUID(uuid)) {
 -          // fully expect this to be a uuid, if its not then something is wrong and walog GC should not proceed!
 -          throw new IllegalArgumentException("Expected uuid, but got " + uuid + " from " + entry);
 -        }
 -
 -        Path pathFromNN = nameToFileMap.remove(uuid);
 -        if (pathFromNN != null) {
 -          status.currentLog.inUse++;
 -          sortedWALogs.remove(uuid);
 +    long count = 0;
 +    RootTabletStateStore root = new RootTabletStateStore(context);
 +    MetaDataStateStore meta = new MetaDataStateStore(context);
 +    Iterator<TabletLocationState> states = Iterators.concat(root.iterator(), meta.iterator());
 +    while (states.hasNext()) {
 +      count++;
 +      TabletLocationState state = states.next();
 +      if (state.getState(liveServers) == TabletState.ASSIGNED_TO_DEAD_SERVER) {
 +        candidates.remove(state.current);
 +      }
 +      for (Collection<String> wals : state.walogs) {
 +        for (String wal : wals) {
-           TServerInstance dead = walToDeadServer.get(wal);
++          TServerInstance dead = walToDeadServer.get(new Path(wal));
 +          if (dead != null) {
 +            candidates.get(dead).remove(wal);
 +          }
          }
 -
 -        count++;
        }
      }
 -
      return count;
    }
  

http://git-wip-us.apache.org/repos/asf/accumulo/blob/4635de86/server/master/src/main/java/org/apache/accumulo/master/Master.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/4635de86/server/tserver/src/main/findbugs/exclude-filter.xml
----------------------------------------------------------------------
diff --cc server/tserver/src/main/findbugs/exclude-filter.xml
index 0000000,aec413a..45f6a78
mode 000000,100644..100644
--- a/server/tserver/src/main/findbugs/exclude-filter.xml
+++ b/server/tserver/src/main/findbugs/exclude-filter.xml
@@@ -1,0 -1,24 +1,24 @@@
+ <!--
+   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.
+ -->
+ <FindBugsFilter>
+   <Match>
+     <!-- locking is confusing, but probably correct -->
+     <Class name="org.apache.accumulo.tserver.tablet.Tablet" />
 -    <Method name="beginUpdatingLogsUsed" params="org.apache.accumulo.tserver.InMemoryMap,java.util.Collection,boolean" returns="boolean" />
++    <Method name="beginUpdatingLogsUsed" params="org.apache.accumulo.tserver.InMemoryMap,org.apache.accumulo.tserver.log.DfsLogger,boolean" returns="boolean" />
+     <Bug code="UL" pattern="UL_UNRELEASED_LOCK" />
+   </Match>
+ </FindBugsFilter>

http://git-wip-us.apache.org/repos/asf/accumulo/blob/4635de86/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
----------------------------------------------------------------------
diff --cc server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
index 0c1edfa,95fe24a..e964719
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
@@@ -2442,8 -2441,9 +2440,9 @@@ public class Tablet implements TabletCo
      return currentLogs.size();
    }
  
+   /* don't release the lock if this method returns true for success; instead, the caller should clean up by calling finishUpdatingLogsUsed() */
    @Override
 -  public boolean beginUpdatingLogsUsed(InMemoryMap memTable, Collection<DfsLogger> more, boolean mincFinish) {
 +  public boolean beginUpdatingLogsUsed(InMemoryMap memTable, DfsLogger more, boolean mincFinish) {
  
      boolean releaseLock = true;
  

http://git-wip-us.apache.org/repos/asf/accumulo/blob/4635de86/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletCommitter.java
----------------------------------------------------------------------
diff --cc server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletCommitter.java
index 39bde5c,4bc05a6..91a955e
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletCommitter.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletCommitter.java
@@@ -34,7 -35,10 +34,10 @@@ public interface TabletCommitter 
  
    void commit(CommitSession commitSession, List<Mutation> mutations);
  
+   /**
+    * If this method returns true, the caller must call {@link #finishUpdatingLogsUsed()} to clean up
+    */
 -  boolean beginUpdatingLogsUsed(InMemoryMap memTable, Collection<DfsLogger> copy, boolean mincFinish);
 +  boolean beginUpdatingLogsUsed(InMemoryMap memTable, DfsLogger copy, boolean mincFinish);
  
    void finishUpdatingLogsUsed();
  


[30/34] accumulo git commit: ACCUMULO-3423 optimize WAL metadata table updates

Posted by ec...@apache.org.
ACCUMULO-3423 optimize WAL metadata table updates


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

Branch: refs/heads/master
Commit: 3fdd29f5222f9d1d32ca28b5ecf1d740a8d20f87
Parents: ea25e98
Author: Eric C. Newton <er...@gmail.com>
Authored: Fri Apr 24 18:15:05 2015 -0400
Committer: Eric C. Newton <er...@gmail.com>
Committed: Fri Apr 24 18:18:56 2015 -0400

----------------------------------------------------------------------
 .../client/impl/ReplicationOperationsImpl.java  |   4 +-
 .../org/apache/accumulo/core/conf/Property.java |   4 +-
 .../accumulo/core/metadata/RootTable.java       |   1 +
 .../core/metadata/schema/MetadataSchema.java    |  48 ++
 .../core/tabletserver/log/LogEntry.java         |  78 ++-
 .../core/metadata/MetadataTableSchemaTest.java  |  47 ++
 .../org/apache/accumulo/server/TabletLevel.java |  34 ++
 .../apache/accumulo/server/fs/VolumeUtil.java   |  22 +-
 .../apache/accumulo/server/init/Initialize.java |   1 +
 .../server/master/state/MetaDataStateStore.java |  47 +-
 .../master/state/MetaDataTableScanner.java      |   6 +-
 .../master/state/TabletLocationState.java       |   7 +
 .../server/master/state/TabletStateStore.java   |  16 +-
 .../master/state/ZooTabletStateStore.java       |  35 +-
 .../accumulo/server/replication/StatusUtil.java |  13 +
 .../accumulo/server/util/ListVolumesUsed.java   |  18 +-
 .../server/util/MasterMetadataUtil.java         |  18 +-
 .../accumulo/server/util/MetadataTableUtil.java | 239 +++++---
 .../server/util/ReplicationTableUtil.java       |  13 +-
 .../server/util/ReplicationTableUtilTest.java   |   2 +-
 .../gc/GarbageCollectWriteAheadLogs.java        | 499 +++++++---------
 .../accumulo/gc/SimpleGarbageCollector.java     |   1 -
 .../CloseWriteAheadLogReferences.java           |  23 +-
 .../gc/GarbageCollectWriteAheadLogsTest.java    | 567 -------------------
 .../CloseWriteAheadLogReferencesTest.java       | 151 +----
 .../java/org/apache/accumulo/master/Master.java |   3 +
 .../master/MasterClientServiceHandler.java      |   3 +-
 .../accumulo/master/TabletGroupWatcher.java     |  37 +-
 .../accumulo/master/replication/WorkMaker.java  |   1 +
 .../accumulo/master/state/MergeStats.java       |   3 +-
 .../master/ReplicationOperationsImplTest.java   |   9 +-
 .../apache/accumulo/master/TestMergeState.java  |   2 +-
 .../master/state/RootTabletStateStoreTest.java  |   4 +-
 .../src/main/findbugs/exclude-filter.xml        |   2 +-
 .../server/GarbageCollectionLogger.java         |   3 +-
 .../apache/accumulo/tserver/TabletServer.java   | 182 +++---
 .../apache/accumulo/tserver/log/DfsLogger.java  |  14 +-
 .../accumulo/tserver/log/SortedLogRecovery.java |   8 +-
 .../tserver/log/TabletServerLogger.java         | 187 +++---
 .../accumulo/tserver/tablet/CommitSession.java  |   3 +-
 .../tserver/tablet/DatafileManager.java         |   4 +-
 .../apache/accumulo/tserver/tablet/Tablet.java  |  59 +-
 .../tserver/tablet/TabletCommitter.java         |   3 +-
 .../accumulo/tserver/log/LogEntryTest.java      |  56 ++
 .../test/performance/thrift/NullTserver.java    |   6 +-
 .../accumulo/proxy/ProxyDurabilityIT.java       |   9 +-
 .../test/BadDeleteMarkersCreatedIT.java         |   2 +-
 .../org/apache/accumulo/test/BalanceIT.java     |  20 +-
 .../org/apache/accumulo/test/CleanWalIT.java    |   1 +
 .../accumulo/test/ConditionalWriterIT.java      |   1 +
 .../accumulo/test/GarbageCollectWALIT.java      |  81 +++
 .../MissingWalHeaderCompletesRecoveryIT.java    |  14 +-
 .../accumulo/test/NoMutationRecoveryIT.java     | 178 ------
 .../org/apache/accumulo/test/ShellServerIT.java |   2 +-
 .../org/apache/accumulo/test/UnusedWALIT.java   | 144 +++++
 .../java/org/apache/accumulo/test/VolumeIT.java |  17 +
 .../accumulo/test/functional/ReadWriteIT.java   |   8 +
 .../accumulo/test/functional/WALSunnyDayIT.java | 250 ++++++++
 .../test/functional/WatchTheWatchCountIT.java   |   2 +-
 .../test/performance/RollWALPerformanceIT.java  | 126 +++++
 ...bageCollectorCommunicatesWithTServersIT.java |  35 +-
 .../replication/MultiInstanceReplicationIT.java |   2 +-
 .../test/replication/ReplicationIT.java         | 370 ++++--------
 63 files changed, 1857 insertions(+), 1888 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/core/src/main/java/org/apache/accumulo/core/client/impl/ReplicationOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ReplicationOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ReplicationOperationsImpl.java
index 6a5c74a..925877d 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ReplicationOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ReplicationOperationsImpl.java
@@ -153,9 +153,7 @@ public class ReplicationOperationsImpl implements ReplicationOperations {
     try {
       for (Entry<Key,Value> entry : metaBs) {
         LogEntry logEntry = LogEntry.fromKeyValue(entry.getKey(), entry.getValue());
-        for (String log : logEntry.logSet) {
-          wals.add(new Path(log).toString());
-        }
+        wals.add(new Path(logEntry.filename).toString());
       }
     } finally {
       metaBs.close();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/core/src/main/java/org/apache/accumulo/core/conf/Property.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/conf/Property.java b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
index 429abad..a5bef0a 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/Property.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/Property.java
@@ -345,8 +345,8 @@ public enum Property {
       + "no longer in use are removed from the filesystem."),
   GC_PORT("gc.port.client", "50091", PropertyType.PORT, "The listening port for the garbage collector's monitor service"),
   GC_DELETE_THREADS("gc.threads.delete", "16", PropertyType.COUNT, "The number of threads used to delete files"),
-  GC_TRASH_IGNORE("gc.trash.ignore", "false", PropertyType.BOOLEAN, "Do not use the Trash, even if it is configured"),
-  GC_FILE_ARCHIVE("gc.file.archive", "false", PropertyType.BOOLEAN, "Archive any files/directories instead of moving to the HDFS trash or deleting"),
+  GC_TRASH_IGNORE("gc.trash.ignore", "false", PropertyType.BOOLEAN, "Do not use the Trash, even if it is configured."),
+  GC_FILE_ARCHIVE("gc.file.archive", "false", PropertyType.BOOLEAN, "Archive any files/directories instead of moving to the HDFS trash or deleting."),
   GC_TRACE_PERCENT("gc.trace.percent", "0.01", PropertyType.FRACTION, "Percent of gc cycles to trace"),
 
   // properties that are specific to the monitor server behavior

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/core/src/main/java/org/apache/accumulo/core/metadata/RootTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/RootTable.java b/core/src/main/java/org/apache/accumulo/core/metadata/RootTable.java
index 292ba3b..97d73d1 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/RootTable.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/RootTable.java
@@ -41,6 +41,7 @@ public class RootTable {
   public static final String ZROOT_TABLET_FUTURE_LOCATION = ZROOT_TABLET + "/future_location";
   public static final String ZROOT_TABLET_LAST_LOCATION = ZROOT_TABLET + "/lastlocation";
   public static final String ZROOT_TABLET_WALOGS = ZROOT_TABLET + "/walogs";
+  public static final String ZROOT_TABLET_CURRENT_LOGS = ZROOT_TABLET + "/current_logs";
   public static final String ZROOT_TABLET_PATH = ZROOT_TABLET + "/dir";
 
   public static final KeyExtent EXTENT = new KeyExtent(new Text(ID), null, null);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/core/src/main/java/org/apache/accumulo/core/metadata/schema/MetadataSchema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/schema/MetadataSchema.java b/core/src/main/java/org/apache/accumulo/core/metadata/schema/MetadataSchema.java
index 6baae17..c787d6d 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/schema/MetadataSchema.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/schema/MetadataSchema.java
@@ -16,11 +16,14 @@
  */
 package org.apache.accumulo.core.metadata.schema;
 
+import static java.nio.charset.StandardCharsets.UTF_8;
+
 import org.apache.accumulo.core.client.admin.TimeType;
 import org.apache.accumulo.core.data.ArrayByteSequence;
 import org.apache.accumulo.core.data.Key;
 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.schema.Section;
 import org.apache.accumulo.core.util.ColumnFQ;
 import org.apache.hadoop.io.Text;
@@ -278,4 +281,49 @@ public class MetadataSchema {
       buff.set(buff.getBytes(), section.getRowPrefix().length(), buff.getLength() - section.getRowPrefix().length());
     }
   }
+
+  /**
+   * Holds references to the WALs in use in a live Tablet Server.
+   * <p>
+   * <code>~wal+tserver:port[sessionId] log:hdfs://localhost:8020/accumulo/wal/tserver+port/WAL  [] -></code>
+   */
+  public static class CurrentLogsSection {
+    private static final Section section = new Section(RESERVED_PREFIX + "wal+", true, RESERVED_PREFIX + "wal,", false);
+    private static byte LEFT_BRACKET = (byte)'[';
+    public static final Text COLF = new Text("log");
+    public static final Value UNUSED = new Value("unused".getBytes(UTF_8));
+
+    public static Range getRange() {
+      return section.getRange();
+    }
+
+    public static String getRowPrefix() {
+      return section.getRowPrefix();
+    }
+
+    public static void getTabletServer(Key k, Text hostPort, Text session) {
+      Preconditions.checkNotNull(k);
+      Preconditions.checkNotNull(hostPort);
+      Preconditions.checkNotNull(session);
+
+      Text row = new Text();
+      k.getRow(row);
+      if (!row.toString().startsWith(section.getRowPrefix())) {
+        throw new IllegalArgumentException("Bad key " + k.toString());
+      }
+      for (int sessionStart = section.getRowPrefix().length(); sessionStart < row.getLength() - 1; sessionStart++) {
+        if (row.charAt(sessionStart) == LEFT_BRACKET) {
+          hostPort.set(row.getBytes(), section.getRowPrefix().length(), sessionStart - section.getRowPrefix().length());
+          session.set(row.getBytes(), sessionStart + 1, row.getLength() - sessionStart - 2);
+          return;
+        }
+      }
+      throw new IllegalArgumentException("Bad key " + k.toString());
+    }
+
+    public static void getPath(Key k, Text path) {
+      k.getColumnQualifier(path);
+    }
+
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/core/src/main/java/org/apache/accumulo/core/tabletserver/log/LogEntry.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/tabletserver/log/LogEntry.java b/core/src/main/java/org/apache/accumulo/core/tabletserver/log/LogEntry.java
index 7fe61d1..ab70bb0 100644
--- a/core/src/main/java/org/apache/accumulo/core/tabletserver/log/LogEntry.java
+++ b/core/src/main/java/org/apache/accumulo/core/tabletserver/log/LogEntry.java
@@ -16,10 +16,10 @@
  */
 package org.apache.accumulo.core.tabletserver.log;
 
+import static java.nio.charset.StandardCharsets.UTF_8;
+
 import java.io.IOException;
-import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collection;
 
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Value;
@@ -29,30 +29,29 @@ import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.io.Text;
 
-import com.google.common.base.Joiner;
-
 public class LogEntry {
-  public KeyExtent extent;
-  public long timestamp;
-  public String server;
-  public String filename;
-  public int tabletId;
-  public Collection<String> logSet;
-
-  public LogEntry() {}
+  public final KeyExtent extent;
+  public final long timestamp;
+  public final String server;
+  public final String filename;
 
   public LogEntry(LogEntry le) {
     this.extent = le.extent;
     this.timestamp = le.timestamp;
     this.server = le.server;
     this.filename = le.filename;
-    this.tabletId = le.tabletId;
-    this.logSet = new ArrayList<String>(le.logSet);
+  }
+
+  public LogEntry(KeyExtent extent, long timestamp, String server, String filename) {
+    this.extent = extent;
+    this.timestamp = timestamp;
+    this.server = server;
+    this.filename = filename;
   }
 
   @Override
   public String toString() {
-    return extent.toString() + " " + filename + " (" + tabletId + ")";
+    return extent.toString() + " " + filename;
   }
 
   public String getName() {
@@ -65,43 +64,35 @@ public class LogEntry {
     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 {
+  static public LogEntry fromBytes(byte bytes[]) throws IOException {
     DataInputBuffer inp = new DataInputBuffer();
     inp.reset(bytes, bytes.length);
-    extent = new KeyExtent();
+    KeyExtent 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;
+    long timestamp = inp.readLong();
+    String server = inp.readUTF();
+    String filename = inp.readUTF();
+    return new LogEntry(extent, timestamp, server, filename);
   }
 
   static private final Text EMPTY_TEXT = new Text();
 
   public static LogEntry fromKeyValue(Key key, Value value) {
-    LogEntry result = new LogEntry();
-    result.extent = new KeyExtent(key.getRow(), EMPTY_TEXT);
+    String qualifier = key.getColumnQualifier().toString();
+    if (qualifier.indexOf('/') < 1) {
+      throw new IllegalArgumentException("Bad key for log entry: " + key);
+    }
+    KeyExtent extent = new KeyExtent(key.getRow(), EMPTY_TEXT);
     String[] parts = key.getColumnQualifier().toString().split("/", 2);
-    result.server = parts[0];
-    result.filename = parts[1];
-    parts = value.toString().split("\\|");
-    result.tabletId = Integer.parseInt(parts[1]);
-    result.logSet = Arrays.asList(parts[0].split(";"));
-    result.timestamp = key.getTimestamp();
-    return result;
+    String server = parts[0];
+    // handle old-style log entries that specify log sets
+    parts = value.toString().split("\\|")[0].split(";");
+    String filename = parts[parts.length - 1];
+    long timestamp = key.getTimestamp();
+    return new LogEntry(extent, timestamp, server, filename);
   }
 
   public Text getRow() {
@@ -112,11 +103,16 @@ public class LogEntry {
     return MetadataSchema.TabletsSection.LogColumnFamily.NAME;
   }
 
+  public String getUniqueID() {
+    String parts[] = filename.split("/");
+    return parts[parts.length - 1];
+  }
+
   public Text getColumnQualifier() {
     return new Text(server + "/" + filename);
   }
 
   public Value getValue() {
-    return new Value((Joiner.on(";").join(logSet) + "|" + tabletId).getBytes());
+    return new Value(filename.getBytes(UTF_8));
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/core/src/test/java/org/apache/accumulo/core/metadata/MetadataTableSchemaTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/metadata/MetadataTableSchemaTest.java b/core/src/test/java/org/apache/accumulo/core/metadata/MetadataTableSchemaTest.java
new file mode 100644
index 0000000..cfe59f2
--- /dev/null
+++ b/core/src/test/java/org/apache/accumulo/core/metadata/MetadataTableSchemaTest.java
@@ -0,0 +1,47 @@
+/*
+ * 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.metadata;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.CurrentLogsSection;
+import org.apache.hadoop.io.Text;
+import org.junit.Test;
+
+public class MetadataTableSchemaTest {
+
+  @Test
+  public void testGetTabletServer() throws Exception {
+    Key key = new Key("~wal+host:43861[14a7df0e6420003]", "log", "hdfs://localhost:50514/accumulo/wal/host:43861/70c27ab3-6662-40ab-80fb-01c1f1a59df3");
+    Text hostPort = new Text();
+    Text session = new Text();
+    CurrentLogsSection.getTabletServer(key, hostPort, session);
+    assertEquals("host:43861", hostPort.toString());
+    assertEquals("14a7df0e6420003", session.toString());
+    try {
+      Key bogus = new Key("~wal/host:43861[14a7df0e6420003]", "log", "hdfs://localhost:50514/accumulo/wal/host:43861/70c27ab3-6662-40ab-80fb-01c1f1a59df3");
+      CurrentLogsSection.getTabletServer(bogus, hostPort, session);
+      fail("bad argument not thrown");
+    } catch (IllegalArgumentException ex) {
+
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/server/base/src/main/java/org/apache/accumulo/server/TabletLevel.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/TabletLevel.java b/server/base/src/main/java/org/apache/accumulo/server/TabletLevel.java
new file mode 100644
index 0000000..91e5ee9
--- /dev/null
+++ b/server/base/src/main/java/org/apache/accumulo/server/TabletLevel.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.server;
+
+import org.apache.accumulo.core.data.KeyExtent;
+
+public enum TabletLevel {
+  ROOT,
+  META,
+  NORMAL;
+
+  public static TabletLevel getLevel(KeyExtent extent) {
+    if (!extent.isMeta())
+      return NORMAL;
+    if (extent.isRootTablet())
+      return ROOT;
+    return META;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeUtil.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeUtil.java b/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeUtil.java
index c3595cd..4722e60 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeUtil.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeUtil.java
@@ -128,15 +128,12 @@ public class VolumeUtil {
       switchedPath = le.filename;
 
     ArrayList<String> switchedLogs = new ArrayList<String>();
-    for (String log : le.logSet) {
-      String switchedLog = switchVolume(le.filename, FileType.WAL, replacements);
-      if (switchedLog != null) {
-        switchedLogs.add(switchedLog);
-        numSwitched++;
-      } else {
-        switchedLogs.add(log);
-      }
-
+    String switchedLog = switchVolume(le.filename, FileType.WAL, replacements);
+    if (switchedLog != null) {
+      switchedLogs.add(switchedLog);
+      numSwitched++;
+    } else {
+      switchedLogs.add(le.filename);
     }
 
     if (numSwitched == 0) {
@@ -144,9 +141,7 @@ public class VolumeUtil {
       return null;
     }
 
-    LogEntry newLogEntry = new LogEntry(le);
-    newLogEntry.filename = switchedPath;
-    newLogEntry.logSet = switchedLogs;
+    LogEntry newLogEntry = new LogEntry(le.extent, le.timestamp, le.server, switchedPath);
 
     log.trace("Switched " + le + " to " + newLogEntry);
 
@@ -244,7 +239,7 @@ public class VolumeUtil {
         log.debug("Tablet directory switched, need to record old log files " + logsToRemove + " " + ProtobufUtil.toString(status));
         // Before deleting these logs, we need to mark them for replication
         for (LogEntry logEntry : logsToRemove) {
-          ReplicationTableUtil.updateFiles(context, extent, logEntry.logSet, status);
+          ReplicationTableUtil.updateFiles(context, extent, logEntry.filename, status);
         }
       }
     }
@@ -253,7 +248,6 @@ public class VolumeUtil {
 
     // method this should return the exact strings that are in the metadata table
     return ret;
-
   }
 
   private static String decommisionedTabletDir(AccumuloServerContext context, ZooLock zooLock, VolumeManager vm, KeyExtent extent, String metaDir)

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java b/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java
index c6f1dd8..9afb93f 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java
@@ -533,6 +533,7 @@ public class Initialize implements KeywordExecutable {
     zoo.putPersistentData(zkInstanceRoot + Constants.ZPROBLEMS, EMPTY_BYTE_ARRAY, NodeExistsPolicy.FAIL);
     zoo.putPersistentData(zkInstanceRoot + RootTable.ZROOT_TABLET, EMPTY_BYTE_ARRAY, NodeExistsPolicy.FAIL);
     zoo.putPersistentData(zkInstanceRoot + RootTable.ZROOT_TABLET_WALOGS, EMPTY_BYTE_ARRAY, NodeExistsPolicy.FAIL);
+    zoo.putPersistentData(zkInstanceRoot + RootTable.ZROOT_TABLET_CURRENT_LOGS, EMPTY_BYTE_ARRAY, NodeExistsPolicy.FAIL);
     zoo.putPersistentData(zkInstanceRoot + RootTable.ZROOT_TABLET_PATH, rootTabletDir.getBytes(UTF_8), NodeExistsPolicy.FAIL);
     zoo.putPersistentData(zkInstanceRoot + Constants.ZMASTERS, EMPTY_BYTE_ARRAY, NodeExistsPolicy.FAIL);
     zoo.putPersistentData(zkInstanceRoot + Constants.ZMASTER_LOCK, EMPTY_BYTE_ARRAY, NodeExistsPolicy.FAIL);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/server/base/src/main/java/org/apache/accumulo/server/master/state/MetaDataStateStore.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/state/MetaDataStateStore.java b/server/base/src/main/java/org/apache/accumulo/server/master/state/MetaDataStateStore.java
index 7ee6f0c..c154bd0 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/state/MetaDataStateStore.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/state/MetaDataStateStore.java
@@ -17,6 +17,9 @@
 package org.apache.accumulo.server.master.state;
 
 import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.accumulo.core.client.BatchWriter;
@@ -27,9 +30,14 @@ import org.apache.accumulo.core.client.impl.ClientContext;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema;
+import org.apache.accumulo.core.tabletserver.log.LogEntry;
 import org.apache.accumulo.server.AccumuloServerContext;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+import org.apache.log4j.Logger;
 
 public class MetaDataStateStore extends TabletStateStore {
+  private static final Logger log = Logger.getLogger(MetaDataStateStore.class);
 
   private static final int THREADS = 4;
   private static final int LATENCY = 1000;
@@ -59,7 +67,7 @@ public class MetaDataStateStore extends TabletStateStore {
 
   @Override
   public ClosableIterator<TabletLocationState> iterator() {
-    return new MetaDataTableScanner(context, MetadataSchema.TabletsSection.getRange(), state);
+    return new MetaDataTableScanner(context, MetadataSchema.TabletsSection.getRange(), state, targetTableName);
   }
 
   @Override
@@ -116,7 +124,7 @@ public class MetaDataStateStore extends TabletStateStore {
   }
 
   @Override
-  public void unassign(Collection<TabletLocationState> tablets) throws DistributedStoreException {
+  public void unassign(Collection<TabletLocationState> tablets, Map<TServerInstance, List<Path>> logsForDeadServers) throws DistributedStoreException {
 
     BatchWriter writer = createBatchWriter();
     try {
@@ -124,6 +132,15 @@ public class MetaDataStateStore extends TabletStateStore {
         Mutation m = new Mutation(tls.extent.getMetadataEntry());
         if (tls.current != null) {
           tls.current.clearLocation(m);
+          if (logsForDeadServers != null) {
+            List<Path> logs = logsForDeadServers.get(tls.current);
+            if (logs != null) {
+              for (Path log : logs) {
+                LogEntry entry = new LogEntry(tls.extent, 0, tls.current.hostPort(), log.toString());
+                m.put(entry.getColumnFamily(), entry.getColumnQualifier(), entry.getValue());
+              }
+            }
+          }
         }
         if (tls.future != null) {
           tls.future.clearFutureLocation(m);
@@ -145,4 +162,30 @@ public class MetaDataStateStore extends TabletStateStore {
   public String name() {
     return "Normal Tablets";
   }
+
+  @Override
+  public void markLogsAsUnused(AccumuloServerContext context, Map<TServerInstance,List<Path>> logs) throws DistributedStoreException {
+    BatchWriter writer = createBatchWriter();
+    try {
+      for (Entry<TServerInstance,List<Path>> entry : logs.entrySet()) {
+        if (entry.getValue().isEmpty()) {
+          continue;
+        }
+        Mutation m = new Mutation(MetadataSchema.CurrentLogsSection.getRowPrefix() + entry.getKey().toString());
+        for (Path log : entry.getValue()) {
+          m.put(MetadataSchema.CurrentLogsSection.COLF, new Text(log.toString()), MetadataSchema.CurrentLogsSection.UNUSED);
+        }
+        writer.addMutation(m);
+      }
+    } catch (Exception ex) {
+      log.error("Error marking logs as unused: " + logs);
+      throw new DistributedStoreException(ex);
+    } finally {
+      try {
+        writer.close();
+      } catch (MutationsRejectedException e) {
+        throw new DistributedStoreException(e);
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/server/base/src/main/java/org/apache/accumulo/server/master/state/MetaDataTableScanner.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/state/MetaDataTableScanner.java b/server/base/src/main/java/org/apache/accumulo/server/master/state/MetaDataTableScanner.java
index d64c108..bec2dc4 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/state/MetaDataTableScanner.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/state/MetaDataTableScanner.java
@@ -141,6 +141,7 @@ public class MetaDataTableScanner implements ClosableIterator<TabletLocationStat
     boolean chopped = false;
 
     for (Entry<Key,Value> entry : decodedRow.entrySet()) {
+
       Key key = entry.getKey();
       Text row = key.getRow();
       Text cf = key.getColumnFamily();
@@ -173,8 +174,9 @@ public class MetaDataTableScanner implements ClosableIterator<TabletLocationStat
       }
     }
     if (extent == null) {
-      log.warn("No prev-row for key extent: " + decodedRow);
-      return null;
+      String msg = "No prev-row for key extent " + decodedRow;
+      log.error(msg);
+      throw new BadLocationStateException(msg, k.getRow());
     }
     return new TabletLocationState(extent, future, current, last, walogs, chopped);
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/server/base/src/main/java/org/apache/accumulo/server/master/state/TabletLocationState.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/state/TabletLocationState.java b/server/base/src/main/java/org/apache/accumulo/server/master/state/TabletLocationState.java
index fb30440..8116ecf 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/state/TabletLocationState.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/state/TabletLocationState.java
@@ -68,6 +68,13 @@ public class TabletLocationState {
   final public Collection<Collection<String>> walogs;
   final public boolean chopped;
 
+  public TServerInstance futureOrCurrent() {
+    if (current != null) {
+      return current;
+    }
+    return future;
+  }
+
   @Override
   public String toString() {
     return extent + "@(" + future + "," + current + "," + last + ")" + (chopped ? " chopped" : "");

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/server/base/src/main/java/org/apache/accumulo/server/master/state/TabletStateStore.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/state/TabletStateStore.java b/server/base/src/main/java/org/apache/accumulo/server/master/state/TabletStateStore.java
index 5413e31..acc10d8 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/state/TabletStateStore.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/state/TabletStateStore.java
@@ -18,8 +18,11 @@ package org.apache.accumulo.server.master.state;
 
 import java.util.Collection;
 import java.util.Collections;
+import java.util.List;
+import java.util.Map;
 
 import org.apache.accumulo.server.AccumuloServerContext;
+import org.apache.hadoop.fs.Path;
 
 /**
  * Interface for storing information about tablet assignments. There are three implementations:
@@ -56,10 +59,12 @@ public abstract class TabletStateStore implements Iterable<TabletLocationState>
    *
    * @param tablets
    *          the tablets' current information
+   * @param logsForDeadServers
+   *          a cache of logs in use by servers when they died
    */
-  abstract public void unassign(Collection<TabletLocationState> tablets) throws DistributedStoreException;
+  abstract public void unassign(Collection<TabletLocationState> tablets, Map<TServerInstance, List<Path>> logsForDeadServers) throws DistributedStoreException;
 
-  public static void unassign(AccumuloServerContext context, TabletLocationState tls) throws DistributedStoreException {
+  public static void unassign(AccumuloServerContext context, TabletLocationState tls, Map<TServerInstance, List<Path>> logsForDeadServers) throws DistributedStoreException {
     TabletStateStore store;
     if (tls.extent.isRootTablet()) {
       store = new ZooTabletStateStore();
@@ -68,7 +73,7 @@ public abstract class TabletStateStore implements Iterable<TabletLocationState>
     } else {
       store = new MetaDataStateStore(context);
     }
-    store.unassign(Collections.singletonList(tls));
+    store.unassign(Collections.singletonList(tls), logsForDeadServers);
   }
 
   public static void setLocation(AccumuloServerContext context, Assignment assignment) throws DistributedStoreException {
@@ -83,4 +88,9 @@ public abstract class TabletStateStore implements Iterable<TabletLocationState>
     store.setLocations(Collections.singletonList(assignment));
   }
 
+  /**
+   * When a server fails, its logs must be marked as unused after the log markers are moved to the tablets.
+   */
+  abstract public void markLogsAsUnused(AccumuloServerContext context, Map<TServerInstance, List<Path>> logs) throws DistributedStoreException;
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/server/base/src/main/java/org/apache/accumulo/server/master/state/ZooTabletStateStore.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/state/ZooTabletStateStore.java b/server/base/src/main/java/org/apache/accumulo/server/master/state/ZooTabletStateStore.java
index ab99396..bce20fd 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/state/ZooTabletStateStore.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/state/ZooTabletStateStore.java
@@ -21,12 +21,17 @@ import static java.nio.charset.StandardCharsets.UTF_8;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.accumulo.core.metadata.RootTable;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema;
 import org.apache.accumulo.core.tabletserver.log.LogEntry;
+import org.apache.accumulo.server.AccumuloServerContext;
 import org.apache.commons.lang.NotImplementedException;
+import org.apache.hadoop.fs.Path;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -85,10 +90,9 @@ public class ZooTabletStateStore extends TabletStateStore {
           for (String entry : store.getChildren(RootTable.ZROOT_TABLET_WALOGS)) {
             byte[] logInfo = store.get(RootTable.ZROOT_TABLET_WALOGS + "/" + entry);
             if (logInfo != null) {
-              LogEntry logEntry = new LogEntry();
-              logEntry.fromBytes(logInfo);
-              logs.add(logEntry.logSet);
-              log.debug("root tablet logSet " + logEntry.logSet);
+              LogEntry logEntry = LogEntry.fromBytes(logInfo);
+              logs.add(Collections.singleton(logEntry.filename));
+              log.debug("root tablet log " + logEntry.filename);
             }
           }
           TabletLocationState result = new TabletLocationState(RootTable.EXTENT, futureSession, currentSession, lastSession, logs, false);
@@ -161,12 +165,28 @@ public class ZooTabletStateStore extends TabletStateStore {
   }
 
   @Override
-  public void unassign(Collection<TabletLocationState> tablets) throws DistributedStoreException {
+  public void unassign(Collection<TabletLocationState> tablets, Map<TServerInstance, List<Path>> logsForDeadServers) throws DistributedStoreException {
     if (tablets.size() != 1)
       throw new IllegalArgumentException("There is only one root tablet");
     TabletLocationState tls = tablets.iterator().next();
     if (tls.extent.compareTo(RootTable.EXTENT) != 0)
       throw new IllegalArgumentException("You can only store the root tablet location");
+    if (logsForDeadServers != null) {
+      List<Path> logs = logsForDeadServers.get(tls.futureOrCurrent());
+      if (logs != null) {
+        for (Path entry : logs) {
+          LogEntry logEntry = new LogEntry(RootTable.EXTENT, System.currentTimeMillis(), tls.futureOrCurrent().getLocation().toString(), entry.toString());
+          byte[] value;
+          try {
+            value = logEntry.toBytes();
+          } catch (IOException ex) {
+            throw new DistributedStoreException(ex);
+          }
+          store.put(RootTable.ZROOT_TABLET_WALOGS + "/" + logEntry.getUniqueID(), value);
+          store.remove(RootTable.ZROOT_TABLET_CURRENT_LOGS + "/" + MetadataSchema.CurrentLogsSection.getRowPrefix() + tls.current.toString() + logEntry.getUniqueID());
+        }
+      }
+    }
     store.remove(RootTable.ZROOT_TABLET_LOCATION);
     store.remove(RootTable.ZROOT_TABLET_FUTURE_LOCATION);
     log.debug("unassign root tablet location");
@@ -177,4 +197,9 @@ public class ZooTabletStateStore extends TabletStateStore {
     return "Root Table";
   }
 
+  @Override
+  public void markLogsAsUnused(AccumuloServerContext context, Map<TServerInstance,List<Path>> logs) {
+    // the root table is not replicated, so unassigning the root tablet has removed the current log marker
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/server/base/src/main/java/org/apache/accumulo/server/replication/StatusUtil.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/replication/StatusUtil.java b/server/base/src/main/java/org/apache/accumulo/server/replication/StatusUtil.java
index 898e3d4..d72eea2 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/replication/StatusUtil.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/replication/StatusUtil.java
@@ -153,6 +153,19 @@ public class StatusUtil {
   /**
    * @return A {@link Status} for an open file of unspecified length, all of which needs replicating.
    */
+  public static Status openWithUnknownLength(long timeCreated) {
+    Builder builder = Status.newBuilder();
+    builder.setBegin(0);
+    builder.setEnd(0);
+    builder.setInfiniteEnd(true);
+    builder.setClosed(false);
+    builder.setCreatedTime(timeCreated);
+    return builder.build();
+  }
+
+  /**
+   * @return A {@link Status} for an open file of unspecified length, all of which needs replicating.
+   */
   public static Status openWithUnknownLength() {
     return INF_END_REPLICATION_STATUS;
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/server/base/src/main/java/org/apache/accumulo/server/util/ListVolumesUsed.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/ListVolumesUsed.java b/server/base/src/main/java/org/apache/accumulo/server/util/ListVolumesUsed.java
index e90d1dd..9e3fc7d 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/ListVolumesUsed.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/ListVolumesUsed.java
@@ -35,6 +35,7 @@ import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.accumulo.server.conf.ServerConfigurationFactory;
 import org.apache.accumulo.server.fs.VolumeManager.FileType;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
 
 /**
  *
@@ -61,9 +62,6 @@ public class ListVolumesUsed {
 
   private static void getLogURIs(TreeSet<String> volumes, LogEntry logEntry) {
     volumes.add(getLogURI(logEntry.filename));
-    for (String logSet : logEntry.logSet) {
-      volumes.add(getLogURI(logSet));
-    }
   }
 
   private static void listZookeeper() throws Exception {
@@ -123,6 +121,20 @@ public class ListVolumesUsed {
 
     for (String volume : volumes)
       System.out.println("\tVolume : " + volume);
+
+    volumes.clear();
+    scanner.clearColumns();
+    scanner.setRange(MetadataSchema.CurrentLogsSection.getRange());
+    Text path = new Text();
+    for (Entry<Key,Value> entry : scanner) {
+      MetadataSchema.CurrentLogsSection.getPath(entry.getKey(), path);
+      volumes.add(getLogURI(path.toString()));
+    }
+
+    System.out.println("Listing volumes referenced in " + name + " current logs section");
+
+    for (String volume : volumes)
+      System.out.println("\tVolume : " + volume);
   }
 
   public static void listVolumes(ClientContext context) throws Exception {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/server/base/src/main/java/org/apache/accumulo/server/util/MasterMetadataUtil.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/MasterMetadataUtil.java b/server/base/src/main/java/org/apache/accumulo/server/util/MasterMetadataUtil.java
index 14eba68..4a5650e 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/MasterMetadataUtil.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/MasterMetadataUtil.java
@@ -248,35 +248,27 @@ public class MasterMetadataUtil {
       if (unusedWalLogs != null) {
         updateRootTabletDataFile(extent, path, mergeFile, dfv, time, filesInUseByScans, address, zooLock, unusedWalLogs, lastLocation, flushId);
       }
-
       return;
     }
-
     Mutation m = getUpdateForTabletDataFile(extent, path, mergeFile, dfv, time, filesInUseByScans, address, zooLock, unusedWalLogs, lastLocation, flushId);
-
     MetadataTableUtil.update(context, zooLock, m, extent);
-
   }
 
   /**
    * Update the data file for the root tablet
    */
-  protected static void updateRootTabletDataFile(KeyExtent extent, FileRef path, FileRef mergeFile, DataFileValue dfv, String time,
+  private static void updateRootTabletDataFile(KeyExtent extent, FileRef path, FileRef mergeFile, DataFileValue dfv, String time,
       Set<FileRef> filesInUseByScans, String address, ZooLock zooLock, Set<String> unusedWalLogs, TServerInstance lastLocation, long flushId) {
     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 = MetadataTableUtil.getZookeeperLogLocation();
-    boolean foundEntry = false;
     for (String entry : unusedWalLogs) {
       String[] parts = entry.split("/");
       String zpath = root + "/" + parts[parts.length - 1];
       while (true) {
         try {
           if (zk.exists(zpath)) {
+            log.debug("Removing WAL reference for root table " + zpath);
             zk.recursiveDelete(zpath, NodeMissingPolicy.SKIP);
-            foundEntry = true;
           }
           break;
         } catch (KeeperException e) {
@@ -287,16 +279,15 @@ public class MasterMetadataUtil {
         UtilWaitThread.sleep(1000);
       }
     }
-    if (unusedWalLogs.size() > 0 && !foundEntry)
-      log.warn("WALog entry for root tablet did not exist " + unusedWalLogs);
   }
 
+
   /**
    * Create an update that updates a tablet
    *
    * @return A Mutation to update a tablet from the given information
    */
-  protected static Mutation getUpdateForTabletDataFile(KeyExtent extent, FileRef path, FileRef mergeFile, DataFileValue dfv, String time,
+  private static Mutation getUpdateForTabletDataFile(KeyExtent extent, FileRef path, FileRef mergeFile, DataFileValue dfv, String time,
       Set<FileRef> filesInUseByScans, String address, ZooLock zooLock, Set<String> unusedWalLogs, TServerInstance lastLocation, long flushId) {
     Mutation m = new Mutation(extent.getMetadataEntry());
 
@@ -324,6 +315,7 @@ public class MasterMetadataUtil {
 
     TabletsSection.ServerColumnFamily.FLUSH_COLUMN.put(m, new Value(Long.toString(flushId).getBytes(UTF_8)));
 
+
     return m;
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java
----------------------------------------------------------------------
diff --git 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
index 5e74aac..4470c55 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
@@ -23,8 +23,6 @@ import static org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSec
 
 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;
@@ -61,6 +59,7 @@ 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.CurrentLogsSection;
 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;
@@ -82,10 +81,12 @@ import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy;
 import org.apache.accumulo.server.AccumuloServerContext;
 import org.apache.accumulo.server.ServerConstants;
+import org.apache.accumulo.server.TabletLevel;
 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.VolumeManagerImpl;
+import org.apache.accumulo.server.master.state.TServerInstance;
 import org.apache.accumulo.server.tablets.TabletTime;
 import org.apache.accumulo.server.zookeeper.ZooLock;
 import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
@@ -121,7 +122,7 @@ public class MetadataTableUtil {
     return metadataTable;
   }
 
-  private synchronized static Writer getRootTable(ClientContext context) {
+  public synchronized static Writer getRootTable(ClientContext context) {
     Credentials credentials = context.getCredentials();
     Writer rootTable = root_tables.get(credentials);
     if (rootTable == null) {
@@ -223,7 +224,7 @@ public class MetadataTableUtil {
 
       // add before removing in case of process death
       for (LogEntry logEntry : logsToAdd)
-        addLogEntry(context, logEntry, zooLock);
+        addRootLogEntry(context, zooLock, logEntry);
 
       removeUnusedWALEntries(context, extent, logsToRemove, zooLock);
     } else {
@@ -248,6 +249,35 @@ public class MetadataTableUtil {
     }
   }
 
+  private static interface ZooOperation {
+    void run(IZooReaderWriter rw) throws KeeperException, InterruptedException, IOException;
+  }
+
+  private static void retryZooKeeperUpdate(ClientContext context, ZooLock zooLock, ZooOperation op) {
+    while (true) {
+      try {
+        IZooReaderWriter zoo = ZooReaderWriter.getInstance();
+        if (zoo.isLockHeld(zooLock.getLockID())) {
+          op.run(zoo);
+        }
+        break;
+      } catch (Exception e) {
+        log.error("Unexpected exception {}", e.getMessage(), e);
+      }
+      UtilWaitThread.sleep(1000);
+    }
+  }
+
+  private static void addRootLogEntry(AccumuloServerContext context, ZooLock zooLock, final LogEntry entry) {
+    retryZooKeeperUpdate(context, zooLock, new ZooOperation() {
+      @Override
+      public void run(IZooReaderWriter rw) throws KeeperException, InterruptedException, IOException {
+        String root = getZookeeperLogLocation();
+        rw.putPersistentData(root + "/" + entry.getUniqueID(), entry.toBytes(), NodeExistsPolicy.OVERWRITE);
+      }
+    });
+  }
+
   public static SortedMap<FileRef,DataFileValue> getDataFileSizes(KeyExtent extent, ClientContext context) throws IOException {
     TreeMap<FileRef,DataFileValue> sizes = new TreeMap<FileRef,DataFileValue>();
 
@@ -447,34 +477,6 @@ public class MetadataTableUtil {
     return ZooUtil.getRoot(HdfsZooInstance.getInstance()) + RootTable.ZROOT_TABLET_WALOGS;
   }
 
-  public static void addLogEntry(ClientContext context, 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.getMessage(), e);
-        } catch (InterruptedException e) {
-          log.error("{}", e.getMessage(), e);
-        } catch (IOException e) {
-          log.error("{}", e.getMessage(), e);
-        }
-        UtilWaitThread.sleep(1000);
-      }
-    } else {
-      Mutation m = new Mutation(entry.getRow());
-      m.put(entry.getColumnFamily(), entry.getColumnQualifier(), entry.getValue());
-      update(context, zooLock, m, entry.extent);
-    }
-  }
-
   public static void setRootTabletDir(String dir) throws IOException {
     IZooReaderWriter zoo = ZooReaderWriter.getInstance();
     String zpath = ZooUtil.getRoot(HdfsZooInstance.getInstance()) + RootTable.ZROOT_TABLET_PATH;
@@ -565,22 +567,11 @@ public class MetadataTableUtil {
       }
     }
 
-    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;
   }
 
-  static void getRootLogEntries(ArrayList<LogEntry> result) throws KeeperException, InterruptedException, IOException {
+  static void getRootLogEntries(final 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
@@ -588,11 +579,10 @@ public class MetadataTableUtil {
     while (true) {
       result.clear();
       for (String child : zoo.getChildren(root)) {
-        LogEntry e = new LogEntry();
         try {
-          e.fromBytes(zoo.getData(root + "/" + child, null));
+          LogEntry e = LogEntry.fromBytes(zoo.getData(root + "/" + child, null));
           // upgrade from !0;!0<< -> +r<<
-          e.extent = RootTable.EXTENT;
+          e = new LogEntry(RootTable.EXTENT, 0, e.server, e.filename);
           result.add(e);
         } catch (KeeperException.NoNodeException ex) {
           continue;
@@ -662,28 +652,23 @@ public class MetadataTableUtil {
     return new LogEntryIterator(context);
   }
 
-  public static void removeUnusedWALEntries(AccumuloServerContext context, KeyExtent extent, List<LogEntry> logEntries, ZooLock zooLock) {
+  public static void removeUnusedWALEntries(AccumuloServerContext context, KeyExtent extent, final List<LogEntry> entries, ZooLock zooLock) {
     if (extent.isRootTablet()) {
-      for (LogEntry entry : logEntries) {
-        String root = getZookeeperLogLocation();
-        while (true) {
-          try {
-            IZooReaderWriter zoo = ZooReaderWriter.getInstance();
-            if (zoo.isLockHeld(zooLock.getLockID())) {
-              String parts[] = entry.filename.split("/");
-              zoo.recursiveDelete(root + "/" + parts[parts.length - 1], NodeMissingPolicy.SKIP);
-            }
-            break;
-          } catch (Exception e) {
-            log.error("{}", e.getMessage(), e);
+      retryZooKeeperUpdate(context, zooLock, new ZooOperation() {
+        @Override
+        public void run(IZooReaderWriter rw) throws KeeperException, InterruptedException, IOException {
+          String root = getZookeeperLogLocation();
+          for (LogEntry entry : entries) {
+            String path = root + "/" + entry.getUniqueID();
+            log.debug("Removing " + path + " from zookeeper");
+            rw.recursiveDelete(path, NodeMissingPolicy.SKIP);
           }
-          UtilWaitThread.sleep(1000);
         }
-      }
+      });
     } else {
       Mutation m = new Mutation(extent.getMetadataEntry());
-      for (LogEntry entry : logEntries) {
-        m.putDelete(LogColumnFamily.NAME, new Text(entry.getName()));
+      for (LogEntry entry : entries) {
+        m.putDelete(entry.getColumnFamily(), entry.getColumnQualifier());
       }
       update(context, zooLock, m, extent);
     }
@@ -1068,4 +1053,130 @@ public class MetadataTableUtil {
     return tabletEntries;
   }
 
+  public static void addNewLogMarker(ClientContext context, ZooLock zooLock, final TServerInstance tabletSession, final Path filename, TabletLevel level) {
+    log.debug("Adding log entry " + filename);
+    if (level == TabletLevel.ROOT) {
+      retryZooKeeperUpdate(context, zooLock, new ZooOperation() {
+        @Override
+        public void run(IZooReaderWriter rw) throws KeeperException, InterruptedException, IOException {
+          String root = ZooUtil.getRoot(HdfsZooInstance.getInstance()) + RootTable.ZROOT_TABLET_CURRENT_LOGS;
+          String uniqueId = filename.getName();
+          StringBuilder path = new StringBuilder(root);
+          path.append("/");
+          path.append(CurrentLogsSection.getRowPrefix());
+          path.append(tabletSession.toString());
+          path.append(uniqueId);
+          rw.putPersistentData(path.toString(), filename.toString().getBytes(UTF_8), NodeExistsPolicy.OVERWRITE);
+        }
+      });
+    } else {
+      Mutation m = new Mutation(CurrentLogsSection.getRowPrefix() + tabletSession.toString());
+      m.put(CurrentLogsSection.COLF, new Text(filename.toString()), new Value(EMPTY_BYTES));
+      String tableName = MetadataTable.NAME;
+      if (level == TabletLevel.META) {
+        tableName = RootTable.NAME;
+      }
+      BatchWriter bw = null;
+      try {
+        bw = context.getConnector().createBatchWriter(tableName, null);
+        bw.addMutation(m);
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      } finally {
+        if (bw != null) {
+          try {
+            bw.close();
+          } catch (Exception e2) {
+            throw new RuntimeException(e2);
+          }
+        }
+      }
+    }
+  }
+
+  private static void removeCurrentRootLogMarker(ClientContext context, ZooLock zooLock, final TServerInstance tabletSession, final Path filename) {
+    retryZooKeeperUpdate(context, zooLock, new ZooOperation() {
+      @Override
+      public void run(IZooReaderWriter rw) throws KeeperException, InterruptedException, IOException {
+        String root = ZooUtil.getRoot(HdfsZooInstance.getInstance()) + RootTable.ZROOT_TABLET_CURRENT_LOGS;
+        String uniqueId = filename.getName();
+        String path = root + "/" + CurrentLogsSection.getRowPrefix() + tabletSession.toString() + uniqueId;
+        log.debug("Removing entry " + path + " from zookeeper");
+        rw.recursiveDelete(path, NodeMissingPolicy.SKIP);
+      }
+    });
+  }
+
+  public static void markLogUnused(ClientContext context, ZooLock lock, TServerInstance tabletSession, Set<Path> all) throws AccumuloException {
+    // There could be a marker at the meta and/or root level, mark them both as unused
+    try {
+      BatchWriter root = null;
+      BatchWriter meta = null;
+      try {
+        root = context.getConnector().createBatchWriter(RootTable.NAME, null);
+        meta = context.getConnector().createBatchWriter(MetadataTable.NAME, null);
+        for (Path fname : all) {
+          Text tname = new Text(fname.toString());
+          Mutation m = new Mutation(MetadataSchema.CurrentLogsSection.getRowPrefix() + tabletSession.toString());
+          m.putDelete(MetadataSchema.CurrentLogsSection.COLF, tname);
+          root.addMutation(m);
+          log.debug("deleting " + MetadataSchema.CurrentLogsSection.getRowPrefix() + tabletSession.toString() + " log:" + fname);
+          m = new Mutation(MetadataSchema.CurrentLogsSection.getRowPrefix() + tabletSession.toString());
+          m.put(MetadataSchema.CurrentLogsSection.COLF, tname, MetadataSchema.CurrentLogsSection.UNUSED);
+          meta.addMutation(m);
+          removeCurrentRootLogMarker(context, lock, tabletSession, fname);
+        }
+      } finally {
+        if (root != null) {
+          root.close();
+        }
+        if (meta != null) {
+          meta.close();
+        }
+      }
+    } catch (Exception ex) {
+      throw new AccumuloException(ex);
+    }
+  }
+
+  public static void fetchLogsForDeadServer(ClientContext context, ZooLock lock, KeyExtent extent, TServerInstance server, Map<TServerInstance,List<Path>> logsForDeadServers)
+      throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
+    // already cached
+    if (logsForDeadServers.containsKey(server)) {
+      return;
+    }
+    if (extent.isRootTablet()) {
+      final List<Path> logs = new ArrayList<>();
+      retryZooKeeperUpdate(context, lock, new ZooOperation() {
+        @Override
+        public void run(IZooReaderWriter rw) throws KeeperException, InterruptedException, IOException {
+          String root = ZooUtil.getRoot(HdfsZooInstance.getInstance()) + RootTable.ZROOT_TABLET_CURRENT_LOGS;
+          logs.clear();
+          for (String child : rw.getChildren(root)) {
+            logs.add(new Path(new String(rw.getData(root + "/" + child, null), UTF_8)));
+          }
+        }
+      });
+      logsForDeadServers.put(server, logs);
+    } else {
+      // use the correct meta table
+      String table = MetadataTable.NAME;
+      if (extent.isMeta()) {
+        table = RootTable.NAME;
+      }
+      // fetch the current logs in use, and put them in the cache
+      Scanner scanner = context.getConnector().createScanner(table, Authorizations.EMPTY);
+      scanner.setRange(new Range(MetadataSchema.CurrentLogsSection.getRowPrefix() + server.toString()));
+      List<Path> logs = new ArrayList<>();
+      Text path = new Text();
+      for (Entry<Key,Value> entry : scanner) {
+        MetadataSchema.CurrentLogsSection.getPath(entry.getKey(), path);
+        if (!entry.getValue().equals(MetadataSchema.CurrentLogsSection.UNUSED)) {
+          logs.add(new Path(path.toString()));
+        }
+      }
+      logsForDeadServers.put(server, logs);
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/server/base/src/main/java/org/apache/accumulo/server/util/ReplicationTableUtil.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/ReplicationTableUtil.java b/server/base/src/main/java/org/apache/accumulo/server/util/ReplicationTableUtil.java
index 8e755a3..c6d5ce4 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/ReplicationTableUtil.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/ReplicationTableUtil.java
@@ -16,7 +16,6 @@
  */
 package org.apache.accumulo.server.util;
 
-import java.util.Collection;
 import java.util.Collections;
 import java.util.EnumSet;
 import java.util.HashMap;
@@ -176,20 +175,14 @@ public class ReplicationTableUtil {
   /**
    * Write replication ingest entries for each provided file with the given {@link Status}.
    */
-  public static void updateFiles(ClientContext context, KeyExtent extent, Collection<String> files, Status stat) {
+  public static void updateFiles(ClientContext context, KeyExtent extent, String file, Status stat) {
     if (log.isDebugEnabled()) {
-      log.debug("Updating replication status for " + extent + " with " + files + " using " + ProtobufUtil.toString(stat));
+      log.debug("Updating replication status for " + extent + " with " + file + " using " + ProtobufUtil.toString(stat));
     }
     // TODO could use batch writer, would need to handle failure and retry like update does - ACCUMULO-1294
-    if (files.isEmpty()) {
-      return;
-    }
 
     Value v = ProtobufUtil.toValue(stat);
-    for (String file : files) {
-      // TODO Can preclude this addition if the extent is for a table we don't need to replicate
-      update(context, createUpdateMutation(new Path(file), v, extent), extent);
-    }
+    update(context, createUpdateMutation(new Path(file), v, extent), extent);
   }
 
   static Mutation createUpdateMutation(Path file, Value v, KeyExtent extent) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/server/base/src/test/java/org/apache/accumulo/server/util/ReplicationTableUtilTest.java
----------------------------------------------------------------------
diff --git a/server/base/src/test/java/org/apache/accumulo/server/util/ReplicationTableUtilTest.java b/server/base/src/test/java/org/apache/accumulo/server/util/ReplicationTableUtilTest.java
index 3983bde..04a83d3 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/util/ReplicationTableUtilTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/util/ReplicationTableUtilTest.java
@@ -91,7 +91,7 @@ public class ReplicationTableUtilTest {
     String myFile = "file:////home/user/accumulo/wal/server+port/" + uuid;
 
     long createdTime = System.currentTimeMillis();
-    ReplicationTableUtil.updateFiles(context, new KeyExtent(new Text("1"), null, null), Collections.singleton(myFile), StatusUtil.fileCreated(createdTime));
+    ReplicationTableUtil.updateFiles(context, new KeyExtent(new Text("1"), null, null), myFile, StatusUtil.fileCreated(createdTime));
 
     verify(writer);
 


[21/34] accumulo git commit: Merge branch 'master' of http://git-wip-us.apache.org/repos/asf/accumulo

Posted by ec...@apache.org.
Merge branch 'master' of http://git-wip-us.apache.org/repos/asf/accumulo


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

Branch: refs/heads/master
Commit: bd6dbba5b871e89aacdfb4b06b223047e9a0a159
Parents: 65d0145 cf69821
Author: Eric Newton <er...@gmail.com>
Authored: Thu Apr 16 12:05:44 2015 -0400
Committer: Eric Newton <er...@gmail.com>
Committed: Thu Apr 16 12:05:44 2015 -0400

----------------------------------------------------------------------
 .../core/client/NewTableConfiguration.java      | 108 ----------------
 .../client/admin/CompactionStrategyConfig.java  |   8 +-
 .../client/admin/NewTableConfiguration.java     | 107 ++++++++++++++++
 .../core/client/admin/TableOperations.java      |   1 -
 .../core/client/impl/ConditionalWriterImpl.java |   2 +-
 .../core/client/impl/TableOperationsImpl.java   |   2 +-
 .../core/client/mapreduce/RangeInputSplit.java  |   5 +
 .../core/client/mock/MockTableOperations.java   |   2 +-
 .../accumulo/core/file/BloomFilterLayer.java    |   2 +-
 .../accumulo/core/util/LoggingRunnable.java     |  50 --------
 .../accumulo/core/util/NamingThreadFactory.java |   1 +
 .../client/impl/TableOperationsHelperTest.java  |   2 +-
 .../client/mock/MockTableOperationsTest.java    |   2 +-
 .../minicluster/MiniAccumuloInstance.java       |   3 +-
 .../minicluster/MiniAccumuloClusterTest.java    |   2 +-
 .../org/apache/accumulo/proxy/ProxyServer.java  |   2 +-
 .../accumulo/server/client/BulkImporter.java    |   2 +-
 .../server/problems/ProblemReports.java         |   2 +-
 .../accumulo/server/rpc/TServerUtils.java       |   2 +-
 .../org/apache/accumulo/monitor/Monitor.java    |   2 +-
 .../apache/accumulo/tserver/TabletServer.java   |   4 +-
 .../tserver/TabletServerResourceManager.java    |   5 +-
 .../EverythingCompactionStrategy.java           |   2 +-
 .../apache/accumulo/tserver/log/DfsLogger.java  |   2 +-
 .../tserver/tablet/MinorCompactionTask.java     |   8 +-
 .../apache/accumulo/tserver/tablet/Tablet.java  |  63 ++++-----
 .../shell/commands/CreateTableCommand.java      |   2 +-
 .../apache/accumulo/test/HardListIterator.java  | 115 +++++++++++++++++
 .../accumulo/test/ConditionalWriterIT.java      |   2 +-
 .../test/CreateTableWithNewTableConfigIT.java   |   2 +-
 .../org/apache/accumulo/test/NamespacesIT.java  |   2 +-
 .../apache/accumulo/test/TableOperationsIT.java | 127 +++++++++++++++++++
 .../java/org/apache/accumulo/test/VolumeIT.java |   2 +-
 .../accumulo/test/functional/LogicalTimeIT.java |   2 +-
 .../accumulo/test/functional/MergeIT.java       |   2 +-
 .../test/replication/CyclicReplicationIT.java   |   2 +-
 36 files changed, 423 insertions(+), 226 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/bd6dbba5/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bd6dbba5/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bd6dbba5/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bd6dbba5/test/src/test/java/org/apache/accumulo/test/ConditionalWriterIT.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/bd6dbba5/test/src/test/java/org/apache/accumulo/test/VolumeIT.java
----------------------------------------------------------------------


[20/34] accumulo git commit: ACCUMULO-3423 reduce the number of calls to add a flag to the metadata table

Posted by ec...@apache.org.
ACCUMULO-3423 reduce the number of calls to add a flag to the metadata table


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

Branch: refs/heads/master
Commit: 65d01458b69c20ec5ae3f83a4beb266481e8c840
Parents: 8f5e002
Author: Eric Newton <er...@gmail.com>
Authored: Thu Apr 16 12:03:12 2015 -0400
Committer: Eric Newton <er...@gmail.com>
Committed: Thu Apr 16 12:03:12 2015 -0400

----------------------------------------------------------------------
 .../org/apache/accumulo/server/TabletLevel.java | 34 ++++++++++++++++++++
 .../accumulo/server/util/MetadataTableUtil.java |  7 ++--
 .../apache/accumulo/tserver/TabletLevel.java    | 34 --------------------
 .../apache/accumulo/tserver/TabletServer.java   | 16 ++++-----
 .../tserver/log/TabletServerLogger.java         | 10 ++++--
 5 files changed, 52 insertions(+), 49 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/65d01458/server/base/src/main/java/org/apache/accumulo/server/TabletLevel.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/TabletLevel.java b/server/base/src/main/java/org/apache/accumulo/server/TabletLevel.java
new file mode 100644
index 0000000..91e5ee9
--- /dev/null
+++ b/server/base/src/main/java/org/apache/accumulo/server/TabletLevel.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.server;
+
+import org.apache.accumulo.core.data.KeyExtent;
+
+public enum TabletLevel {
+  ROOT,
+  META,
+  NORMAL;
+
+  public static TabletLevel getLevel(KeyExtent extent) {
+    if (!extent.isMeta())
+      return NORMAL;
+    if (extent.isRootTablet())
+      return ROOT;
+    return META;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/65d01458/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java
----------------------------------------------------------------------
diff --git 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
index 6d6253d..09a5b40 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
@@ -81,6 +81,7 @@ import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
 import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeMissingPolicy;
 import org.apache.accumulo.server.AccumuloServerContext;
 import org.apache.accumulo.server.ServerConstants;
+import org.apache.accumulo.server.TabletLevel;
 import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.accumulo.server.fs.FileRef;
 import org.apache.accumulo.server.fs.VolumeManager;
@@ -1057,9 +1058,9 @@ public class MetadataTableUtil {
     return tabletEntries;
   }
 
-  public static void addNewLogMarker(ClientContext context, ZooLock zooLock, final TServerInstance tabletSession, final Path filename, KeyExtent extent) {
+  public static void addNewLogMarker(ClientContext context, ZooLock zooLock, final TServerInstance tabletSession, final Path filename, TabletLevel level) {
     log.debug("Adding log entry " + filename);
-    if (extent.isRootTablet()) {
+    if (level == TabletLevel.ROOT) {
       retryZooKeeperUpdate(context, zooLock, new ZooOperation() {
         @Override
         public void run(IZooReaderWriter rw) throws KeeperException, InterruptedException, IOException {
@@ -1077,7 +1078,7 @@ public class MetadataTableUtil {
       Mutation m = new Mutation(CurrentLogsSection.getRowPrefix() + tabletSession.toString());
       m.put(CurrentLogsSection.COLF, new Text(filename.toString()), new Value(EMPTY_BYTES));
       String tableName = MetadataTable.NAME;
-      if (extent.isMeta()) {
+      if (level == TabletLevel.META) {
         tableName = RootTable.NAME;
       }
       BatchWriter bw = null;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/65d01458/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletLevel.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletLevel.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletLevel.java
deleted file mode 100644
index 1e82393..0000000
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletLevel.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.tserver;
-
-import org.apache.accumulo.core.data.KeyExtent;
-
-public enum TabletLevel {
-  ROOT,
-  META,
-  NORMAL;
-
-  public static TabletLevel getLevel(KeyExtent extent) {
-    if (!extent.isMeta())
-      return NORMAL;
-    if (extent.isRootTablet())
-      return ROOT;
-    return META;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/65d01458/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
index 7c64cee..f527a2a 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
@@ -149,6 +149,7 @@ import org.apache.accumulo.server.Accumulo;
 import org.apache.accumulo.server.AccumuloServerContext;
 import org.apache.accumulo.server.GarbageCollectionLogger;
 import org.apache.accumulo.server.ServerOpts;
+import org.apache.accumulo.server.TabletLevel;
 import org.apache.accumulo.server.client.ClientServiceHandler;
 import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.accumulo.server.conf.ServerConfigurationFactory;
@@ -3038,23 +3039,18 @@ public class TabletServer extends AccumuloServerContext implements Runnable {
     }
   }
 
-  public void addLoggersToMetadata(DfsLogger copy, KeyExtent extent) {
-    TabletLevel level = TabletLevel.getLevel(extent);
+  public void addLoggersToMetadata(DfsLogger copy, TabletLevel level) {
     // serialize the updates to the metadata per level: avoids updating the level more than once
     // updating one level, may cause updates to other levels, so we need to release the lock on metadataTableLogs
     synchronized (levelLocks[level.ordinal()]) {
       EnumSet<TabletLevel> set = null;
-      synchronized (metadataTableLogs) {
-        set = metadataTableLogs.putIfAbsent(copy, EnumSet.of(level));
-      }
+      set = metadataTableLogs.putIfAbsent(copy, EnumSet.of(level));
       if (set == null || !set.contains(level) || level == TabletLevel.ROOT) {
         log.info("Writing log marker for level " + level + " " + copy.getFileName());
-        MetadataTableUtil.addNewLogMarker(this, this.getLock(), this.getTabletSession(), copy.getPath(), extent);
-      }
-      synchronized (metadataTableLogs) {
-        set = metadataTableLogs.get(copy);
-        set.add(level);
+        MetadataTableUtil.addNewLogMarker(this, this.getLock(), this.getTabletSession(), copy.getPath(), level);
       }
+      set = metadataTableLogs.get(copy);
+      set.add(level);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/65d01458/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
index dd54798..2541e50 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
@@ -20,6 +20,7 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -41,6 +42,7 @@ import org.apache.accumulo.core.protobuf.ProtobufUtil;
 import org.apache.accumulo.core.replication.ReplicationConfigurationUtil;
 import org.apache.accumulo.core.util.SimpleThreadPool;
 import org.apache.accumulo.core.util.UtilWaitThread;
+import org.apache.accumulo.server.TabletLevel;
 import org.apache.accumulo.server.conf.TableConfiguration;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.replication.StatusUtil;
@@ -227,8 +229,12 @@ public class TabletServerLogger {
             log.debug("Creating next WAL");
             DfsLogger alog = new DfsLogger(tserver.getServerConfig(), syncCounter, flushCounter);
             alog.open(tserver.getClientAddressString());
+            EnumSet<TabletLevel> levels = EnumSet.noneOf(TabletLevel.class);
             for (Tablet tablet : tserver.getOnlineTablets()) {
-              tserver.addLoggersToMetadata(alog, tablet.getExtent());
+              levels.add(TabletLevel.getLevel(tablet.getExtent()));
+            }
+            for (TabletLevel level : levels) {
+              tserver.addLoggersToMetadata(alog, level);
             }
             log.debug("Created next WAL " + alog.getFileName());
             while (!nextLog.offer(alog, 12, TimeUnit.HOURS)) {
@@ -308,7 +314,7 @@ public class TabletServerLogger {
                 // Scribble out a tablet definition and then write to the metadata table
                 defineTablet(commitSession);
                 if (currentLogId == logId.get())
-                  tserver.addLoggersToMetadata(copy, commitSession.getExtent());
+                  tserver.addLoggersToMetadata(copy, TabletLevel.getLevel(commitSession.getExtent()));
               } finally {
                 commitSession.finishUpdatingLogsUsed();
               }


[05/34] accumulo git commit: ACCUMULO-3423 respond to elserj's review

Posted by ec...@apache.org.
ACCUMULO-3423 respond to elserj's review


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

Branch: refs/heads/master
Commit: c8f3b7d3b8591ea2330437549b9578ae9feebaaf
Parents: b2539fb
Author: Eric C. Newton <er...@gmail.com>
Authored: Thu Feb 26 15:17:04 2015 -0500
Committer: Eric C. Newton <er...@gmail.com>
Committed: Thu Feb 26 15:17:04 2015 -0500

----------------------------------------------------------------------
 .../accumulo/core/replication/StatusUtil.java   |   9 +
 .../thrift/TabletClientService.java             | 749 ++++++++++++++++++-
 core/src/main/thrift/tabletserver.thrift        |   1 +
 .../server/master/state/MetaDataStateStore.java |   3 +
 .../master/state/MetaDataTableScanner.java      |   7 +-
 .../server/master/state/TabletStateStore.java   |   3 +
 .../master/state/ZooTabletStateStore.java       |   1 +
 .../accumulo/server/util/MetadataTableUtil.java |   7 +-
 .../gc/GarbageCollectWriteAheadLogs.java        |   4 +-
 .../accumulo/master/replication/WorkMaker.java  |   2 +-
 .../apache/accumulo/tserver/TabletServer.java   |   5 +
 .../tserver/log/TabletServerLogger.java         |  42 +-
 .../apache/accumulo/tserver/tablet/Tablet.java  |   1 +
 .../test/performance/thrift/NullTserver.java    |   3 +
 14 files changed, 793 insertions(+), 44 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/c8f3b7d3/core/src/main/java/org/apache/accumulo/core/replication/StatusUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/replication/StatusUtil.java b/core/src/main/java/org/apache/accumulo/core/replication/StatusUtil.java
index a7cd3f5..d8ec403 100644
--- a/core/src/main/java/org/apache/accumulo/core/replication/StatusUtil.java
+++ b/core/src/main/java/org/apache/accumulo/core/replication/StatusUtil.java
@@ -32,6 +32,7 @@ public class StatusUtil {
   private static final Value INF_END_REPLICATION_STATUS_VALUE, CLOSED_STATUS_VALUE;
 
   private static final Status.Builder CREATED_STATUS_BUILDER;
+  private static final Status.Builder INF_END_REPLICATION_STATUS_BUILDER;
 
   static {
     CREATED_STATUS_BUILDER = Status.newBuilder();
@@ -45,6 +46,7 @@ public class StatusUtil {
     builder.setEnd(0);
     builder.setInfiniteEnd(true);
     builder.setClosed(false);
+    INF_END_REPLICATION_STATUS_BUILDER = builder;
     INF_END_REPLICATION_STATUS = builder.build();
     INF_END_REPLICATION_STATUS_VALUE = ProtobufUtil.toValue(INF_END_REPLICATION_STATUS);
 
@@ -153,6 +155,13 @@ public class StatusUtil {
   /**
    * @return A {@link Status} for an open file of unspecified length, all of which needs replicating.
    */
+  public static Status openWithUnknownLength(long timeCreated) {
+    return INF_END_REPLICATION_STATUS_BUILDER.setCreatedTime(timeCreated).build();
+  }
+
+  /**
+   * @return A {@link Status} for an open file of unspecified length, all of which needs replicating.
+   */
   public static Status openWithUnknownLength() {
     return INF_END_REPLICATION_STATUS;
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c8f3b7d3/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/TabletClientService.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/TabletClientService.java b/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/TabletClientService.java
index d6d4afd..02bd4e1 100644
--- a/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/TabletClientService.java
+++ b/core/src/main/java/org/apache/accumulo/core/tabletserver/thrift/TabletClientService.java
@@ -110,6 +110,8 @@ import org.slf4j.LoggerFactory;
 
     public List<ActiveCompaction> getActiveCompactions(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials) throws org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException, org.apache.thrift.TException;
 
+    public void removeLogs(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, List<String> filenames) throws org.apache.thrift.TException;
+
     public List<String> getActiveLogs(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials) throws org.apache.thrift.TException;
 
   }
@@ -174,6 +176,8 @@ import org.slf4j.LoggerFactory;
 
     public void getActiveCompactions(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
+    public void removeLogs(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, List<String> filenames, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
+
     public void getActiveLogs(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException;
 
   }
@@ -896,6 +900,20 @@ import org.slf4j.LoggerFactory;
       throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "getActiveCompactions failed: unknown result");
     }
 
+    public void removeLogs(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, List<String> filenames) throws org.apache.thrift.TException
+    {
+      send_removeLogs(tinfo, credentials, filenames);
+    }
+
+    public void send_removeLogs(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, List<String> filenames) throws org.apache.thrift.TException
+    {
+      removeLogs_args args = new removeLogs_args();
+      args.setTinfo(tinfo);
+      args.setCredentials(credentials);
+      args.setFilenames(filenames);
+      sendBase("removeLogs", args);
+    }
+
     public List<String> getActiveLogs(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials) throws org.apache.thrift.TException
     {
       send_getActiveLogs(tinfo, credentials);
@@ -2099,6 +2117,43 @@ import org.slf4j.LoggerFactory;
       }
     }
 
+    public void removeLogs(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, List<String> filenames, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
+      checkReady();
+      removeLogs_call method_call = new removeLogs_call(tinfo, credentials, filenames, resultHandler, this, ___protocolFactory, ___transport);
+      this.___currentMethod = method_call;
+      ___manager.call(method_call);
+    }
+
+    public static class removeLogs_call extends org.apache.thrift.async.TAsyncMethodCall {
+      private org.apache.accumulo.core.trace.thrift.TInfo tinfo;
+      private org.apache.accumulo.core.security.thrift.TCredentials credentials;
+      private List<String> filenames;
+      public removeLogs_call(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, List<String> filenames, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException {
+        super(client, protocolFactory, transport, resultHandler, true);
+        this.tinfo = tinfo;
+        this.credentials = credentials;
+        this.filenames = filenames;
+      }
+
+      public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException {
+        prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("removeLogs", org.apache.thrift.protocol.TMessageType.CALL, 0));
+        removeLogs_args args = new removeLogs_args();
+        args.setTinfo(tinfo);
+        args.setCredentials(credentials);
+        args.setFilenames(filenames);
+        args.write(prot);
+        prot.writeMessageEnd();
+      }
+
+      public void getResult() throws org.apache.thrift.TException {
+        if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) {
+          throw new IllegalStateException("Method call not finished!");
+        }
+        org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array());
+        org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport);
+      }
+    }
+
     public void getActiveLogs(org.apache.accumulo.core.trace.thrift.TInfo tinfo, org.apache.accumulo.core.security.thrift.TCredentials credentials, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException {
       checkReady();
       getActiveLogs_call method_call = new getActiveLogs_call(tinfo, credentials, resultHandler, this, ___protocolFactory, ___transport);
@@ -2176,6 +2231,7 @@ import org.slf4j.LoggerFactory;
       processMap.put("fastHalt", new fastHalt());
       processMap.put("getActiveScans", new getActiveScans());
       processMap.put("getActiveCompactions", new getActiveCompactions());
+      processMap.put("removeLogs", new removeLogs());
       processMap.put("getActiveLogs", new getActiveLogs());
       return processMap;
     }
@@ -2837,6 +2893,25 @@ import org.slf4j.LoggerFactory;
       }
     }
 
+    public static class removeLogs<I extends Iface> extends org.apache.thrift.ProcessFunction<I, removeLogs_args> {
+      public removeLogs() {
+        super("removeLogs");
+      }
+
+      public removeLogs_args getEmptyArgsInstance() {
+        return new removeLogs_args();
+      }
+
+      protected boolean isOneway() {
+        return true;
+      }
+
+      public org.apache.thrift.TBase getResult(I iface, removeLogs_args args) throws org.apache.thrift.TException {
+        iface.removeLogs(args.tinfo, args.credentials, args.filenames);
+        return null;
+      }
+    }
+
     public static class getActiveLogs<I extends Iface> extends org.apache.thrift.ProcessFunction<I, getActiveLogs_args> {
       public getActiveLogs() {
         super("getActiveLogs");
@@ -2899,6 +2974,7 @@ import org.slf4j.LoggerFactory;
       processMap.put("fastHalt", new fastHalt());
       processMap.put("getActiveScans", new getActiveScans());
       processMap.put("getActiveCompactions", new getActiveCompactions());
+      processMap.put("removeLogs", new removeLogs());
       processMap.put("getActiveLogs", new getActiveLogs());
       return processMap;
     }
@@ -4291,6 +4367,34 @@ import org.slf4j.LoggerFactory;
       }
     }
 
+    public static class removeLogs<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, removeLogs_args, Void> {
+      public removeLogs() {
+        super("removeLogs");
+      }
+
+      public removeLogs_args getEmptyArgsInstance() {
+        return new removeLogs_args();
+      }
+
+      public AsyncMethodCallback<Void> getResultHandler(final AsyncFrameBuffer fb, final int seqid) {
+        final org.apache.thrift.AsyncProcessFunction fcall = this;
+        return new AsyncMethodCallback<Void>() { 
+          public void onComplete(Void o) {
+          }
+          public void onError(Exception e) {
+          }
+        };
+      }
+
+      protected boolean isOneway() {
+        return true;
+      }
+
+      public void start(I iface, removeLogs_args args, org.apache.thrift.async.AsyncMethodCallback<Void> resultHandler) throws TException {
+        iface.removeLogs(args.tinfo, args.credentials, args.filenames,resultHandler);
+      }
+    }
+
     public static class getActiveLogs<I extends AsyncIface> extends org.apache.thrift.AsyncProcessFunction<I, getActiveLogs_args, List<String>> {
       public getActiveLogs() {
         super("getActiveLogs");
@@ -33069,6 +33173,619 @@ import org.slf4j.LoggerFactory;
 
   }
 
+  public static class removeLogs_args implements org.apache.thrift.TBase<removeLogs_args, removeLogs_args._Fields>, java.io.Serializable, Cloneable, Comparable<removeLogs_args>   {
+    private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("removeLogs_args");
+
+    private static final org.apache.thrift.protocol.TField TINFO_FIELD_DESC = new org.apache.thrift.protocol.TField("tinfo", org.apache.thrift.protocol.TType.STRUCT, (short)1);
+    private static final org.apache.thrift.protocol.TField CREDENTIALS_FIELD_DESC = new org.apache.thrift.protocol.TField("credentials", org.apache.thrift.protocol.TType.STRUCT, (short)2);
+    private static final org.apache.thrift.protocol.TField FILENAMES_FIELD_DESC = new org.apache.thrift.protocol.TField("filenames", org.apache.thrift.protocol.TType.LIST, (short)3);
+
+    private static final Map<Class<? extends IScheme>, SchemeFactory> schemes = new HashMap<Class<? extends IScheme>, SchemeFactory>();
+    static {
+      schemes.put(StandardScheme.class, new removeLogs_argsStandardSchemeFactory());
+      schemes.put(TupleScheme.class, new removeLogs_argsTupleSchemeFactory());
+    }
+
+    public org.apache.accumulo.core.trace.thrift.TInfo tinfo; // required
+    public org.apache.accumulo.core.security.thrift.TCredentials credentials; // required
+    public List<String> filenames; // required
+
+    /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */
+    public enum _Fields implements org.apache.thrift.TFieldIdEnum {
+      TINFO((short)1, "tinfo"),
+      CREDENTIALS((short)2, "credentials"),
+      FILENAMES((short)3, "filenames");
+
+      private static final Map<String, _Fields> byName = new HashMap<String, _Fields>();
+
+      static {
+        for (_Fields field : EnumSet.allOf(_Fields.class)) {
+          byName.put(field.getFieldName(), field);
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, or null if its not found.
+       */
+      public static _Fields findByThriftId(int fieldId) {
+        switch(fieldId) {
+          case 1: // TINFO
+            return TINFO;
+          case 2: // CREDENTIALS
+            return CREDENTIALS;
+          case 3: // FILENAMES
+            return FILENAMES;
+          default:
+            return null;
+        }
+      }
+
+      /**
+       * Find the _Fields constant that matches fieldId, throwing an exception
+       * if it is not found.
+       */
+      public static _Fields findByThriftIdOrThrow(int fieldId) {
+        _Fields fields = findByThriftId(fieldId);
+        if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!");
+        return fields;
+      }
+
+      /**
+       * Find the _Fields constant that matches name, or null if its not found.
+       */
+      public static _Fields findByName(String name) {
+        return byName.get(name);
+      }
+
+      private final short _thriftId;
+      private final String _fieldName;
+
+      _Fields(short thriftId, String fieldName) {
+        _thriftId = thriftId;
+        _fieldName = fieldName;
+      }
+
+      public short getThriftFieldId() {
+        return _thriftId;
+      }
+
+      public String getFieldName() {
+        return _fieldName;
+      }
+    }
+
+    // isset id assignments
+    public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap;
+    static {
+      Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class);
+      tmpMap.put(_Fields.TINFO, new org.apache.thrift.meta_data.FieldMetaData("tinfo", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.trace.thrift.TInfo.class)));
+      tmpMap.put(_Fields.CREDENTIALS, new org.apache.thrift.meta_data.FieldMetaData("credentials", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, org.apache.accumulo.core.security.thrift.TCredentials.class)));
+      tmpMap.put(_Fields.FILENAMES, new org.apache.thrift.meta_data.FieldMetaData("filenames", org.apache.thrift.TFieldRequirementType.DEFAULT, 
+          new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, 
+              new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))));
+      metaDataMap = Collections.unmodifiableMap(tmpMap);
+      org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(removeLogs_args.class, metaDataMap);
+    }
+
+    public removeLogs_args() {
+    }
+
+    public removeLogs_args(
+      org.apache.accumulo.core.trace.thrift.TInfo tinfo,
+      org.apache.accumulo.core.security.thrift.TCredentials credentials,
+      List<String> filenames)
+    {
+      this();
+      this.tinfo = tinfo;
+      this.credentials = credentials;
+      this.filenames = filenames;
+    }
+
+    /**
+     * Performs a deep copy on <i>other</i>.
+     */
+    public removeLogs_args(removeLogs_args other) {
+      if (other.isSetTinfo()) {
+        this.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo(other.tinfo);
+      }
+      if (other.isSetCredentials()) {
+        this.credentials = new org.apache.accumulo.core.security.thrift.TCredentials(other.credentials);
+      }
+      if (other.isSetFilenames()) {
+        List<String> __this__filenames = new ArrayList<String>(other.filenames);
+        this.filenames = __this__filenames;
+      }
+    }
+
+    public removeLogs_args deepCopy() {
+      return new removeLogs_args(this);
+    }
+
+    @Override
+    public void clear() {
+      this.tinfo = null;
+      this.credentials = null;
+      this.filenames = null;
+    }
+
+    public org.apache.accumulo.core.trace.thrift.TInfo getTinfo() {
+      return this.tinfo;
+    }
+
+    public removeLogs_args setTinfo(org.apache.accumulo.core.trace.thrift.TInfo tinfo) {
+      this.tinfo = tinfo;
+      return this;
+    }
+
+    public void unsetTinfo() {
+      this.tinfo = null;
+    }
+
+    /** Returns true if field tinfo is set (has been assigned a value) and false otherwise */
+    public boolean isSetTinfo() {
+      return this.tinfo != null;
+    }
+
+    public void setTinfoIsSet(boolean value) {
+      if (!value) {
+        this.tinfo = null;
+      }
+    }
+
+    public org.apache.accumulo.core.security.thrift.TCredentials getCredentials() {
+      return this.credentials;
+    }
+
+    public removeLogs_args setCredentials(org.apache.accumulo.core.security.thrift.TCredentials credentials) {
+      this.credentials = credentials;
+      return this;
+    }
+
+    public void unsetCredentials() {
+      this.credentials = null;
+    }
+
+    /** Returns true if field credentials is set (has been assigned a value) and false otherwise */
+    public boolean isSetCredentials() {
+      return this.credentials != null;
+    }
+
+    public void setCredentialsIsSet(boolean value) {
+      if (!value) {
+        this.credentials = null;
+      }
+    }
+
+    public int getFilenamesSize() {
+      return (this.filenames == null) ? 0 : this.filenames.size();
+    }
+
+    public java.util.Iterator<String> getFilenamesIterator() {
+      return (this.filenames == null) ? null : this.filenames.iterator();
+    }
+
+    public void addToFilenames(String elem) {
+      if (this.filenames == null) {
+        this.filenames = new ArrayList<String>();
+      }
+      this.filenames.add(elem);
+    }
+
+    public List<String> getFilenames() {
+      return this.filenames;
+    }
+
+    public removeLogs_args setFilenames(List<String> filenames) {
+      this.filenames = filenames;
+      return this;
+    }
+
+    public void unsetFilenames() {
+      this.filenames = null;
+    }
+
+    /** Returns true if field filenames is set (has been assigned a value) and false otherwise */
+    public boolean isSetFilenames() {
+      return this.filenames != null;
+    }
+
+    public void setFilenamesIsSet(boolean value) {
+      if (!value) {
+        this.filenames = null;
+      }
+    }
+
+    public void setFieldValue(_Fields field, Object value) {
+      switch (field) {
+      case TINFO:
+        if (value == null) {
+          unsetTinfo();
+        } else {
+          setTinfo((org.apache.accumulo.core.trace.thrift.TInfo)value);
+        }
+        break;
+
+      case CREDENTIALS:
+        if (value == null) {
+          unsetCredentials();
+        } else {
+          setCredentials((org.apache.accumulo.core.security.thrift.TCredentials)value);
+        }
+        break;
+
+      case FILENAMES:
+        if (value == null) {
+          unsetFilenames();
+        } else {
+          setFilenames((List<String>)value);
+        }
+        break;
+
+      }
+    }
+
+    public Object getFieldValue(_Fields field) {
+      switch (field) {
+      case TINFO:
+        return getTinfo();
+
+      case CREDENTIALS:
+        return getCredentials();
+
+      case FILENAMES:
+        return getFilenames();
+
+      }
+      throw new IllegalStateException();
+    }
+
+    /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */
+    public boolean isSet(_Fields field) {
+      if (field == null) {
+        throw new IllegalArgumentException();
+      }
+
+      switch (field) {
+      case TINFO:
+        return isSetTinfo();
+      case CREDENTIALS:
+        return isSetCredentials();
+      case FILENAMES:
+        return isSetFilenames();
+      }
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean equals(Object that) {
+      if (that == null)
+        return false;
+      if (that instanceof removeLogs_args)
+        return this.equals((removeLogs_args)that);
+      return false;
+    }
+
+    public boolean equals(removeLogs_args that) {
+      if (that == null)
+        return false;
+
+      boolean this_present_tinfo = true && this.isSetTinfo();
+      boolean that_present_tinfo = true && that.isSetTinfo();
+      if (this_present_tinfo || that_present_tinfo) {
+        if (!(this_present_tinfo && that_present_tinfo))
+          return false;
+        if (!this.tinfo.equals(that.tinfo))
+          return false;
+      }
+
+      boolean this_present_credentials = true && this.isSetCredentials();
+      boolean that_present_credentials = true && that.isSetCredentials();
+      if (this_present_credentials || that_present_credentials) {
+        if (!(this_present_credentials && that_present_credentials))
+          return false;
+        if (!this.credentials.equals(that.credentials))
+          return false;
+      }
+
+      boolean this_present_filenames = true && this.isSetFilenames();
+      boolean that_present_filenames = true && that.isSetFilenames();
+      if (this_present_filenames || that_present_filenames) {
+        if (!(this_present_filenames && that_present_filenames))
+          return false;
+        if (!this.filenames.equals(that.filenames))
+          return false;
+      }
+
+      return true;
+    }
+
+    @Override
+    public int hashCode() {
+      return 0;
+    }
+
+    @Override
+    public int compareTo(removeLogs_args other) {
+      if (!getClass().equals(other.getClass())) {
+        return getClass().getName().compareTo(other.getClass().getName());
+      }
+
+      int lastComparison = 0;
+
+      lastComparison = Boolean.valueOf(isSetTinfo()).compareTo(other.isSetTinfo());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetTinfo()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tinfo, other.tinfo);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(isSetCredentials()).compareTo(other.isSetCredentials());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetCredentials()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.credentials, other.credentials);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      lastComparison = Boolean.valueOf(isSetFilenames()).compareTo(other.isSetFilenames());
+      if (lastComparison != 0) {
+        return lastComparison;
+      }
+      if (isSetFilenames()) {
+        lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.filenames, other.filenames);
+        if (lastComparison != 0) {
+          return lastComparison;
+        }
+      }
+      return 0;
+    }
+
+    public _Fields fieldForId(int fieldId) {
+      return _Fields.findByThriftId(fieldId);
+    }
+
+    public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException {
+      schemes.get(iprot.getScheme()).getScheme().read(iprot, this);
+    }
+
+    public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException {
+      schemes.get(oprot.getScheme()).getScheme().write(oprot, this);
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder("removeLogs_args(");
+      boolean first = true;
+
+      sb.append("tinfo:");
+      if (this.tinfo == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.tinfo);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("credentials:");
+      if (this.credentials == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.credentials);
+      }
+      first = false;
+      if (!first) sb.append(", ");
+      sb.append("filenames:");
+      if (this.filenames == null) {
+        sb.append("null");
+      } else {
+        sb.append(this.filenames);
+      }
+      first = false;
+      sb.append(")");
+      return sb.toString();
+    }
+
+    public void validate() throws org.apache.thrift.TException {
+      // check for required fields
+      // check for sub-struct validity
+      if (tinfo != null) {
+        tinfo.validate();
+      }
+      if (credentials != null) {
+        credentials.validate();
+      }
+    }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException {
+      try {
+        write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException {
+      try {
+        read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in)));
+      } catch (org.apache.thrift.TException te) {
+        throw new java.io.IOException(te);
+      }
+    }
+
+    private static class removeLogs_argsStandardSchemeFactory implements SchemeFactory {
+      public removeLogs_argsStandardScheme getScheme() {
+        return new removeLogs_argsStandardScheme();
+      }
+    }
+
+    private static class removeLogs_argsStandardScheme extends StandardScheme<removeLogs_args> {
+
+      public void read(org.apache.thrift.protocol.TProtocol iprot, removeLogs_args struct) throws org.apache.thrift.TException {
+        org.apache.thrift.protocol.TField schemeField;
+        iprot.readStructBegin();
+        while (true)
+        {
+          schemeField = iprot.readFieldBegin();
+          if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { 
+            break;
+          }
+          switch (schemeField.id) {
+            case 1: // TINFO
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+                struct.tinfo.read(iprot);
+                struct.setTinfoIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 2: // CREDENTIALS
+              if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) {
+                struct.credentials = new org.apache.accumulo.core.security.thrift.TCredentials();
+                struct.credentials.read(iprot);
+                struct.setCredentialsIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            case 3: // FILENAMES
+              if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
+                {
+                  org.apache.thrift.protocol.TList _list314 = iprot.readListBegin();
+                  struct.filenames = new ArrayList<String>(_list314.size);
+                  for (int _i315 = 0; _i315 < _list314.size; ++_i315)
+                  {
+                    String _elem316;
+                    _elem316 = iprot.readString();
+                    struct.filenames.add(_elem316);
+                  }
+                  iprot.readListEnd();
+                }
+                struct.setFilenamesIsSet(true);
+              } else { 
+                org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+              }
+              break;
+            default:
+              org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type);
+          }
+          iprot.readFieldEnd();
+        }
+        iprot.readStructEnd();
+
+        // check for required fields of primitive type, which can't be checked in the validate method
+        struct.validate();
+      }
+
+      public void write(org.apache.thrift.protocol.TProtocol oprot, removeLogs_args struct) throws org.apache.thrift.TException {
+        struct.validate();
+
+        oprot.writeStructBegin(STRUCT_DESC);
+        if (struct.tinfo != null) {
+          oprot.writeFieldBegin(TINFO_FIELD_DESC);
+          struct.tinfo.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        if (struct.credentials != null) {
+          oprot.writeFieldBegin(CREDENTIALS_FIELD_DESC);
+          struct.credentials.write(oprot);
+          oprot.writeFieldEnd();
+        }
+        if (struct.filenames != null) {
+          oprot.writeFieldBegin(FILENAMES_FIELD_DESC);
+          {
+            oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.filenames.size()));
+            for (String _iter317 : struct.filenames)
+            {
+              oprot.writeString(_iter317);
+            }
+            oprot.writeListEnd();
+          }
+          oprot.writeFieldEnd();
+        }
+        oprot.writeFieldStop();
+        oprot.writeStructEnd();
+      }
+
+    }
+
+    private static class removeLogs_argsTupleSchemeFactory implements SchemeFactory {
+      public removeLogs_argsTupleScheme getScheme() {
+        return new removeLogs_argsTupleScheme();
+      }
+    }
+
+    private static class removeLogs_argsTupleScheme extends TupleScheme<removeLogs_args> {
+
+      @Override
+      public void write(org.apache.thrift.protocol.TProtocol prot, removeLogs_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol oprot = (TTupleProtocol) prot;
+        BitSet optionals = new BitSet();
+        if (struct.isSetTinfo()) {
+          optionals.set(0);
+        }
+        if (struct.isSetCredentials()) {
+          optionals.set(1);
+        }
+        if (struct.isSetFilenames()) {
+          optionals.set(2);
+        }
+        oprot.writeBitSet(optionals, 3);
+        if (struct.isSetTinfo()) {
+          struct.tinfo.write(oprot);
+        }
+        if (struct.isSetCredentials()) {
+          struct.credentials.write(oprot);
+        }
+        if (struct.isSetFilenames()) {
+          {
+            oprot.writeI32(struct.filenames.size());
+            for (String _iter318 : struct.filenames)
+            {
+              oprot.writeString(_iter318);
+            }
+          }
+        }
+      }
+
+      @Override
+      public void read(org.apache.thrift.protocol.TProtocol prot, removeLogs_args struct) throws org.apache.thrift.TException {
+        TTupleProtocol iprot = (TTupleProtocol) prot;
+        BitSet incoming = iprot.readBitSet(3);
+        if (incoming.get(0)) {
+          struct.tinfo = new org.apache.accumulo.core.trace.thrift.TInfo();
+          struct.tinfo.read(iprot);
+          struct.setTinfoIsSet(true);
+        }
+        if (incoming.get(1)) {
+          struct.credentials = new org.apache.accumulo.core.security.thrift.TCredentials();
+          struct.credentials.read(iprot);
+          struct.setCredentialsIsSet(true);
+        }
+        if (incoming.get(2)) {
+          {
+            org.apache.thrift.protocol.TList _list319 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.filenames = new ArrayList<String>(_list319.size);
+            for (int _i320 = 0; _i320 < _list319.size; ++_i320)
+            {
+              String _elem321;
+              _elem321 = iprot.readString();
+              struct.filenames.add(_elem321);
+            }
+          }
+          struct.setFilenamesIsSet(true);
+        }
+      }
+    }
+
+  }
+
   public static class getActiveLogs_args implements org.apache.thrift.TBase<getActiveLogs_args, getActiveLogs_args._Fields>, java.io.Serializable, Cloneable, Comparable<getActiveLogs_args>   {
     private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("getActiveLogs_args");
 
@@ -33839,13 +34556,13 @@ import org.slf4j.LoggerFactory;
             case 0: // SUCCESS
               if (schemeField.type == org.apache.thrift.protocol.TType.LIST) {
                 {
-                  org.apache.thrift.protocol.TList _list314 = iprot.readListBegin();
-                  struct.success = new ArrayList<String>(_list314.size);
-                  for (int _i315 = 0; _i315 < _list314.size; ++_i315)
+                  org.apache.thrift.protocol.TList _list322 = iprot.readListBegin();
+                  struct.success = new ArrayList<String>(_list322.size);
+                  for (int _i323 = 0; _i323 < _list322.size; ++_i323)
                   {
-                    String _elem316;
-                    _elem316 = iprot.readString();
-                    struct.success.add(_elem316);
+                    String _elem324;
+                    _elem324 = iprot.readString();
+                    struct.success.add(_elem324);
                   }
                   iprot.readListEnd();
                 }
@@ -33873,9 +34590,9 @@ import org.slf4j.LoggerFactory;
           oprot.writeFieldBegin(SUCCESS_FIELD_DESC);
           {
             oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.success.size()));
-            for (String _iter317 : struct.success)
+            for (String _iter325 : struct.success)
             {
-              oprot.writeString(_iter317);
+              oprot.writeString(_iter325);
             }
             oprot.writeListEnd();
           }
@@ -33906,9 +34623,9 @@ import org.slf4j.LoggerFactory;
         if (struct.isSetSuccess()) {
           {
             oprot.writeI32(struct.success.size());
-            for (String _iter318 : struct.success)
+            for (String _iter326 : struct.success)
             {
-              oprot.writeString(_iter318);
+              oprot.writeString(_iter326);
             }
           }
         }
@@ -33920,13 +34637,13 @@ import org.slf4j.LoggerFactory;
         BitSet incoming = iprot.readBitSet(1);
         if (incoming.get(0)) {
           {
-            org.apache.thrift.protocol.TList _list319 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
-            struct.success = new ArrayList<String>(_list319.size);
-            for (int _i320 = 0; _i320 < _list319.size; ++_i320)
+            org.apache.thrift.protocol.TList _list327 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32());
+            struct.success = new ArrayList<String>(_list327.size);
+            for (int _i328 = 0; _i328 < _list327.size; ++_i328)
             {
-              String _elem321;
-              _elem321 = iprot.readString();
-              struct.success.add(_elem321);
+              String _elem329;
+              _elem329 = iprot.readString();
+              struct.success.add(_elem329);
             }
           }
           struct.setSuccessIsSet(true);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c8f3b7d3/core/src/main/thrift/tabletserver.thrift
----------------------------------------------------------------------
diff --git a/core/src/main/thrift/tabletserver.thrift b/core/src/main/thrift/tabletserver.thrift
index f25a08f..4a31036 100644
--- a/core/src/main/thrift/tabletserver.thrift
+++ b/core/src/main/thrift/tabletserver.thrift
@@ -205,6 +205,7 @@ service TabletClientService extends client.ClientService {
   
   list<ActiveScan> getActiveScans(2:trace.TInfo tinfo, 1:security.TCredentials credentials) throws (1:client.ThriftSecurityException sec)
   list<ActiveCompaction> getActiveCompactions(2:trace.TInfo tinfo, 1:security.TCredentials credentials) throws (1:client.ThriftSecurityException sec)
+  oneway void removeLogs(1:trace.TInfo tinfo, 2:security.TCredentials credentials, 3:list<string> filenames)
   list<string> getActiveLogs(1:trace.TInfo tinfo, 2:security.TCredentials credentials)
 }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c8f3b7d3/server/base/src/main/java/org/apache/accumulo/server/master/state/MetaDataStateStore.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/state/MetaDataStateStore.java b/server/base/src/main/java/org/apache/accumulo/server/master/state/MetaDataStateStore.java
index 270bb31..1749904 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/state/MetaDataStateStore.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/state/MetaDataStateStore.java
@@ -33,8 +33,10 @@ import org.apache.accumulo.core.metadata.schema.MetadataSchema;
 import org.apache.accumulo.core.tabletserver.log.LogEntry;
 import org.apache.accumulo.server.AccumuloServerContext;
 import org.apache.hadoop.io.Text;
+import org.apache.log4j.Logger;
 
 public class MetaDataStateStore extends TabletStateStore {
+  private static final Logger log = Logger.getLogger(MetaDataStateStore.class);
 
   private static final int THREADS = 4;
   private static final int LATENCY = 1000;
@@ -172,6 +174,7 @@ public class MetaDataStateStore extends TabletStateStore {
         writer.addMutation(m);
       }
     } catch (Exception ex) {
+      log.error("Error marking logs as unused: " + logs);
       throw new DistributedStoreException(ex);
     } finally {
       try {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c8f3b7d3/server/base/src/main/java/org/apache/accumulo/server/master/state/MetaDataTableScanner.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/state/MetaDataTableScanner.java b/server/base/src/main/java/org/apache/accumulo/server/master/state/MetaDataTableScanner.java
index dac7fe6..130364b 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/state/MetaDataTableScanner.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/state/MetaDataTableScanner.java
@@ -43,9 +43,10 @@ import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.Lo
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.server.master.state.TabletLocationState.BadLocationStateException;
 import org.apache.hadoop.io.Text;
+import org.apache.log4j.Logger;
 
 public class MetaDataTableScanner implements ClosableIterator<TabletLocationState> {
-  //private static final Logger log = Logger.getLogger(MetaDataTableScanner.class);
+  private static final Logger log = Logger.getLogger(MetaDataTableScanner.class);
 
   BatchScanner mdScanner = null;
   Iterator<Entry<Key,Value>> iter = null;
@@ -172,7 +173,9 @@ public class MetaDataTableScanner implements ClosableIterator<TabletLocationStat
       }
     }
     if (extent == null) {
-      throw new BadLocationStateException("No prev-row for key extent " + decodedRow, k.getRow());
+      String msg = "No prev-row for key extent " + decodedRow;
+      log.error(msg);
+      throw new BadLocationStateException(msg, k.getRow());
     }
     return new TabletLocationState(extent, future, current, last, walogs, chopped);
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c8f3b7d3/server/base/src/main/java/org/apache/accumulo/server/master/state/TabletStateStore.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/state/TabletStateStore.java b/server/base/src/main/java/org/apache/accumulo/server/master/state/TabletStateStore.java
index de90d98..13db05b 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/state/TabletStateStore.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/state/TabletStateStore.java
@@ -87,6 +87,9 @@ public abstract class TabletStateStore implements Iterable<TabletLocationState>
     store.setLocations(Collections.singletonList(assignment));
   }
 
+  /**
+   * When a server fails, its logs must be marked as unused after the log markers are moved to the tablets.
+   */
   abstract public void markLogsAsUnused(AccumuloServerContext context, Map<TServerInstance, List<String>> logs) throws DistributedStoreException;
 
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c8f3b7d3/server/base/src/main/java/org/apache/accumulo/server/master/state/ZooTabletStateStore.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/state/ZooTabletStateStore.java b/server/base/src/main/java/org/apache/accumulo/server/master/state/ZooTabletStateStore.java
index a044434..66bad4e 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/state/ZooTabletStateStore.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/state/ZooTabletStateStore.java
@@ -197,6 +197,7 @@ public class ZooTabletStateStore extends TabletStateStore {
 
   @Override
   public void markLogsAsUnused(AccumuloServerContext context, Map<TServerInstance,List<String>> logs) {
+    // the root table is not replicated, so unassigning the root tablet has removed the current log marker
   }
 
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c8f3b7d3/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java
----------------------------------------------------------------------
diff --git 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
index a95cffa..ebf4b1b 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
@@ -89,6 +89,7 @@ import org.apache.accumulo.server.master.state.TServerInstance;
 import org.apache.accumulo.server.tablets.TabletTime;
 import org.apache.accumulo.server.zookeeper.ZooLock;
 import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
@@ -1067,7 +1068,7 @@ public class MetadataTableUtil {
         @Override
         public void run(IZooReaderWriter rw) throws KeeperException, InterruptedException, IOException {
           String root = ZooUtil.getRoot(HdfsZooInstance.getInstance()) + RootTable.ZROOT_TABLET_CURRENT_LOGS;
-          String[] parts = filename.split("/");
+          String[] parts = StringUtils.split(filename, '/');
           String uniqueId = parts[parts.length - 1];
           String path = root + "/" + CurrentLogsSection.getRowPrefix() + tabletSession.toString() + uniqueId;
           rw.putPersistentData(path, filename.getBytes(UTF_8), NodeExistsPolicy.OVERWRITE);
@@ -1075,7 +1076,7 @@ public class MetadataTableUtil {
       });
     } else {
       Mutation m = new Mutation(CurrentLogsSection.getRowPrefix() + tabletSession.toString());
-      m.put("log", filename, new Value(EMPTY_BYTES));
+      m.put(CurrentLogsSection.COLF, new Text(filename), new Value(EMPTY_BYTES));
       String tableName = MetadataTable.NAME;
       if (extent.isMeta()) {
         tableName = RootTable.NAME;
@@ -1095,7 +1096,7 @@ public class MetadataTableUtil {
       @Override
       public void run(IZooReaderWriter rw) throws KeeperException, InterruptedException, IOException {
         String root = ZooUtil.getRoot(HdfsZooInstance.getInstance()) + RootTable.ZROOT_TABLET_CURRENT_LOGS;
-        String[] parts = filename.split("/");
+        String[] parts = StringUtils.split(filename, '/');
         String uniqueId = parts[parts.length - 1];
         String path = root + "/" + CurrentLogsSection.getRowPrefix() + tabletSession.toString() + uniqueId;
         log.debug("Removing entry " + path + " from zookeeper");

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c8f3b7d3/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 2561eec..a7703e9 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
@@ -105,8 +105,8 @@ public class GarbageCollectWriteAheadLogs {
       LiveTServerSet liveServers = new LiveTServerSet(context, new Listener() {
         @Override
         public void update(LiveTServerSet current, Set<TServerInstance> deleted, Set<TServerInstance> added) {
-          log.debug("New tablet server noticed: " + added);
-          log.debug("Tablet server removed: " + deleted);
+          log.debug("New tablet servers noticed: " + added);
+          log.debug("Tablet servers removed: " + deleted);
         }
       });
       Set<TServerInstance> currentServers = liveServers.getCurrentServers();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c8f3b7d3/server/master/src/main/java/org/apache/accumulo/master/replication/WorkMaker.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/replication/WorkMaker.java b/server/master/src/main/java/org/apache/accumulo/master/replication/WorkMaker.java
index bc4c64f..4490824 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/replication/WorkMaker.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/replication/WorkMaker.java
@@ -107,7 +107,7 @@ public class WorkMaker {
         // Don't create the record if we have nothing to do.
         // TODO put this into a filter on serverside
         if (!shouldCreateWork(status)) {
-          log.info("Not creating work: " + status.toString());
+          log.debug("Not creating work: " + status.toString());
           continue;
         }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c8f3b7d3/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
index b12fccc..3b7ff03 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
@@ -1715,6 +1715,11 @@ public class TabletServer extends AccumuloServerContext implements Runnable {
       String log = logger.getLogFile();
       return Collections.singletonList(log);
     }
+
+    @Override
+    public void removeLogs(TInfo tinfo, TCredentials credentials, List<String> filenames) throws TException {
+      log.warn("Garbage collector is attempting to remove logs through the tablet server");
+    }
   }
 
   private class SplitRunner implements Runnable {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c8f3b7d3/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
index 6455726..46101c1 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
@@ -25,6 +25,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
+import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
@@ -37,7 +38,9 @@ import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.protobuf.ProtobufUtil;
 import org.apache.accumulo.core.replication.ReplicationConfigurationUtil;
+import org.apache.accumulo.core.replication.StatusUtil;
 import org.apache.accumulo.core.replication.proto.Replication.Status;
+import org.apache.accumulo.core.util.SimpleThreadPool;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.server.conf.TableConfiguration;
 import org.apache.accumulo.server.fs.VolumeManager;
@@ -73,8 +76,8 @@ public class TabletServerLogger {
 
   // The current logger
   private DfsLogger currentLog = null;
-  private DfsLogger nextLog = null;
-  private Thread nextLogThread = null;
+  private final AtomicReference<DfsLogger> nextLog = new AtomicReference<>(null);
+  private final ThreadPoolExecutor nextLogMaker = new SimpleThreadPool(1, "WALog creator");
 
   // The current generation of logs.
   // Because multiple threads can be using a log at one time, a log
@@ -194,16 +197,16 @@ public class TabletServerLogger {
     }
 
     try {
-      if (nextLog != null) {
-        log.info("Using next log " + nextLog.getFileName());
-        currentLog = nextLog;
-        nextLog = null;
+      DfsLogger next = nextLog.getAndSet(null);
+      if (next != null) {
+        log.info("Using next log " + next.getFileName());
+        currentLog = next;
       } else {
         DfsLogger alog = new DfsLogger(tserver.getServerConfig(), syncCounter, flushCounter);
         alog.open(tserver.getClientAddressString());
         currentLog = alog;
       }
-      if (nextLog == null) {
+      if (nextLog.get() == null) {
         createNextLog();
       }
       logId.incrementAndGet();
@@ -217,31 +220,30 @@ public class TabletServerLogger {
     }
   }
 
+  // callers are synchronized already
   private void createNextLog() {
-    if (nextLogThread == null) {
-      nextLogThread = new Thread() {
+    if (nextLogMaker.getActiveCount() == 0) {
+      nextLogMaker.submit(new Runnable() {
         @Override
         public void run() {
           try {
-            log.info("Creating next WAL");
-            this.setName("Creating next WAL");
+            log.debug("Creating next WAL");
             DfsLogger alog = new DfsLogger(tserver.getServerConfig(), syncCounter, flushCounter);
             alog.open(tserver.getClientAddressString());
             for (Tablet tablet : tserver.getOnlineTablets()) {
-              // TODO
               tserver.addLoggersToMetadata(alog, tablet.getExtent());
             }
-            nextLog = alog;
-
-            log.info("Created next WAL " + alog.getFileName());
+            log.debug("Created next WAL " + alog.getFileName());
+            alog = nextLog.getAndSet(alog);
+            if (alog != null) {
+              log.debug("closing unused next log: " + alog.getFileName());
+              alog.close();
+            }
           } catch (Exception t) {
             log.error(t, t);
-          } finally {
-            nextLogThread = null;
           }
         }
-      };
-      nextLogThread.start();
+      });
     }
   }
 
@@ -317,7 +319,7 @@ public class TabletServerLogger {
               // Need to release
               KeyExtent extent = commitSession.getExtent();
               if (ReplicationConfigurationUtil.isEnabled(extent, tserver.getTableConfiguration(extent))) {
-                Status status = Status.newBuilder().setInfiniteEnd(true).setCreatedTime(System.currentTimeMillis()).build();
+                Status status = StatusUtil.fileCreated(System.currentTimeMillis());
                 log.debug("Writing " + ProtobufUtil.toString(status) + " to metadata table for " + copy.getFileName());
                 // Got some new WALs, note this in the metadata table
                 ReplicationTableUtil.updateFiles(tserver, commitSession.getExtent(), copy.getFileName(), status);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c8f3b7d3/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
index fa1ae86..0c1edfa 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
@@ -2377,6 +2377,7 @@ public class Tablet implements TabletCommitter {
 
   private ConcurrentSkipListSet<DfsLogger> currentLogs = new ConcurrentSkipListSet<DfsLogger>();
 
+  // currentLogs may be updated while a tablet is otherwise locked
   public Set<DfsLogger> getCurrentLogFiles() {
     return new HashSet<DfsLogger>(currentLogs);
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c8f3b7d3/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java b/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java
index b8a60c1..ac7fd70 100644
--- a/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java
+++ b/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java
@@ -228,6 +228,9 @@ public class NullTserver {
     public List<String> getActiveLogs(TInfo tinfo, TCredentials credentials) throws TException {
       return null;
     }
+
+    @Override
+    public void removeLogs(TInfo tinfo, TCredentials credentials, List<String> filenames) throws TException { }
   }
 
   static class Opts extends Help {


[02/34] accumulo git commit: ACCUMULO-3625 use log markers against tservers, not tablets

Posted by ec...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/b2539fb1/server/master/src/main/java/org/apache/accumulo/master/TabletGroupWatcher.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/TabletGroupWatcher.java b/server/master/src/main/java/org/apache/accumulo/master/TabletGroupWatcher.java
index 755e322..edea93f 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/TabletGroupWatcher.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/TabletGroupWatcher.java
@@ -161,6 +161,7 @@ class TabletGroupWatcher extends Daemon {
         List<Assignment> assigned = new ArrayList<Assignment>();
         List<TabletLocationState> assignedToDeadServers = new ArrayList<TabletLocationState>();
         Map<KeyExtent,TServerInstance> unassigned = new HashMap<KeyExtent,TServerInstance>();
+        Map<TServerInstance, List<String>> logsForDeadServers = new TreeMap<>();
 
         MasterState masterState = master.getMasterState();
         int[] counts = new int[TabletState.values().length];
@@ -173,6 +174,7 @@ class TabletGroupWatcher extends Daemon {
           if (tls == null) {
             continue;
           }
+          Master.log.debug(store.name() + " location State: " + tls);
           // ignore entries for tables that do not exist in zookeeper
           if (TableManager.getInstance().getTableState(tls.extent.getTableId().toString()) == null)
             continue;
@@ -182,7 +184,7 @@ class TabletGroupWatcher extends Daemon {
 
           // Don't overwhelm the tablet servers with work
           if (unassigned.size() + unloaded > Master.MAX_TSERVER_WORK_CHUNK * currentTServers.size()) {
-            flushChanges(destinations, assignments, assigned, assignedToDeadServers, unassigned);
+            flushChanges(destinations, assignments, assigned, assignedToDeadServers, logsForDeadServers, unassigned);
             assignments.clear();
             assigned.clear();
             assignedToDeadServers.clear();
@@ -237,7 +239,7 @@ class TabletGroupWatcher extends Daemon {
                 assignedToDeadServers.add(tls);
                 if (server.equals(this.master.migrations.get(tls.extent)))
                   this.master.migrations.remove(tls.extent);
-                // log.info("Current servers " + currentTServers.keySet());
+                MetadataTableUtil.fetchLogsForDeadServer(master, master.getMasterLock(), tls.extent, tls.futureOrCurrent(), logsForDeadServers);
                 break;
               case UNASSIGNED:
                 // maybe it's a finishing migration
@@ -266,7 +268,7 @@ class TabletGroupWatcher extends Daemon {
                 break;
               case ASSIGNED_TO_DEAD_SERVER:
                 assignedToDeadServers.add(tls);
-                // log.info("Current servers " + currentTServers.keySet());
+                MetadataTableUtil.fetchLogsForDeadServer(master, master.getMasterLock(), tls.extent, tls.futureOrCurrent(), logsForDeadServers);
                 break;
               case HOSTED:
                 TServerConnection conn = this.master.tserverSet.getConnection(server);
@@ -285,7 +287,8 @@ class TabletGroupWatcher extends Daemon {
           counts[state.ordinal()]++;
         }
 
-        flushChanges(destinations, assignments, assigned, assignedToDeadServers, unassigned);
+        flushChanges(destinations, assignments, assigned, assignedToDeadServers, logsForDeadServers, unassigned);
+        store.markLogsAsUnused(master, logsForDeadServers);
 
         // provide stats after flushing changes to avoid race conditions w/ delete table
         stats.end(masterState);
@@ -723,12 +726,19 @@ class TabletGroupWatcher extends Daemon {
     }
   }
 
-  private void flushChanges(SortedMap<TServerInstance,TabletServerStatus> currentTServers, List<Assignment> assignments, List<Assignment> assigned,
-      List<TabletLocationState> assignedToDeadServers, Map<KeyExtent,TServerInstance> unassigned) throws DistributedStoreException, TException {
+  private void flushChanges(
+      SortedMap<TServerInstance,TabletServerStatus> currentTServers,
+      List<Assignment> assignments,
+      List<Assignment> assigned,
+      List<TabletLocationState> assignedToDeadServers,
+      Map<TServerInstance, List<String>> logsForDeadServers,
+      Map<KeyExtent,TServerInstance> unassigned)
+          throws DistributedStoreException, TException {
     if (!assignedToDeadServers.isEmpty()) {
       int maxServersToShow = min(assignedToDeadServers.size(), 100);
       Master.log.debug(assignedToDeadServers.size() + " assigned to dead servers: " + assignedToDeadServers.subList(0, maxServersToShow) + "...");
-      store.unassign(assignedToDeadServers);
+      Master.log.debug("logs for dead servers: " + logsForDeadServers);
+      store.unassign(assignedToDeadServers, logsForDeadServers);
       this.master.nextEvent.event("Marked %d tablets as unassigned because they don't have current servers", assignedToDeadServers.size());
     }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b2539fb1/server/master/src/main/java/org/apache/accumulo/master/replication/WorkMaker.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/replication/WorkMaker.java b/server/master/src/main/java/org/apache/accumulo/master/replication/WorkMaker.java
index b8e0b40..bc4c64f 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/replication/WorkMaker.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/replication/WorkMaker.java
@@ -107,6 +107,7 @@ public class WorkMaker {
         // Don't create the record if we have nothing to do.
         // TODO put this into a filter on serverside
         if (!shouldCreateWork(status)) {
+          log.info("Not creating work: " + status.toString());
           continue;
         }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b2539fb1/server/master/src/main/java/org/apache/accumulo/master/state/MergeStats.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/state/MergeStats.java b/server/master/src/main/java/org/apache/accumulo/master/state/MergeStats.java
index 8cdaf9f..895717a 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/state/MergeStats.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/state/MergeStats.java
@@ -30,6 +30,7 @@ import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.RootTable;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.server.cli.ClientOpts;
@@ -186,7 +187,7 @@ public class MergeStats {
     Text tableId = extent.getTableId();
     Text first = KeyExtent.getMetadataEntry(tableId, start);
     Range range = new Range(first, false, null, true);
-    scanner.setRange(range);
+    scanner.setRange(range.clip(MetadataSchema.TabletsSection.getRange()));
     KeyExtent prevExtent = null;
 
     log.debug("Scanning range " + range);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b2539fb1/server/master/src/test/java/org/apache/accumulo/master/TestMergeState.java
----------------------------------------------------------------------
diff --git a/server/master/src/test/java/org/apache/accumulo/master/TestMergeState.java b/server/master/src/test/java/org/apache/accumulo/master/TestMergeState.java
index b0240f1..b39dcb8 100644
--- a/server/master/src/test/java/org/apache/accumulo/master/TestMergeState.java
+++ b/server/master/src/test/java/org/apache/accumulo/master/TestMergeState.java
@@ -186,7 +186,7 @@ public class TestMergeState {
     // take it offline
     m = tablet.getPrevRowUpdateMutation();
     Collection<Collection<String>> walogs = Collections.emptyList();
-    metaDataStateStore.unassign(Collections.singletonList(new TabletLocationState(tablet, null, state.someTServer, null, walogs, false)));
+    metaDataStateStore.unassign(Collections.singletonList(new TabletLocationState(tablet, null, state.someTServer, null, walogs, false)), null);
 
     // now we can split
     stats = scan(state, metaDataStateStore);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b2539fb1/server/master/src/test/java/org/apache/accumulo/master/state/RootTabletStateStoreTest.java
----------------------------------------------------------------------
diff --git a/server/master/src/test/java/org/apache/accumulo/master/state/RootTabletStateStoreTest.java b/server/master/src/test/java/org/apache/accumulo/master/state/RootTabletStateStoreTest.java
index abceae4..db16bcb 100644
--- a/server/master/src/test/java/org/apache/accumulo/master/state/RootTabletStateStoreTest.java
+++ b/server/master/src/test/java/org/apache/accumulo/master/state/RootTabletStateStoreTest.java
@@ -181,7 +181,7 @@ public class RootTabletStateStoreTest {
     } catch (BadLocationStateException e) {
       fail("Unexpected error " + e);
     }
-    tstore.unassign(Collections.singletonList(assigned));
+    tstore.unassign(Collections.singletonList(assigned), null);
     count = 0;
     for (TabletLocationState location : tstore) {
       assertEquals(location.extent, root);
@@ -209,7 +209,7 @@ public class RootTabletStateStoreTest {
       fail("Unexpected error " + e);
     }
     try {
-      tstore.unassign(Collections.singletonList(broken));
+      tstore.unassign(Collections.singletonList(broken), null);
       Assert.fail("should not get here");
     } catch (IllegalArgumentException ex) {}
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b2539fb1/server/tserver/src/main/java/org/apache/accumulo/server/GarbageCollectionLogger.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/server/GarbageCollectionLogger.java b/server/tserver/src/main/java/org/apache/accumulo/server/GarbageCollectionLogger.java
index 7d09fe3..b05032e 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/server/GarbageCollectionLogger.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/server/GarbageCollectionLogger.java
@@ -34,7 +34,8 @@ public class GarbageCollectionLogger {
   private long gcTimeIncreasedCount = 0;
   private static long lastMemoryCheckTime = 0;
 
-  public GarbageCollectionLogger() {}
+  public GarbageCollectionLogger() {
+  }
 
   public synchronized void logGCInfo(AccumuloConfiguration conf) {
     final long now = System.currentTimeMillis();
@@ -96,7 +97,7 @@ public class GarbageCollectionLogger {
     final long keepAliveTimeout = conf.getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT);
     if (lastMemoryCheckTime > 0 && lastMemoryCheckTime < now) {
       final long diff = now - lastMemoryCheckTime;
-      if (diff > keepAliveTimeout) {
+      if (diff > keepAliveTimeout + 1000) {
         log.warn(String.format("GC pause checker not called in a timely fashion. Expected every %.1f seconds but was %.1f seconds since last check",
             keepAliveTimeout / 1000., diff / 1000.));
       }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b2539fb1/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletLevel.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletLevel.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletLevel.java
new file mode 100644
index 0000000..1e82393
--- /dev/null
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletLevel.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.tserver;
+
+import org.apache.accumulo.core.data.KeyExtent;
+
+public enum TabletLevel {
+  ROOT,
+  META,
+  NORMAL;
+
+  public static TabletLevel getLevel(KeyExtent extent) {
+    if (!extent.isMeta())
+      return NORMAL;
+    if (extent.isRootTablet())
+      return ROOT;
+    return META;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b2539fb1/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
index 662ee31..b12fccc 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
@@ -19,7 +19,6 @@ package org.apache.accumulo.tserver;
 import static java.nio.charset.StandardCharsets.UTF_8;
 import static org.apache.accumulo.server.problems.ProblemType.TABLET_LOAD;
 
-import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.lang.management.ManagementFactory;
 import java.net.UnknownHostException;
@@ -29,6 +28,7 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
+import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
@@ -44,6 +44,7 @@ import java.util.TreeMap;
 import java.util.TreeSet;
 import java.util.concurrent.BlockingDeque;
 import java.util.concurrent.CancellationException;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.LinkedBlockingDeque;
 import java.util.concurrent.ThreadPoolExecutor;
@@ -146,7 +147,6 @@ import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
 import org.apache.accumulo.server.Accumulo;
 import org.apache.accumulo.server.AccumuloServerContext;
 import org.apache.accumulo.server.GarbageCollectionLogger;
-import org.apache.accumulo.server.ServerConstants;
 import org.apache.accumulo.server.ServerOpts;
 import org.apache.accumulo.server.client.ClientServiceHandler;
 import org.apache.accumulo.server.client.HdfsZooInstance;
@@ -1438,6 +1438,7 @@ public class TabletServer extends AccumuloServerContext implements Runnable {
       }
     }
 
+
     @Override
     public void loadTablet(TInfo tinfo, TCredentials credentials, String lock, final TKeyExtent textent) {
 
@@ -1498,6 +1499,7 @@ public class TabletServer extends AccumuloServerContext implements Runnable {
       final AssignmentHandler ah = new AssignmentHandler(extent);
       // final Runnable ah = new LoggingRunnable(log, );
       // Root tablet assignment must take place immediately
+
       if (extent.isRootTablet()) {
         new Daemon("Root Tablet Assignment") {
           @Override
@@ -1690,66 +1692,6 @@ public class TabletServer extends AccumuloServerContext implements Runnable {
     }
 
     @Override
-    public void removeLogs(TInfo tinfo, TCredentials credentials, List<String> filenames) throws TException {
-      String myname = getClientAddressString();
-      myname = myname.replace(':', '+');
-      Set<String> loggers = new HashSet<String>();
-      logger.getLogFiles(loggers);
-      Set<String> loggerUUIDs = new HashSet<String>();
-      for (String logger : loggers)
-        loggerUUIDs.add(new Path(logger).getName());
-
-      nextFile: for (String filename : filenames) {
-        String uuid = new Path(filename).getName();
-        // skip any log we're currently using
-        if (loggerUUIDs.contains(uuid))
-          continue nextFile;
-
-        List<Tablet> onlineTabletsCopy = new ArrayList<Tablet>();
-        synchronized (onlineTablets) {
-          onlineTabletsCopy.addAll(onlineTablets.values());
-        }
-        for (Tablet tablet : onlineTabletsCopy) {
-          for (String current : tablet.getCurrentLogFiles()) {
-            if (current.contains(uuid)) {
-              log.info("Attempted to delete " + filename + " from tablet " + tablet.getExtent());
-              continue nextFile;
-            }
-          }
-        }
-
-        try {
-          Path source = new Path(filename);
-          if (TabletServer.this.getConfiguration().getBoolean(Property.TSERV_ARCHIVE_WALOGS)) {
-            Path walogArchive = fs.matchingFileSystem(source, ServerConstants.getWalogArchives());
-            fs.mkdirs(walogArchive);
-            Path dest = new Path(walogArchive, source.getName());
-            log.info("Archiving walog " + source + " to " + dest);
-            if (!fs.rename(source, dest))
-              log.error("rename is unsuccessful");
-          } else {
-            log.info("Deleting walog " + filename);
-            Path sourcePath = new Path(filename);
-            if (!(!TabletServer.this.getConfiguration().getBoolean(Property.GC_TRASH_IGNORE) && fs.moveToTrash(sourcePath))
-                && !fs.deleteRecursively(sourcePath))
-              log.warn("Failed to delete walog " + source);
-            for (String recovery : ServerConstants.getRecoveryDirs()) {
-              Path recoveryPath = new Path(recovery, source.getName());
-              try {
-                if (fs.moveToTrash(recoveryPath) || fs.deleteRecursively(recoveryPath))
-                  log.info("Deleted any recovery log " + filename);
-              } catch (FileNotFoundException ex) {
-                // ignore
-              }
-            }
-          }
-        } catch (IOException e) {
-          log.warn("Error attempting to delete write-ahead log " + filename + ": " + e);
-        }
-      }
-    }
-
-    @Override
     public List<ActiveCompaction> getActiveCompactions(TInfo tinfo, TCredentials credentials) throws ThriftSecurityException, TException {
       try {
         checkPermission(credentials, null, "getActiveCompactions");
@@ -1770,14 +1712,13 @@ public class TabletServer extends AccumuloServerContext implements Runnable {
 
     @Override
     public List<String> getActiveLogs(TInfo tinfo, TCredentials credentials) throws TException {
-      Set<String> logs = new HashSet<String>();
-      logger.getLogFiles(logs);
-      return new ArrayList<String>(logs);
+      String log = logger.getLogFile();
+      return Collections.singletonList(log);
     }
   }
 
   private class SplitRunner implements Runnable {
-    private Tablet tablet;
+    private final Tablet tablet;
 
     public SplitRunner(Tablet tablet) {
       this.tablet = tablet;
@@ -2031,7 +1972,7 @@ public class TabletServer extends AccumuloServerContext implements Runnable {
           log.error("Unexpected error ", e);
         }
         log.debug("Unassigning " + tls);
-        TabletStateStore.unassign(TabletServer.this, tls);
+        TabletStateStore.unassign(TabletServer.this, tls, null);
       } catch (DistributedStoreException ex) {
         log.warn("Unable to update storage", ex);
       } catch (KeeperException e) {
@@ -2238,29 +2179,6 @@ public class TabletServer extends AccumuloServerContext implements Runnable {
     }
   }
 
-  public void addLoggersToMetadata(List<DfsLogger> logs, KeyExtent extent, int id) {
-    if (!this.onlineTablets.containsKey(extent)) {
-      log.info("Not adding " + logs.size() + " logs for extent " + extent + " as alias " + id + " tablet is offline");
-      // minor compaction due to recovery... don't make updates... if it finishes, there will be no WALs,
-      // if it doesn't, we'll need to do the same recovery with the old files.
-      return;
-    }
-
-    log.info("Adding " + logs.size() + " logs for extent " + extent + " as alias " + id);
-    long now = RelativeTime.currentTimeMillis();
-    List<String> logSet = new ArrayList<String>();
-    for (DfsLogger log : logs)
-      logSet.add(log.getFileName());
-    LogEntry entry = new LogEntry();
-    entry.extent = extent;
-    entry.tabletId = id;
-    entry.timestamp = now;
-    entry.server = logs.get(0).getLogger();
-    entry.filename = logs.get(0).getFileName();
-    entry.logSet = logSet;
-    MetadataTableUtil.addLogEntry(this, entry, getLock());
-  }
-
   private HostAndPort startServer(AccumuloConfiguration conf, String address, Property portHint, TProcessor processor, String threadName)
       throws UnknownHostException {
     Property maxMessageSizeProperty = (conf.get(Property.TSERV_MAX_MESSAGE_SIZE) != null ? Property.TSERV_MAX_MESSAGE_SIZE : Property.GENERAL_MAX_MESSAGE_SIZE);
@@ -2968,6 +2886,7 @@ public class TabletServer extends AccumuloServerContext implements Runnable {
   public void minorCompactionFinished(CommitSession tablet, String newDatafile, int walogSeq) throws IOException {
     totalMinorCompactions.incrementAndGet();
     logger.minorCompactionFinished(tablet, newDatafile, walogSeq);
+    removeUnusedWALs();
   }
 
   public void minorCompactionStarted(CommitSession tablet, int lastUpdateSequence, String newMapfileLocation) throws IOException {
@@ -2986,14 +2905,11 @@ public class TabletServer extends AccumuloServerContext implements Runnable {
     });
     for (LogEntry entry : sorted) {
       Path recovery = null;
-      for (String log : entry.logSet) {
-        Path finished = RecoveryPath.getRecoveryPath(fs, fs.getFullPath(FileType.WAL, log));
-        finished = SortedLogState.getFinishedMarkerPath(finished);
-        TabletServer.log.info("Looking for " + finished);
-        if (fs.exists(finished)) {
-          recovery = finished.getParent();
-          break;
-        }
+      Path finished = RecoveryPath.getRecoveryPath(fs, fs.getFullPath(FileType.WAL, entry.filename));
+      finished = SortedLogState.getFinishedMarkerPath(finished);
+      TabletServer.log.info("Looking for " + finished);
+      if (fs.exists(finished)) {
+        recovery = finished.getParent();
       }
       if (recovery == null)
         throw new IOException("Unable to find recovery files for extent " + extent + " logEntry: " + entry);
@@ -3035,7 +2951,9 @@ public class TabletServer extends AccumuloServerContext implements Runnable {
   }
 
   public Collection<Tablet> getOnlineTablets() {
-    return Collections.unmodifiableCollection(onlineTablets.values());
+    synchronized (onlineTablets) {
+      return new ArrayList<Tablet>(onlineTablets.values());
+    }
   }
 
   public VolumeManager getFileSystem() {
@@ -3061,4 +2979,54 @@ public class TabletServer extends AccumuloServerContext implements Runnable {
   public SecurityOperation getSecurityOperation() {
     return security;
   }
+
+  // avoid unnecessary redundant markings to meta
+  ConcurrentHashMap<DfsLogger, EnumSet<TabletLevel>> metadataTableLogs = new ConcurrentHashMap<>();
+
+  // remove any meta entries after a rolled log is no longer referenced
+  Set<DfsLogger> closedLogs = new HashSet<>();
+
+  private void removeUnusedWALs() {
+    Set<DfsLogger> candidates;
+    synchronized (closedLogs) {
+      candidates = new HashSet<>(closedLogs);
+    }
+    for (Tablet tablet : getOnlineTablets()) {
+      candidates.removeAll(tablet.getCurrentLogFiles());
+    }
+    try {
+      Set<String> filenames = new HashSet<>();
+      for (DfsLogger candidate : candidates) {
+        filenames.add(candidate.getFileName());
+      }
+      MetadataTableUtil.markLogUnused(this, this.getLock(), this.getTabletSession(), filenames);
+      synchronized (closedLogs) {
+        closedLogs.removeAll(candidates);
+      }
+    } catch (AccumuloException ex) {
+      log.info(ex.toString(), ex);
+    }
+  }
+
+  public void addLoggersToMetadata(DfsLogger copy, KeyExtent extent) {
+    TabletLevel level = TabletLevel.getLevel(extent);
+    synchronized (level) {
+      EnumSet<TabletLevel> set = metadataTableLogs.putIfAbsent(copy, EnumSet.of(level));
+      if (set == null || !set.contains(level) || level == TabletLevel.ROOT) {
+        log.info("Writing log marker for level " + level + " " + copy.getFileName());
+        MetadataTableUtil.addNewLogMarker(this, this.getLock(), this.getTabletSession(), copy.getFileName(), extent);
+        if (set != null) {
+          set.add(level);
+        }
+      }
+    }
+  }
+
+  public void walogClosed(DfsLogger currentLog) {
+    metadataTableLogs.remove(currentLog);
+    synchronized (closedLogs) {
+      closedLogs.add(currentLog);
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b2539fb1/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
index 5acf5eb..f8bcfbc 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
@@ -74,7 +74,7 @@ import com.google.common.base.Optional;
  * Wrap a connection to a logger.
  *
  */
-public class DfsLogger {
+public class DfsLogger implements Comparable<DfsLogger> {
   public static final String LOG_FILE_HEADER_V2 = "--- Log File Header (v2) ---";
   public static final String LOG_FILE_HEADER_V3 = "--- Log File Header (v3) ---";
 
@@ -621,4 +621,9 @@ public class DfsLogger {
     return Joiner.on(":").join(parts[parts.length - 2].split("[+]"));
   }
 
+  @Override
+  public int compareTo(DfsLogger o) {
+    return getFileName().compareTo(o.getFileName());
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b2539fb1/server/tserver/src/main/java/org/apache/accumulo/tserver/log/SortedLogRecovery.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/SortedLogRecovery.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/SortedLogRecovery.java
index 405ec70..efdbbf4 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/SortedLogRecovery.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/SortedLogRecovery.java
@@ -179,7 +179,7 @@ public class SortedLogRecovery {
     // find the maximum tablet id... because a tablet may leave a tserver and then come back, in which case it would have a different tablet id
     // for the maximum tablet id, find the minimum sequence #... may be ok to find the max seq, but just want to make the code behave like it used to
     while (reader.next(key, value)) {
-      // LogReader.printEntry(entry);
+      // log.debug("Event " + key.event + " tablet " + key.tablet);
       if (key.event != DEFINE_TABLET)
         break;
       if (key.tablet.equals(extent) || key.tablet.equals(alternative)) {
@@ -208,7 +208,7 @@ public class SortedLogRecovery {
         if (lastStartToFinish.compactionStatus == Status.INITIAL)
           lastStartToFinish.compactionStatus = Status.COMPLETE;
         if (key.seq <= lastStartToFinish.lastStart)
-          throw new RuntimeException("Sequence numbers are not increasing for start/stop events.");
+          throw new RuntimeException("Sequence numbers are not increasing for start/stop events: " + key.seq + " vs " + lastStartToFinish.lastStart);
         lastStartToFinish.update(fileno, key.seq);
 
         // Tablet server finished the minor compaction, but didn't remove the entry from the METADATA table.
@@ -217,7 +217,7 @@ public class SortedLogRecovery {
           lastStartToFinish.update(-1);
       } else if (key.event == COMPACTION_FINISH) {
         if (key.seq <= lastStartToFinish.lastStart)
-          throw new RuntimeException("Sequence numbers are not increasing for start/stop events.");
+          throw new RuntimeException("Sequence numbers are not increasing for start/stop events: " + key.seq + " vs " + lastStartToFinish.lastStart);
         if (lastStartToFinish.compactionStatus == Status.INITIAL)
           lastStartToFinish.compactionStatus = Status.LOOKING_FOR_FINISH;
         else if (lastStartToFinish.lastFinish > lastStartToFinish.lastStart)
@@ -248,8 +248,6 @@ public class SortedLogRecovery {
         break;
       if (key.tid != tid)
         break;
-      // log.info("Replaying " + key);
-      // log.info(value);
       if (key.event == MUTATION) {
         mr.receive(value.mutations.get(0));
       } else if (key.event == MANY_MUTATIONS) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b2539fb1/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
index 5c3fc2d..6455726 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
@@ -21,7 +21,6 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
@@ -29,6 +28,7 @@ import java.util.Set;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
@@ -37,7 +37,6 @@ import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.protobuf.ProtobufUtil;
 import org.apache.accumulo.core.replication.ReplicationConfigurationUtil;
-import org.apache.accumulo.core.replication.StatusUtil;
 import org.apache.accumulo.core.replication.proto.Replication.Status;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.server.conf.TableConfiguration;
@@ -49,6 +48,7 @@ import org.apache.accumulo.tserver.TabletMutations;
 import org.apache.accumulo.tserver.TabletServer;
 import org.apache.accumulo.tserver.log.DfsLogger.LoggerOperation;
 import org.apache.accumulo.tserver.tablet.CommitSession;
+import org.apache.accumulo.tserver.tablet.Tablet;
 import org.apache.hadoop.fs.Path;
 import org.apache.log4j.Logger;
 
@@ -71,20 +71,22 @@ public class TabletServerLogger {
 
   private final TabletServer tserver;
 
-  // The current log set: always updated to a new set with every change of loggers
-  private final List<DfsLogger> loggers = new ArrayList<DfsLogger>();
+  // The current logger
+  private DfsLogger currentLog = null;
+  private DfsLogger nextLog = null;
+  private Thread nextLogThread = null;
 
-  // The current generation of logSet.
-  // Because multiple threads can be using a log set at one time, a log
+  // The current generation of logs.
+  // Because multiple threads can be using a log at one time, a log
   // failure is likely to affect multiple threads, who will all attempt to
-  // create a new logSet. This will cause many unnecessary updates to the
+  // create a new log. This will cause many unnecessary updates to the
   // metadata table.
   // We'll use this generational counter to determine if another thread has
-  // already fetched a new logSet.
-  private AtomicInteger logSetId = new AtomicInteger();
+  // already fetched a new log.
+  private final AtomicInteger logId = new AtomicInteger();
 
   // Use a ReadWriteLock to allow multiple threads to use the log set, but obtain a write lock to change them
-  private final ReentrantReadWriteLock logSetLock = new ReentrantReadWriteLock();
+  private final ReentrantReadWriteLock logIdLock = new ReentrantReadWriteLock();
 
   private final AtomicInteger seqGen = new AtomicInteger();
 
@@ -145,61 +147,66 @@ public class TabletServerLogger {
     this.flushCounter = flushCounter;
   }
 
-  private int initializeLoggers(final List<DfsLogger> copy) throws IOException {
-    final int[] result = {-1};
-    testLockAndRun(logSetLock, new TestCallWithWriteLock() {
+  private DfsLogger initializeLoggers(final AtomicInteger logIdOut) throws IOException {
+    final AtomicReference<DfsLogger> result = new AtomicReference<DfsLogger>();
+    testLockAndRun(logIdLock, new TestCallWithWriteLock() {
       @Override
       boolean test() {
-        copy.clear();
-        copy.addAll(loggers);
-        if (!loggers.isEmpty())
-          result[0] = logSetId.get();
-        return loggers.isEmpty();
+        result.set(currentLog);
+        if (currentLog != null)
+          logIdOut.set(logId.get());
+        return currentLog == null;
       }
 
       @Override
       void withWriteLock() throws IOException {
         try {
-          createLoggers();
-          copy.clear();
-          copy.addAll(loggers);
-          if (copy.size() > 0)
-            result[0] = logSetId.get();
+          createLogger();
+          result.set(currentLog);
+          if (currentLog != null)
+            logIdOut.set(logId.get());
           else
-            result[0] = -1;
+            logIdOut.set(-1);
         } catch (IOException e) {
           log.error("Unable to create loggers", e);
         }
       }
     });
-    return result[0];
+    return result.get();
   }
 
-  public void getLogFiles(Set<String> loggersOut) {
-    logSetLock.readLock().lock();
+  public String getLogFile() {
+    logIdLock.readLock().lock();
     try {
-      for (DfsLogger logger : loggers) {
-        loggersOut.add(logger.getFileName());
-      }
+      return currentLog.getFileName();
     } finally {
-      logSetLock.readLock().unlock();
+      logIdLock.readLock().unlock();
     }
   }
 
-  synchronized private void createLoggers() throws IOException {
-    if (!logSetLock.isWriteLockedByCurrentThread()) {
+  synchronized private void createLogger() throws IOException {
+    if (!logIdLock.isWriteLockedByCurrentThread()) {
       throw new IllegalStateException("createLoggers should be called with write lock held!");
     }
 
-    if (loggers.size() != 0) {
-      throw new IllegalStateException("createLoggers should not be called when loggers.size() is " + loggers.size());
+    if (currentLog != null) {
+      throw new IllegalStateException("createLoggers should not be called when current log is set");
     }
 
     try {
-      DfsLogger alog = new DfsLogger(tserver.getServerConfig(), syncCounter, flushCounter);
-      alog.open(tserver.getClientAddressString());
-      loggers.add(alog);
-      logSetId.incrementAndGet();
+      if (nextLog != null) {
+        log.info("Using next log " + nextLog.getFileName());
+        currentLog = nextLog;
+        nextLog = null;
+      } else {
+        DfsLogger alog = new DfsLogger(tserver.getServerConfig(), syncCounter, flushCounter);
+        alog.open(tserver.getClientAddressString());
+        currentLog = alog;
+      }
+      if (nextLog == null) {
+        createNextLog();
+      }
+      logId.incrementAndGet();
       return;
     } catch (Exception t) {
       walErrors.put(System.currentTimeMillis(), "");
@@ -210,30 +217,58 @@ public class TabletServerLogger {
     }
   }
 
+  private void createNextLog() {
+    if (nextLogThread == null) {
+      nextLogThread = new Thread() {
+        @Override
+        public void run() {
+          try {
+            log.info("Creating next WAL");
+            this.setName("Creating next WAL");
+            DfsLogger alog = new DfsLogger(tserver.getServerConfig(), syncCounter, flushCounter);
+            alog.open(tserver.getClientAddressString());
+            for (Tablet tablet : tserver.getOnlineTablets()) {
+              // TODO
+              tserver.addLoggersToMetadata(alog, tablet.getExtent());
+            }
+            nextLog = alog;
+
+            log.info("Created next WAL " + alog.getFileName());
+          } catch (Exception t) {
+            log.error(t, t);
+          } finally {
+            nextLogThread = null;
+          }
+        }
+      };
+      nextLogThread.start();
+    }
+  }
+
   public void resetLoggers() throws IOException {
-    logSetLock.writeLock().lock();
+    logIdLock.writeLock().lock();
     try {
       close();
     } finally {
-      logSetLock.writeLock().unlock();
+      logIdLock.writeLock().unlock();
     }
   }
 
   synchronized private void close() throws IOException {
-    if (!logSetLock.isWriteLockedByCurrentThread()) {
+    if (!logIdLock.isWriteLockedByCurrentThread()) {
       throw new IllegalStateException("close should be called with write lock held!");
     }
     try {
-      for (DfsLogger logger : loggers) {
-        try {
-          logger.close();
-        } catch (DfsLogger.LogClosedException ex) {
-          // ignore
-        } catch (Throwable ex) {
-          log.error("Unable to cleanly close log " + logger.getFileName() + ": " + ex, ex);
-        }
+      try {
+        currentLog.close();
+      } catch (DfsLogger.LogClosedException ex) {
+        // ignore
+      } catch (Throwable ex) {
+        log.error("Unable to cleanly close log " + currentLog.getFileName() + ": " + ex, ex);
+      } finally {
+        this.tserver.walogClosed(currentLog);
       }
-      loggers.clear();
+      currentLog = null;
       logSizeEstimate.set(0);
     } catch (Throwable t) {
       throw new IOException(t);
@@ -251,7 +286,7 @@ public class TabletServerLogger {
 
   private int write(final Collection<CommitSession> sessions, boolean mincFinish, Writer writer) throws IOException {
     // Work very hard not to lock this during calls to the outside world
-    int currentLogSet = logSetId.get();
+    int currentLogId = logId.get();
 
     int seq = -1;
     int attempt = 1;
@@ -259,20 +294,22 @@ public class TabletServerLogger {
     while (!success) {
       try {
         // get a reference to the loggers that no other thread can touch
-        ArrayList<DfsLogger> copy = new ArrayList<DfsLogger>();
-        currentLogSet = initializeLoggers(copy);
+        DfsLogger copy = null;
+        AtomicInteger currentId = new AtomicInteger(-1);
+        copy = initializeLoggers(currentId);
+        currentLogId = currentId.get();
 
         // add the logger to the log set for the memory in the tablet,
         // update the metadata table if we've never used this tablet
 
-        if (currentLogSet == logSetId.get()) {
+        if (currentLogId == logId.get()) {
           for (CommitSession commitSession : sessions) {
             if (commitSession.beginUpdatingLogsUsed(copy, mincFinish)) {
               try {
                 // Scribble out a tablet definition and then write to the metadata table
                 defineTablet(commitSession);
-                if (currentLogSet == logSetId.get())
-                  tserver.addLoggersToMetadata(copy, commitSession.getExtent(), commitSession.getLogId());
+                if (currentLogId == logId.get())
+                  tserver.addLoggersToMetadata(copy, commitSession.getExtent());
               } finally {
                 commitSession.finishUpdatingLogsUsed();
               }
@@ -280,39 +317,29 @@ public class TabletServerLogger {
               // Need to release
               KeyExtent extent = commitSession.getExtent();
               if (ReplicationConfigurationUtil.isEnabled(extent, tserver.getTableConfiguration(extent))) {
-                Set<String> logs = new HashSet<String>();
-                for (DfsLogger logger : copy) {
-                  logs.add(logger.getFileName());
-                }
-                Status status = StatusUtil.fileCreated(System.currentTimeMillis());
-                log.debug("Writing " + ProtobufUtil.toString(status) + " to metadata table for " + logs);
+                Status status = Status.newBuilder().setInfiniteEnd(true).setCreatedTime(System.currentTimeMillis()).build();
+                log.debug("Writing " + ProtobufUtil.toString(status) + " to metadata table for " + copy.getFileName());
                 // Got some new WALs, note this in the metadata table
-                ReplicationTableUtil.updateFiles(tserver, commitSession.getExtent(), logs, status);
+                ReplicationTableUtil.updateFiles(tserver, commitSession.getExtent(), copy.getFileName(), status);
               }
             }
           }
         }
 
         // Make sure that the logs haven't changed out from underneath our copy
-        if (currentLogSet == logSetId.get()) {
+        if (currentLogId == logId.get()) {
 
           // write the mutation to the logs
           seq = seqGen.incrementAndGet();
           if (seq < 0)
             throw new RuntimeException("Logger sequence generator wrapped!  Onos!!!11!eleven");
-          ArrayList<LoggerOperation> queuedOperations = new ArrayList<LoggerOperation>(copy.size());
-          for (DfsLogger wal : copy) {
-            LoggerOperation lop = writer.write(wal, seq);
-            if (lop != null)
-              queuedOperations.add(lop);
-          }
-
-          for (LoggerOperation lop : queuedOperations) {
+          LoggerOperation lop = writer.write(copy, seq);
+          if (lop != null) {
             lop.await();
           }
 
           // double-check: did the log set change?
-          success = (currentLogSet == logSetId.get());
+          success = (currentLogId == logId.get());
         }
       } catch (DfsLogger.LogClosedException ex) {
         log.debug("Logs closed while writing, retrying " + attempt);
@@ -327,13 +354,13 @@ public class TabletServerLogger {
       // Some sort of write failure occurred. Grab the write lock and reset the logs.
       // But since multiple threads will attempt it, only attempt the reset when
       // the logs haven't changed.
-      final int finalCurrent = currentLogSet;
+      final int finalCurrent = currentLogId;
       if (!success) {
-        testLockAndRun(logSetLock, new TestCallWithWriteLock() {
+        testLockAndRun(logIdLock, new TestCallWithWriteLock() {
 
           @Override
           boolean test() {
-            return finalCurrent == logSetId.get();
+            return finalCurrent == logId.get();
           }
 
           @Override
@@ -346,7 +373,7 @@ public class TabletServerLogger {
     }
     // if the log gets too big, reset it .. grab the write lock first
     logSizeEstimate.addAndGet(4 * 3); // event, tid, seq overhead
-    testLockAndRun(logSetLock, new TestCallWithWriteLock() {
+    testLockAndRun(logIdLock, new TestCallWithWriteLock() {
       @Override
       boolean test() {
         return logSizeEstimate.get() > maxSize;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b2539fb1/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CommitSession.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CommitSession.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CommitSession.java
index 17290c0..70b1922 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CommitSession.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CommitSession.java
@@ -16,7 +16,6 @@
  */
 package org.apache.accumulo.tserver.tablet;
 
-import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.accumulo.core.data.KeyExtent;
@@ -85,7 +84,7 @@ public class CommitSession {
     return committer;
   }
 
-  public boolean beginUpdatingLogsUsed(ArrayList<DfsLogger> copy, boolean mincFinish) {
+  public boolean beginUpdatingLogsUsed(DfsLogger copy, boolean mincFinish) {
     return committer.beginUpdatingLogsUsed(memTable, copy, mincFinish);
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b2539fb1/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/DatafileManager.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/DatafileManager.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/DatafileManager.java
index 8ba8128..b05f0c6 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/DatafileManager.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/DatafileManager.java
@@ -424,7 +424,9 @@ class DatafileManager {
         if (log.isDebugEnabled()) {
           log.debug("Recording that data has been ingested into " + tablet.getExtent() + " using " + logFileOnly);
         }
-        ReplicationTableUtil.updateFiles(tablet.getTabletServer(), tablet.getExtent(), logFileOnly, StatusUtil.openWithUnknownLength());
+        for (String logFile : logFileOnly) {
+          ReplicationTableUtil.updateFiles(tablet.getTabletServer(), tablet.getExtent(), logFile, StatusUtil.openWithUnknownLength());
+        }
       }
     } finally {
       tablet.finishClearingUnusedLogs();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b2539fb1/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
index f2d5375..fa1ae86 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
@@ -37,6 +37,7 @@ import java.util.PriorityQueue;
 import java.util.Set;
 import java.util.SortedMap;
 import java.util.TreeMap;
+import java.util.concurrent.ConcurrentSkipListSet;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.AtomicReference;
@@ -198,7 +199,7 @@ public class Tablet implements TabletCommitter {
   }
 
   // stores info about user initiated major compaction that is waiting on a minor compaction to finish
-  private CompactionWaitInfo compactionWaitInfo = new CompactionWaitInfo();
+  private final CompactionWaitInfo compactionWaitInfo = new CompactionWaitInfo();
 
   static enum CompactionState {
     WAITING_TO_START, IN_PROGRESS
@@ -628,8 +629,8 @@ public class Tablet implements TabletCommitter {
           // the WAL isn't closed (WRT replication Status) and thus we're safe to update its progress.
           Status status = StatusUtil.openWithUnknownLength();
           for (LogEntry logEntry : logEntries) {
-            log.debug("Writing updated status to metadata table for " + logEntry.logSet + " " + ProtobufUtil.toString(status));
-            ReplicationTableUtil.updateFiles(tabletServer, extent, logEntry.logSet, status);
+            log.debug("Writing updated status to metadata table for " + logEntry.filename + " " + ProtobufUtil.toString(status));
+            ReplicationTableUtil.updateFiles(tabletServer, extent, logEntry.filename, status);
           }
         }
 
@@ -641,11 +642,9 @@ public class Tablet implements TabletCommitter {
         }
       }
       // make some closed references that represent the recovered logs
-      currentLogs = new HashSet<DfsLogger>();
+      currentLogs = new ConcurrentSkipListSet<DfsLogger>();
       for (LogEntry logEntry : logEntries) {
-        for (String log : logEntry.logSet) {
-          currentLogs.add(new DfsLogger(tabletServer.getServerConfig(), log, logEntry.getColumnQualifier().toString()));
-        }
+        currentLogs.add(new DfsLogger(tabletServer.getServerConfig(), logEntry.filename, logEntry.getColumnQualifier().toString()));
       }
 
       log.info("Write-Ahead Log recovery complete for " + this.extent + " (" + count[0] + " mutations applied, " + getTabletMemory().getNumEntries()
@@ -937,6 +936,8 @@ public class Tablet implements TabletCommitter {
     long count = 0;
 
     try {
+      String oldName = Thread.currentThread().getName();
+      Thread.currentThread().setName("Minor compacting " + this.extent);
       Span span = Trace.start("write");
       CompactionStats stats;
       try {
@@ -957,6 +958,7 @@ public class Tablet implements TabletCommitter {
             commitSession, flushId);
       } finally {
         span.stop();
+        Thread.currentThread().setName(oldName);
       }
       return new DataFileValue(stats.getFileSize(), stats.getEntriesWritten());
     } catch (Exception e) {
@@ -991,7 +993,7 @@ public class Tablet implements TabletCommitter {
   private synchronized MinorCompactionTask prepareForMinC(long flushId, MinorCompactionReason mincReason) {
     CommitSession oldCommitSession = getTabletMemory().prepareForMinC();
     otherLogs = currentLogs;
-    currentLogs = new HashSet<DfsLogger>();
+    currentLogs = new ConcurrentSkipListSet<DfsLogger>();
 
     FileRef mergeFile = null;
     if (mincReason != MinorCompactionReason.RECOVERY) {
@@ -2373,14 +2375,10 @@ public class Tablet implements TabletCommitter {
     }
   }
 
-  private Set<DfsLogger> currentLogs = new HashSet<DfsLogger>();
+  private ConcurrentSkipListSet<DfsLogger> currentLogs = new ConcurrentSkipListSet<DfsLogger>();
 
-  public synchronized Set<String> getCurrentLogFiles() {
-    Set<String> result = new HashSet<String>();
-    for (DfsLogger log : currentLogs) {
-      result.add(log.getFileName());
-    }
-    return result;
+  public Set<DfsLogger> getCurrentLogFiles() {
+    return new HashSet<DfsLogger>(currentLogs);
   }
 
   Set<String> beginClearingUnusedLogs() {
@@ -2439,12 +2437,12 @@ public class Tablet implements TabletCommitter {
   // this lock is basically used to synchronize writing of log info to metadata
   private final ReentrantLock logLock = new ReentrantLock();
 
-  public synchronized int getLogCount() {
+  public int getLogCount() {
     return currentLogs.size();
   }
 
   @Override
-  public boolean beginUpdatingLogsUsed(InMemoryMap memTable, Collection<DfsLogger> more, boolean mincFinish) {
+  public boolean beginUpdatingLogsUsed(InMemoryMap memTable, DfsLogger more, boolean mincFinish) {
 
     boolean releaseLock = true;
 
@@ -2481,28 +2479,26 @@ public class Tablet implements TabletCommitter {
 
         int numAdded = 0;
         int numContained = 0;
-        for (DfsLogger logger : more) {
-          if (addToOther) {
-            if (otherLogs.add(logger))
-              numAdded++;
+        if (addToOther) {
+          if (otherLogs.add(more))
+            numAdded++;
 
-            if (currentLogs.contains(logger))
-              numContained++;
-          } else {
-            if (currentLogs.add(logger))
-              numAdded++;
+          if (currentLogs.contains(more))
+            numContained++;
+        } else {
+          if (currentLogs.add(more))
+            numAdded++;
 
-            if (otherLogs.contains(logger))
-              numContained++;
-          }
+          if (otherLogs.contains(more))
+            numContained++;
         }
 
-        if (numAdded > 0 && numAdded != more.size()) {
+        if (numAdded > 0 && numAdded != 1) {
           // expect to add all or none
           throw new IllegalArgumentException("Added subset of logs " + extent + " " + more + " " + currentLogs);
         }
 
-        if (numContained > 0 && numContained != more.size()) {
+        if (numContained > 0 && numContained != 1) {
           // expect to contain all or none
           throw new IllegalArgumentException("Other logs contained subset of logs " + extent + " " + more + " " + otherLogs);
         }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b2539fb1/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletCommitter.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletCommitter.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletCommitter.java
index b56d0af..39bde5c 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletCommitter.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletCommitter.java
@@ -16,7 +16,6 @@
  */
 package org.apache.accumulo.tserver.tablet;
 
-import java.util.Collection;
 import java.util.List;
 
 import org.apache.accumulo.core.client.Durability;
@@ -35,7 +34,7 @@ public interface TabletCommitter {
 
   void commit(CommitSession commitSession, List<Mutation> mutations);
 
-  boolean beginUpdatingLogsUsed(InMemoryMap memTable, Collection<DfsLogger> copy, boolean mincFinish);
+  boolean beginUpdatingLogsUsed(InMemoryMap memTable, DfsLogger copy, boolean mincFinish);
 
   void finishUpdatingLogsUsed();
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b2539fb1/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java b/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java
index b429607..b8a60c1 100644
--- a/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java
+++ b/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java
@@ -202,9 +202,6 @@ public class NullTserver {
     }
 
     @Override
-    public void removeLogs(TInfo tinfo, TCredentials credentials, List<String> filenames) throws TException {}
-
-    @Override
     public List<ActiveCompaction> getActiveCompactions(TInfo tinfo, TCredentials credentials) throws ThriftSecurityException, TException {
       return new ArrayList<ActiveCompaction>();
     }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b2539fb1/test/src/test/java/org/apache/accumulo/proxy/ProxyDurabilityIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/proxy/ProxyDurabilityIT.java b/test/src/test/java/org/apache/accumulo/proxy/ProxyDurabilityIT.java
index 45799c4..de8ebc8 100644
--- a/test/src/test/java/org/apache/accumulo/proxy/ProxyDurabilityIT.java
+++ b/test/src/test/java/org/apache/accumulo/proxy/ProxyDurabilityIT.java
@@ -58,6 +58,11 @@ import com.google.common.net.HostAndPort;
 public class ProxyDurabilityIT extends ConfigurableMacIT {
 
   @Override
+  protected int defaultTimeoutSeconds() {
+    return 60;
+  }
+
+  @Override
   public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
     hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
     cfg.setProperty(Property.INSTANCE_ZK_TIMEOUT, "5s");

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b2539fb1/test/src/test/java/org/apache/accumulo/test/BadDeleteMarkersCreatedIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/BadDeleteMarkersCreatedIT.java b/test/src/test/java/org/apache/accumulo/test/BadDeleteMarkersCreatedIT.java
index 9cae889..93ca138 100644
--- a/test/src/test/java/org/apache/accumulo/test/BadDeleteMarkersCreatedIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/BadDeleteMarkersCreatedIT.java
@@ -53,7 +53,7 @@ public class BadDeleteMarkersCreatedIT extends AccumuloClusterIT {
 
   @Override
   public int defaultTimeoutSeconds() {
-    return 60;
+    return 120;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b2539fb1/test/src/test/java/org/apache/accumulo/test/BalanceIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/BalanceIT.java b/test/src/test/java/org/apache/accumulo/test/BalanceIT.java
index f793925..8703f18 100644
--- a/test/src/test/java/org/apache/accumulo/test/BalanceIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/BalanceIT.java
@@ -20,25 +20,33 @@ import java.util.SortedSet;
 import java.util.TreeSet;
 
 import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.test.functional.ConfigurableMacIT;
+import org.apache.accumulo.harness.AccumuloClusterIT;
 import org.apache.hadoop.io.Text;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
-public class BalanceIT extends ConfigurableMacIT {
+public class BalanceIT extends AccumuloClusterIT {
+  private static final Logger log = LoggerFactory.getLogger(BalanceIT.class);
 
-  @Test(timeout = 60 * 1000)
+  @Override
+  public int defaultTimeoutSeconds() {
+    return 60;
+  }
+
+  @Test
   public void testBalance() throws Exception {
     String tableName = getUniqueNames(1)[0];
     Connector c = getConnector();
-    System.out.println("Creating table");
+    log.info("Creating table");
     c.tableOperations().create(tableName);
     SortedSet<Text> splits = new TreeSet<Text>();
     for (int i = 0; i < 10; i++) {
       splits.add(new Text("" + i));
     }
-    System.out.println("Adding splits");
+    log.info("Adding splits");
     c.tableOperations().addSplits(tableName, splits);
-    System.out.println("Waiting for balance");
+    log.info("Waiting for balance");
     c.instanceOperations().waitForBalance();
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b2539fb1/test/src/test/java/org/apache/accumulo/test/CleanWalIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/CleanWalIT.java b/test/src/test/java/org/apache/accumulo/test/CleanWalIT.java
index bdfbd13..c113a08 100644
--- a/test/src/test/java/org/apache/accumulo/test/CleanWalIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/CleanWalIT.java
@@ -125,6 +125,7 @@ public class CleanWalIT extends AccumuloClusterIT {
   private int countLogs(String tableName, Connector conn) throws TableNotFoundException {
     Scanner scanner = conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
     scanner.fetchColumnFamily(MetadataSchema.TabletsSection.LogColumnFamily.NAME);
+    scanner.setRange(MetadataSchema.TabletsSection.getRange());
     int count = 0;
     for (Entry<Key,Value> entry : scanner) {
       log.debug("Saw " + entry.getKey() + "=" + entry.getValue());

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b2539fb1/test/src/test/java/org/apache/accumulo/test/ConditionalWriterIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/ConditionalWriterIT.java b/test/src/test/java/org/apache/accumulo/test/ConditionalWriterIT.java
index b68870d..7bd1842 100644
--- a/test/src/test/java/org/apache/accumulo/test/ConditionalWriterIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/ConditionalWriterIT.java
@@ -1251,6 +1251,7 @@ public class ConditionalWriterIT extends AccumuloClusterIT {
     conn.tableOperations().create(tableName);
 
     DistributedTrace.enable("localhost", "testTrace", mac.getClientConfig());
+    UtilWaitThread.sleep(1000);
     Span root = Trace.on("traceTest");
     ConditionalWriter cw = conn.createConditionalWriter(tableName, new ConditionalWriterConfig());
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b2539fb1/test/src/test/java/org/apache/accumulo/test/MissingWalHeaderCompletesRecoveryIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/MissingWalHeaderCompletesRecoveryIT.java b/test/src/test/java/org/apache/accumulo/test/MissingWalHeaderCompletesRecoveryIT.java
index 7f2f6f9..685d71a 100644
--- a/test/src/test/java/org/apache/accumulo/test/MissingWalHeaderCompletesRecoveryIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/MissingWalHeaderCompletesRecoveryIT.java
@@ -17,7 +17,6 @@
 package org.apache.accumulo.test;
 
 import java.io.File;
-import java.util.Collections;
 import java.util.UUID;
 
 import org.apache.accumulo.core.client.BatchWriter;
@@ -25,6 +24,7 @@ import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema;
@@ -125,11 +125,7 @@ public class MissingWalHeaderCompletesRecoveryIT extends ConfigurableMacIT {
     String tableId = conn.tableOperations().tableIdMap().get(tableName);
     Assert.assertNotNull("Table ID was null", tableId);
 
-    LogEntry logEntry = new LogEntry();
-    logEntry.server = "127.0.0.1:12345";
-    logEntry.filename = emptyWalog.toURI().toString();
-    logEntry.tabletId = 10;
-    logEntry.logSet = Collections.singleton(logEntry.filename);
+    LogEntry logEntry = new LogEntry(new KeyExtent(new Text(tableId), null, null), 0, "127.0.0.1:12345", emptyWalog.toURI().toString());
 
     log.info("Taking {} offline", tableName);
     conn.tableOperations().offline(tableName, true);
@@ -184,11 +180,7 @@ public class MissingWalHeaderCompletesRecoveryIT extends ConfigurableMacIT {
     String tableId = conn.tableOperations().tableIdMap().get(tableName);
     Assert.assertNotNull("Table ID was null", tableId);
 
-    LogEntry logEntry = new LogEntry();
-    logEntry.server = "127.0.0.1:12345";
-    logEntry.filename = partialHeaderWalog.toURI().toString();
-    logEntry.tabletId = 10;
-    logEntry.logSet = Collections.singleton(logEntry.filename);
+    LogEntry logEntry = new LogEntry(null, 0, "127.0.0.1:12345", partialHeaderWalog.toURI().toString());
 
     log.info("Taking {} offline", tableName);
     conn.tableOperations().offline(tableName, true);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b2539fb1/test/src/test/java/org/apache/accumulo/test/NoMutationRecoveryIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/NoMutationRecoveryIT.java b/test/src/test/java/org/apache/accumulo/test/NoMutationRecoveryIT.java
deleted file mode 100644
index 10b8810..0000000
--- a/test/src/test/java/org/apache/accumulo/test/NoMutationRecoveryIT.java
+++ /dev/null
@@ -1,132 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.test;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.util.Map.Entry;
-
-import org.apache.accumulo.cluster.ClusterControl;
-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.Scanner;
-import org.apache.accumulo.core.data.Key;
-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.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.RootTable;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.security.TablePermission;
-import org.apache.accumulo.harness.AccumuloClusterIT;
-import org.apache.accumulo.minicluster.ServerType;
-import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
-import org.apache.accumulo.test.functional.FunctionalTestUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.Text;
-import org.junit.Test;
-
-// Verify that a recovery of a log without any mutations removes the log reference
-public class NoMutationRecoveryIT extends AccumuloClusterIT {
-
-  @Override
-  public int defaultTimeoutSeconds() {
-    return 10 * 60;
-  }
-
-  @Override
-  public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
-    cfg.useMiniDFS(true);
-    cfg.setNumTservers(1);
-  }
-
-  public boolean equals(Entry<Key,Value> a, Entry<Key,Value> b) {
-    // comparison, without timestamp
-    Key akey = a.getKey();
-    Key bkey = b.getKey();
-    return akey.compareTo(bkey, PartialKey.ROW_COLFAM_COLQUAL_COLVIS) == 0 && a.getValue().equals(b.getValue());
-  }
-
-  @Test
-  public void test() throws Exception {
-    Connector conn = getConnector();
-    final String table = getUniqueNames(1)[0];
-    conn.tableOperations().create(table);
-    String tableId = conn.tableOperations().tableIdMap().get(table);
-    update(conn, table, new Text("row"), new Text("cf"), new Text("cq"), new Value("value".getBytes()));
-    Entry<Key,Value> logRef = getLogRef(conn, MetadataTable.NAME);
-    conn.tableOperations().flush(table, null, null, true);
-    assertEquals("should not have any refs", 0, FunctionalTestUtils.count(getLogRefs(conn, MetadataTable.NAME, Range.prefix(tableId))));
-    conn.securityOperations().grantTablePermission(conn.whoami(), MetadataTable.NAME, TablePermission.WRITE);
-    update(conn, MetadataTable.NAME, logRef);
-    assertTrue(equals(logRef, getLogRef(conn, MetadataTable.NAME)));
-    conn.tableOperations().flush(MetadataTable.NAME, null, null, true);
-    conn.tableOperations().flush(RootTable.NAME, null, null, true);
-
-    ClusterControl control = cluster.getClusterControl();
-    control.stopAllServers(ServerType.TABLET_SERVER);
-    control.startAllServers(ServerType.TABLET_SERVER);
-
-    Scanner s = conn.createScanner(table, Authorizations.EMPTY);
-    int count = 0;
-    for (Entry<Key,Value> e : s) {
-      assertEquals(e.getKey().getRow().toString(), "row");
-      assertEquals(e.getKey().getColumnFamily().toString(), "cf");
-      assertEquals(e.getKey().getColumnQualifier().toString(), "cq");
-      assertEquals(e.getValue().toString(), "value");
-      count++;
-    }
-    assertEquals(1, count);
-    for (Entry<Key,Value> ref : getLogRefs(conn, MetadataTable.NAME)) {
-      assertFalse(equals(ref, logRef));
-    }
-  }
-
-  private void update(Connector conn, String name, Entry<Key,Value> logRef) throws Exception {
-    Key k = logRef.getKey();
-    update(conn, name, k.getRow(), k.getColumnFamily(), k.getColumnQualifier(), logRef.getValue());
-  }
-
-  private Iterable<Entry<Key,Value>> getLogRefs(Connector conn, String table) throws Exception {
-    return getLogRefs(conn, table, new Range());
-  }
-
-  private Iterable<Entry<Key,Value>> getLogRefs(Connector conn, String table, Range r) throws Exception {
-    Scanner s = conn.createScanner(table, Authorizations.EMPTY);
-    s.fetchColumnFamily(MetadataSchema.TabletsSection.LogColumnFamily.NAME);
-    s.setRange(r);
-    return s;
-  }
-
-  private Entry<Key,Value> getLogRef(Connector conn, String table) throws Exception {
-    return getLogRefs(conn, table).iterator().next();
-  }
-
-  private void update(Connector conn, String table, Text row, Text cf, Text cq, Value value) throws Exception {
-    BatchWriter bw = conn.createBatchWriter(table, new BatchWriterConfig());
-    Mutation m = new Mutation(row);
-    m.put(cf, cq, value);
-    bw.addMutation(m);
-    bw.close();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b2539fb1/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java b/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
index 72f098f..21d1115 100644
--- a/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
@@ -299,7 +299,7 @@ public class ShellServerIT extends SharedMiniClusterIT {
     ts.exec("config -t " + table2 + " -np", true, "345M", true);
     ts.exec("getsplits -t " + table2, true, "row5", true);
     ts.exec("constraint --list -t " + table2, true, "VisibilityConstraint=2", true);
-    ts.exec("onlinetable " + table, true);
+    ts.exec("online " + table, true);
     ts.exec("deletetable -f " + table, true);
     ts.exec("deletetable -f " + table2, true);
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b2539fb1/test/src/test/java/org/apache/accumulo/test/functional/ExamplesIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/ExamplesIT.java b/test/src/test/java/org/apache/accumulo/test/functional/ExamplesIT.java
index 6c81369..d83f038 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/ExamplesIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/ExamplesIT.java
@@ -201,10 +201,10 @@ public class ExamplesIT extends AccumuloClusterIT {
     Entry<Integer,String> entry = getClusterControl().execWithStdout(
         Ingest.class,
         new String[] {"-i", instance, "-z", keepers, "-u", user, "-p", passwd, "--dirTable", dirTable, "--indexTable", indexTable, "--dataTable", dataTable,
-            "--vis", visibility, "--chunkSize", Integer.toString(10000), getUsableDir()});
+            "--vis", visibility, "--chunkSize", Integer.toString(10000), System.getProperty("user.dir") + "/src/test"});
     assertEquals("Got non-zero return code. Stdout=" + entry.getValue(), 0, entry.getKey().intValue());
     entry = getClusterControl().execWithStdout(QueryUtil.class,
-        new String[] {"-i", instance, "-z", keepers, "-p", passwd, "-u", user, "-t", indexTable, "--auths", auths, "--search", "--path", "accumulo-site.xml"});
+        new String[] {"-i", instance, "-z", keepers, "-p", passwd, "-u", user, "-t", indexTable, "--auths", auths, "--search", "--path", "log4j.properties"});
     if (ClusterType.MINI == getClusterType()) {
       MiniAccumuloClusterImpl impl = (MiniAccumuloClusterImpl) cluster;
       for (LogWriter writer : impl.getLogWriters()) {
@@ -214,7 +214,7 @@ public class ExamplesIT extends AccumuloClusterIT {
 
     log.info("result " + entry.getValue());
     assertEquals(0, entry.getKey().intValue());
-    assertTrue(entry.getValue().contains("accumulo-site.xml"));
+    assertTrue(entry.getValue().contains("log4j.properties"));
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b2539fb1/test/src/test/java/org/apache/accumulo/test/functional/WatchTheWatchCountIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/WatchTheWatchCountIT.java b/test/src/test/java/org/apache/accumulo/test/functional/WatchTheWatchCountIT.java
index bd0555b..4c86172 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/WatchTheWatchCountIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/WatchTheWatchCountIT.java
@@ -51,8 +51,8 @@ public class WatchTheWatchCountIT extends ConfigurableMacIT {
       int n = socket.getInputStream().read(buffer);
       String response = new String(buffer, 0, n);
       long total = Long.parseLong(response.split(":")[1].trim());
-      assertTrue("Total watches was not greater than 500, but was " + total, total > 500);
-      assertTrue("Total watches was not less than 600, but was " + total, total < 600);
+      assertTrue("Total watches was not greater than 600, but was " + total, total > 600);
+      assertTrue("Total watches was not less than 600, but was " + total, total < 675);
     } finally {
       socket.close();
     }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b2539fb1/test/src/test/java/org/apache/accumulo/test/performance/RollWALPerformanceIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/performance/RollWALPerformanceIT.java b/test/src/test/java/org/apache/accumulo/test/performance/RollWALPerformanceIT.java
new file mode 100644
index 0000000..fcd1fd7
--- /dev/null
+++ b/test/src/test/java/org/apache/accumulo/test/performance/RollWALPerformanceIT.java
@@ -0,0 +1,126 @@
+/*
+ * 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.test.performance;
+
+import static org.junit.Assert.assertTrue;
+
+import java.util.SortedSet;
+import java.util.TreeSet;
+
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.Instance;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.RootTable;
+import org.apache.accumulo.minicluster.ServerType;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.minicluster.impl.ProcessReference;
+import org.apache.accumulo.test.continuous.ContinuousIngest;
+import org.apache.accumulo.test.functional.ConfigurableMacIT;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Text;
+import org.junit.Test;
+
+public class RollWALPerformanceIT extends ConfigurableMacIT {
+
+  @Override
+  protected void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
+    cfg.setProperty(Property.TSERV_WAL_REPLICATION, "1");
+    cfg.setProperty(Property.TSERV_WALOG_MAX_SIZE, "10M");
+    cfg.setProperty(Property.TABLE_MINC_LOGS_MAX, "100");
+    cfg.setProperty(Property.GC_FILE_ARCHIVE, "false");
+    cfg.setProperty(Property.GC_CYCLE_START, "1s");
+    cfg.setProperty(Property.GC_CYCLE_DELAY, "1s");
+    cfg.useMiniDFS(true);
+  }
+
+  private long ingest() throws Exception {
+    final Connector c = getConnector();
+    final String tableName = getUniqueNames(1)[0];
+
+    log.info("Creating the table");
+    c.tableOperations().create(tableName);
+
+    log.info("Splitting the table");
+    final long SPLIT_COUNT = 100;
+    final long distance = Long.MAX_VALUE / SPLIT_COUNT;
+    final SortedSet<Text> splits = new TreeSet<Text>();
+    for (int i = 1; i < SPLIT_COUNT; i++) {
+      splits.add(new Text(String.format("%016x", i * distance)));
+    }
+    c.tableOperations().addSplits(tableName, splits);
+
+    log.info("Waiting for balance");
+    c.instanceOperations().waitForBalance();
+
+    final Instance inst = c.getInstance();
+
+    log.info("Starting ingest");
+    final long start = System.currentTimeMillis();
+    final String args[] = {
+        "-i", inst.getInstanceName(),
+        "-z", inst.getZooKeepers(),
+        "-u", "root",
+        "-p", ROOT_PASSWORD,
+        "--batchThreads", "2",
+        "--table", tableName,
+        "--num", Long.toString(1000*1000),  // 1M 100 byte entries
+    };
+
+    ContinuousIngest.main(args);
+    final long result = System.currentTimeMillis() - start;
+    log.debug(String.format("Finished in %,d ms", result));
+    log.debug("Dropping table");
+    c.tableOperations().delete(tableName);
+    return result;
+  }
+
+  private long getAverage() throws Exception {
+    final int REPEAT = 3;
+    long totalTime = 0;
+    for (int i = 0; i < REPEAT; i++) {
+      totalTime += ingest();
+    }
+    return totalTime / REPEAT;
+  }
+
+  private void testWalPerformanceOnce() throws Exception {
+    // get time with a small WAL, which will cause many WAL roll-overs
+    long avg1 = getAverage();
+    // use a bigger WAL max size to eliminate WAL roll-overs
+    Connector c = getConnector();
+    c.instanceOperations().setProperty(Property.TSERV_WALOG_MAX_SIZE.getKey(), "1G");
+    c.tableOperations().flush(MetadataTable.NAME, null, null, true);
+    c.tableOperations().flush(RootTable.NAME, null, null, true);
+    for (ProcessReference  tserver : getCluster().getProcesses().get(ServerType.TABLET_SERVER)) {
+      getCluster().killProcess(ServerType.TABLET_SERVER, tserver);
+    }
+    getCluster().start();
+    long avg2 = getAverage();
+    log.info(String.format("Average run time with small WAL %,d with large WAL %,d", avg1, avg2));
+    assertTrue(avg1 > avg2);
+    double percent = (100. * avg1) / avg2;
+    log.info(String.format("Percent of large log: %.2f%%", percent));
+    assertTrue(percent < 125.);
+  }
+
+  @Test(timeout= 20 * 60 * 1000)
+  public void testWalPerformance() throws Exception {
+    testWalPerformanceOnce();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b2539fb1/test/src/test/java/org/apache/accumulo/test/replication/GarbageCollectorCommunicatesWithTServersIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/replication/GarbageCollectorCommunicatesWithTServersIT.java b/test/src/test/java/org/apache/accumulo/test/replication/GarbageCollectorCommunicatesWithTServersIT.java
index 1ef47e5..9af5445 100644
--- a/test/src/test/java/org/apache/accumulo/test/replication/GarbageCollectorCommunicatesWithTServersIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/replication/GarbageCollectorCommunicatesWithTServersIT.java
@@ -38,6 +38,7 @@ import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.master.thrift.MasterClientService;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.CurrentLogsSection;
 import org.apache.accumulo.core.protobuf.ProtobufUtil;
 import org.apache.accumulo.core.replication.ReplicationTable;
 import org.apache.accumulo.core.replication.proto.Replication.Status;
@@ -78,6 +79,7 @@ public class GarbageCollectorCommunicatesWithTServersIT extends ConfigurableMacI
   @Override
   public void configure(MiniAccumuloConfigImpl cfg, Configuration coreSite) {
     cfg.setNumTservers(1);
+    cfg.setProperty(Property.INSTANCE_ZK_TIMEOUT, "5s");
     cfg.setProperty(Property.GC_CYCLE_DELAY, GC_PERIOD_SECONDS + "s");
     // Wait longer to try to let the replication table come online before a cycle runs
     cfg.setProperty(Property.GC_CYCLE_START, "10s");
@@ -102,18 +104,14 @@ public class GarbageCollectorCommunicatesWithTServersIT extends ConfigurableMacI
     Assert.assertNotNull("Could not determine table ID for " + tableName, tableId);
 
     Scanner s = conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
-    Range r = MetadataSchema.TabletsSection.getRange(tableId);
-    s.setRange(r);
-    s.fetchColumnFamily(MetadataSchema.TabletsSection.LogColumnFamily.NAME);
+    s.setRange(CurrentLogsSection.getRange());
+    s.fetchColumnFamily(CurrentLogsSection.COLF);
 
     Set<String> wals = new HashSet<String>();
     for (Entry<Key,Value> entry : s) {
       log.debug("Reading WALs: {}={}", entry.getKey().toStringNoTruncate(), entry.getValue());
       // hostname:port/uri://path/to/wal
-      String cq = entry.getKey().getColumnQualifier().toString();
-      int index = cq.indexOf('/');
-      // Normalize the path
-      String path = new Path(cq.substring(index + 1)).toString();
+      String path = new Path(entry.getKey().getColumnQualifier().toString()).toString();
       log.debug("Extracted file: " + path);
       wals.add(path);
     }
@@ -228,11 +226,6 @@ public class GarbageCollectorCommunicatesWithTServersIT extends ConfigurableMacI
 
     Assert.assertEquals("Expected Status for file to not be closed", false, status.getClosed());
 
-    log.info("Checking to see that log entries are removed from tablet section after MinC");
-    // After compaction, the log column should be gone from the tablet
-    Set<String> walsAfterMinc = getWalsForTable(table);
-    Assert.assertEquals("Expected to find no WALs for tablet", 0, walsAfterMinc.size());
-
     Set<String> filesForTable = getFilesForTable(table);
     Assert.assertEquals("Expected to only find one rfile for table", 1, filesForTable.size());
     log.info("Files for table before MajC: {}", filesForTable);
@@ -258,14 +251,6 @@ public class GarbageCollectorCommunicatesWithTServersIT extends ConfigurableMacI
       fileExists = fs.exists(fileToBeDeleted);
     }
 
-    // At this point in time, we *know* that the GarbageCollector has run which means that the Status
-    // for our WAL should not be altered.
-
-    log.info("Re-checking that WALs are still not referenced for our table");
-
-    Set<String> walsAfterMajc = getWalsForTable(table);
-    Assert.assertEquals("Expected to find no WALs in tablets section: " + walsAfterMajc, 0, walsAfterMajc.size());
-
     Map<String,Status> fileToStatusAfterMinc = getMetadataStatusForTable(table);
     Assert.assertEquals("Expected to still find only one replication status message: " + fileToStatusAfterMinc, 1, fileToStatusAfterMinc.size());
 
@@ -326,11 +311,6 @@ public class GarbageCollectorCommunicatesWithTServersIT extends ConfigurableMacI
 
     Assert.assertEquals("Expected Status for file to not be closed", false, status.getClosed());
 
-    log.info("Checking to see that log entries are removed from tablet section after MinC");
-    // After compaction, the log column should be gone from the tablet
-    Set<String> walsAfterMinc = getWalsForTable(table);
-    Assert.assertEquals("Expected to find no WALs for tablet", 0, walsAfterMinc.size());
-
     Set<String> filesForTable = getFilesForTable(table);
     Assert.assertEquals("Expected to only find one rfile for table", 1, filesForTable.size());
     log.info("Files for table before MajC: {}", filesForTable);
@@ -359,11 +339,6 @@ public class GarbageCollectorCommunicatesWithTServersIT extends ConfigurableMacI
     // At this point in time, we *know* that the GarbageCollector has run which means that the Status
     // for our WAL should not be altered.
 
-    log.info("Re-checking that WALs are still not referenced for our table");
-
-    Set<String> walsAfterMajc = getWalsForTable(table);
-    Assert.assertEquals("Expected to find no WALs in tablets section: " + walsAfterMajc, 0, walsAfterMajc.size());
-
     Map<String,Status> fileToStatusAfterMinc = getMetadataStatusForTable(table);
     Assert.assertEquals("Expected to still find only one replication status message: " + fileToStatusAfterMinc, 1, fileToStatusAfterMinc.size());
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b2539fb1/test/src/test/java/org/apache/accumulo/test/replication/MultiInstanceReplicationIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/replication/MultiInstanceReplicationIT.java b/test/src/test/java/org/apache/accumulo/test/replication/MultiInstanceReplicationIT.java
index 125286f..14812c4 100644
--- a/test/src/test/java/org/apache/accumulo/test/replication/MultiInstanceReplicationIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/replication/MultiInstanceReplicationIT.java
@@ -146,7 +146,7 @@ public class MultiInstanceReplicationIT extends ConfigurableMacIT {
     }
   }
 
-  @Test
+  @Test(timeout = 10 * 60 * 1000)
   public void dataWasReplicatedToThePeer() throws Exception {
     MiniAccumuloConfigImpl peerCfg = new MiniAccumuloConfigImpl(createTestDir(this.getClass().getName() + "_" + this.testName.getMethodName() + "_peer"),
         ROOT_PASSWORD);


[34/34] accumulo git commit: Merge branch '1.7'

Posted by ec...@apache.org.
Merge branch '1.7'


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

Branch: refs/heads/master
Commit: 51f39d292eb0aec15007fdd49fbf709792a3a2a7
Parents: d0a0ac0 55981ad
Author: Eric C. Newton <er...@gmail.com>
Authored: Fri Apr 24 19:06:05 2015 -0400
Committer: Eric C. Newton <er...@gmail.com>
Committed: Fri Apr 24 19:06:05 2015 -0400

----------------------------------------------------------------------
 .../base/src/main/java/org/apache/accumulo/server/TabletLevel.java | 2 +-
 .../accumulo/gc/replication/CloseWriteAheadLogReferencesTest.java  | 1 -
 .../test/java/org/apache/accumulo/tserver/log/LogEntryTest.java    | 2 +-
 .../apache/accumulo/test/MissingWalHeaderCompletesRecoveryIT.java  | 2 +-
 .../java/org/apache/accumulo/test/functional/WALSunnyDayIT.java    | 2 +-
 .../java/org/apache/accumulo/test/replication/ReplicationIT.java   | 2 +-
 6 files changed, 5 insertions(+), 6 deletions(-)
----------------------------------------------------------------------



[28/34] accumulo git commit: ACCUMULO-3423 optimize WAL metadata table updates

Posted by ec...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
index afd3454..aeb73b4 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
@@ -19,7 +19,6 @@ package org.apache.accumulo.tserver;
 import static java.nio.charset.StandardCharsets.UTF_8;
 import static org.apache.accumulo.server.problems.ProblemType.TABLET_LOAD;
 
-import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.lang.management.ManagementFactory;
 import java.net.UnknownHostException;
@@ -30,6 +29,7 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
+import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
@@ -45,6 +45,7 @@ import java.util.TreeMap;
 import java.util.TreeSet;
 import java.util.concurrent.BlockingDeque;
 import java.util.concurrent.CancellationException;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.LinkedBlockingDeque;
 import java.util.concurrent.ThreadPoolExecutor;
@@ -147,8 +148,8 @@ import org.apache.accumulo.fate.zookeeper.ZooUtil.NodeExistsPolicy;
 import org.apache.accumulo.server.Accumulo;
 import org.apache.accumulo.server.AccumuloServerContext;
 import org.apache.accumulo.server.GarbageCollectionLogger;
-import org.apache.accumulo.server.ServerConstants;
 import org.apache.accumulo.server.ServerOpts;
+import org.apache.accumulo.server.TabletLevel;
 import org.apache.accumulo.server.client.ClientServiceHandler;
 import org.apache.accumulo.server.client.HdfsZooInstance;
 import org.apache.accumulo.server.conf.ServerConfigurationFactory;
@@ -1440,6 +1441,7 @@ public class TabletServer extends AccumuloServerContext implements Runnable {
       }
     }
 
+
     @Override
     public void loadTablet(TInfo tinfo, TCredentials credentials, String lock, final TKeyExtent textent) {
 
@@ -1500,6 +1502,7 @@ public class TabletServer extends AccumuloServerContext implements Runnable {
       final AssignmentHandler ah = new AssignmentHandler(extent);
       // final Runnable ah = new LoggingRunnable(log, );
       // Root tablet assignment must take place immediately
+
       if (extent.isRootTablet()) {
         new Daemon("Root Tablet Assignment") {
           @Override
@@ -1692,66 +1695,6 @@ public class TabletServer extends AccumuloServerContext implements Runnable {
     }
 
     @Override
-    public void removeLogs(TInfo tinfo, TCredentials credentials, List<String> filenames) throws TException {
-      String myname = getClientAddressString();
-      myname = myname.replace(':', '+');
-      Set<String> loggers = new HashSet<String>();
-      logger.getLogFiles(loggers);
-      Set<String> loggerUUIDs = new HashSet<String>();
-      for (String logger : loggers)
-        loggerUUIDs.add(new Path(logger).getName());
-
-      nextFile: for (String filename : filenames) {
-        String uuid = new Path(filename).getName();
-        // skip any log we're currently using
-        if (loggerUUIDs.contains(uuid))
-          continue nextFile;
-
-        List<Tablet> onlineTabletsCopy = new ArrayList<Tablet>();
-        synchronized (onlineTablets) {
-          onlineTabletsCopy.addAll(onlineTablets.values());
-        }
-        for (Tablet tablet : onlineTabletsCopy) {
-          for (String current : tablet.getCurrentLogFiles()) {
-            if (current.contains(uuid)) {
-              log.info("Attempted to delete " + filename + " from tablet " + tablet.getExtent());
-              continue nextFile;
-            }
-          }
-        }
-
-        try {
-          Path source = new Path(filename);
-          if (TabletServer.this.getConfiguration().getBoolean(Property.TSERV_ARCHIVE_WALOGS)) {
-            Path walogArchive = fs.matchingFileSystem(source, ServerConstants.getWalogArchives());
-            fs.mkdirs(walogArchive);
-            Path dest = new Path(walogArchive, source.getName());
-            log.info("Archiving walog " + source + " to " + dest);
-            if (!fs.rename(source, dest))
-              log.error("rename is unsuccessful");
-          } else {
-            log.info("Deleting walog " + filename);
-            Path sourcePath = new Path(filename);
-            if (!(!TabletServer.this.getConfiguration().getBoolean(Property.GC_TRASH_IGNORE) && fs.moveToTrash(sourcePath))
-                && !fs.deleteRecursively(sourcePath))
-              log.warn("Failed to delete walog " + source);
-            for (String recovery : ServerConstants.getRecoveryDirs()) {
-              Path recoveryPath = new Path(recovery, source.getName());
-              try {
-                if (fs.moveToTrash(recoveryPath) || fs.deleteRecursively(recoveryPath))
-                  log.info("Deleted any recovery log " + filename);
-              } catch (FileNotFoundException ex) {
-                // ignore
-              }
-            }
-          }
-        } catch (IOException e) {
-          log.warn("Error attempting to delete write-ahead log " + filename + ": " + e);
-        }
-      }
-    }
-
-    @Override
     public List<ActiveCompaction> getActiveCompactions(TInfo tinfo, TCredentials credentials) throws ThriftSecurityException, TException {
       try {
         checkPermission(credentials, null, "getActiveCompactions");
@@ -1772,14 +1715,20 @@ public class TabletServer extends AccumuloServerContext implements Runnable {
 
     @Override
     public List<String> getActiveLogs(TInfo tinfo, TCredentials credentials) throws TException {
-      Set<String> logs = new HashSet<String>();
-      logger.getLogFiles(logs);
-      return new ArrayList<String>(logs);
+      String log = logger.getLogFile();
+      return Collections.singletonList(log);
+    }
+
+    @Override
+    public void removeLogs(TInfo tinfo, TCredentials credentials, List<String> filenames) throws TException {
+      log.warn("Garbage collector is attempting to remove logs through the tablet server");
+      log.warn("This is probably because your file Garbage Collector is an older version than your tablet servers.\n" +
+          "Restart your file Garbage Collector.");
     }
   }
 
   private class SplitRunner implements Runnable {
-    private Tablet tablet;
+    private final Tablet tablet;
 
     public SplitRunner(Tablet tablet) {
       this.tablet = tablet;
@@ -2033,7 +1982,7 @@ public class TabletServer extends AccumuloServerContext implements Runnable {
           log.error("Unexpected error ", e);
         }
         log.debug("Unassigning " + tls);
-        TabletStateStore.unassign(TabletServer.this, tls);
+        TabletStateStore.unassign(TabletServer.this, tls, null);
       } catch (DistributedStoreException ex) {
         log.warn("Unable to update storage", ex);
       } catch (KeeperException e) {
@@ -2243,29 +2192,6 @@ public class TabletServer extends AccumuloServerContext implements Runnable {
     }
   }
 
-  public void addLoggersToMetadata(List<DfsLogger> logs, KeyExtent extent, int id) {
-    if (!this.onlineTablets.containsKey(extent)) {
-      log.info("Not adding " + logs.size() + " logs for extent " + extent + " as alias " + id + " tablet is offline");
-      // minor compaction due to recovery... don't make updates... if it finishes, there will be no WALs,
-      // if it doesn't, we'll need to do the same recovery with the old files.
-      return;
-    }
-
-    log.info("Adding " + logs.size() + " logs for extent " + extent + " as alias " + id);
-    long now = RelativeTime.currentTimeMillis();
-    List<String> logSet = new ArrayList<String>();
-    for (DfsLogger log : logs)
-      logSet.add(log.getFileName());
-    LogEntry entry = new LogEntry();
-    entry.extent = extent;
-    entry.tabletId = id;
-    entry.timestamp = now;
-    entry.server = logs.get(0).getLogger();
-    entry.filename = logs.get(0).getFileName();
-    entry.logSet = logSet;
-    MetadataTableUtil.addLogEntry(this, entry, getLock());
-  }
-
   private HostAndPort startServer(AccumuloConfiguration conf, String address, Property portHint, TProcessor processor, String threadName)
       throws UnknownHostException {
     Property maxMessageSizeProperty = (conf.get(Property.TSERV_MAX_MESSAGE_SIZE) != null ? Property.TSERV_MAX_MESSAGE_SIZE : Property.GENERAL_MAX_MESSAGE_SIZE);
@@ -2984,6 +2910,7 @@ public class TabletServer extends AccumuloServerContext implements Runnable {
   public void minorCompactionFinished(CommitSession tablet, String newDatafile, int walogSeq) throws IOException {
     totalMinorCompactions.incrementAndGet();
     logger.minorCompactionFinished(tablet, newDatafile, walogSeq);
+    markUnusedWALs();
   }
 
   public void minorCompactionStarted(CommitSession tablet, int lastUpdateSequence, String newMapfileLocation) throws IOException {
@@ -3002,14 +2929,11 @@ public class TabletServer extends AccumuloServerContext implements Runnable {
     });
     for (LogEntry entry : sorted) {
       Path recovery = null;
-      for (String log : entry.logSet) {
-        Path finished = RecoveryPath.getRecoveryPath(fs, fs.getFullPath(FileType.WAL, log));
-        finished = SortedLogState.getFinishedMarkerPath(finished);
-        TabletServer.log.info("Looking for " + finished);
-        if (fs.exists(finished)) {
-          recovery = finished.getParent();
-          break;
-        }
+      Path finished = RecoveryPath.getRecoveryPath(fs, fs.getFullPath(FileType.WAL, entry.filename));
+      finished = SortedLogState.getFinishedMarkerPath(finished);
+      TabletServer.log.info("Looking for " + finished);
+      if (fs.exists(finished)) {
+        recovery = finished.getParent();
       }
       if (recovery == null)
         throw new IOException("Unable to find recovery files for extent " + extent + " logEntry: " + entry);
@@ -3046,7 +2970,9 @@ public class TabletServer extends AccumuloServerContext implements Runnable {
   }
 
   public Collection<Tablet> getOnlineTablets() {
-    return Collections.unmodifiableCollection(onlineTablets.values());
+    synchronized (onlineTablets) {
+      return new ArrayList<Tablet>(onlineTablets.values());
+    }
   }
 
   public VolumeManager getFileSystem() {
@@ -3072,4 +2998,62 @@ public class TabletServer extends AccumuloServerContext implements Runnable {
   public SecurityOperation getSecurityOperation() {
     return security;
   }
+
+  // avoid unnecessary redundant markings to meta
+  final ConcurrentHashMap<DfsLogger, EnumSet<TabletLevel>> metadataTableLogs = new ConcurrentHashMap<>();
+  final Object levelLocks[] = new Object[TabletLevel.values().length];
+  {
+    for (int i = 0; i < levelLocks.length; i++) {
+      levelLocks[i] = new Object();
+    }
+  }
+
+
+  // remove any meta entries after a rolled log is no longer referenced
+  Set<DfsLogger> closedLogs = new HashSet<>();
+
+  private void markUnusedWALs() {
+    Set<DfsLogger> candidates;
+    synchronized (closedLogs) {
+      candidates = new HashSet<>(closedLogs);
+    }
+    for (Tablet tablet : getOnlineTablets()) {
+      candidates.removeAll(tablet.getCurrentLogFiles());
+    }
+    try {
+      Set<Path> filenames = new HashSet<>();
+      for (DfsLogger candidate : candidates) {
+        filenames.add(candidate.getPath());
+      }
+      MetadataTableUtil.markLogUnused(this, this.getLock(), this.getTabletSession(), filenames);
+      synchronized (closedLogs) {
+        closedLogs.removeAll(candidates);
+      }
+    } catch (AccumuloException ex) {
+      log.info(ex.toString(), ex);
+    }
+  }
+
+  public void addLoggersToMetadata(DfsLogger copy, TabletLevel level) {
+    // serialize the updates to the metadata per level: avoids updating the level more than once
+    // updating one level, may cause updates to other levels, so we need to release the lock on metadataTableLogs
+    synchronized (levelLocks[level.ordinal()]) {
+      EnumSet<TabletLevel> set = null;
+      set = metadataTableLogs.putIfAbsent(copy, EnumSet.of(level));
+      if (set == null || !set.contains(level) || level == TabletLevel.ROOT) {
+        log.info("Writing log marker for level " + level + " " + copy.getFileName());
+        MetadataTableUtil.addNewLogMarker(this, this.getLock(), this.getTabletSession(), copy.getPath(), level);
+      }
+      set = metadataTableLogs.get(copy);
+      set.add(level);
+    }
+  }
+
+  public void walogClosed(DfsLogger currentLog) {
+    metadataTableLogs.remove(currentLog);
+    synchronized (closedLogs) {
+      closedLogs.add(currentLog);
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
index 8512690..cd7ce08 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
@@ -72,7 +72,7 @@ import com.google.common.base.Optional;
  * Wrap a connection to a logger.
  *
  */
-public class DfsLogger {
+public class DfsLogger implements Comparable<DfsLogger> {
   public static final String LOG_FILE_HEADER_V2 = "--- Log File Header (v2) ---";
   public static final String LOG_FILE_HEADER_V3 = "--- Log File Header (v3) ---";
 
@@ -371,6 +371,7 @@ public class DfsLogger {
 
   public synchronized void open(String address) throws IOException {
     String filename = UUID.randomUUID().toString();
+    log.debug("Address is " + address);
     String logger = Joiner.on("+").join(address.split(":"));
 
     log.debug("DfsLogger.open() begin");
@@ -463,7 +464,11 @@ public class DfsLogger {
   }
 
   public String getFileName() {
-    return logPath.toString();
+    return logPath;
+  }
+
+  public Path getPath() {
+    return new Path(logPath);
   }
 
   public void close() throws IOException {
@@ -609,4 +614,9 @@ public class DfsLogger {
     return Joiner.on(":").join(parts[parts.length - 2].split("[+]"));
   }
 
+  @Override
+  public int compareTo(DfsLogger o) {
+    return getFileName().compareTo(o.getFileName());
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/server/tserver/src/main/java/org/apache/accumulo/tserver/log/SortedLogRecovery.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/SortedLogRecovery.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/SortedLogRecovery.java
index 37882cd..ab3dea2 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/SortedLogRecovery.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/SortedLogRecovery.java
@@ -180,7 +180,7 @@ public class SortedLogRecovery {
     // find the maximum tablet id... because a tablet may leave a tserver and then come back, in which case it would have a different tablet id
     // for the maximum tablet id, find the minimum sequence #... may be ok to find the max seq, but just want to make the code behave like it used to
     while (reader.next(key, value)) {
-      // LogReader.printEntry(entry);
+      // log.debug("Event " + key.event + " tablet " + key.tablet);
       if (key.event != DEFINE_TABLET)
         break;
       if (key.tablet.equals(extent) || key.tablet.equals(alternative)) {
@@ -209,7 +209,7 @@ public class SortedLogRecovery {
         if (lastStartToFinish.compactionStatus == Status.INITIAL)
           lastStartToFinish.compactionStatus = Status.COMPLETE;
         if (key.seq <= lastStartToFinish.lastStart)
-          throw new RuntimeException("Sequence numbers are not increasing for start/stop events.");
+          throw new RuntimeException("Sequence numbers are not increasing for start/stop events: " + key.seq + " vs " + lastStartToFinish.lastStart);
         lastStartToFinish.update(fileno, key.seq);
 
         // Tablet server finished the minor compaction, but didn't remove the entry from the METADATA table.
@@ -218,7 +218,7 @@ public class SortedLogRecovery {
           lastStartToFinish.update(-1);
       } else if (key.event == COMPACTION_FINISH) {
         if (key.seq <= lastStartToFinish.lastStart)
-          throw new RuntimeException("Sequence numbers are not increasing for start/stop events.");
+          throw new RuntimeException("Sequence numbers are not increasing for start/stop events: " + key.seq + " vs " + lastStartToFinish.lastStart);
         if (lastStartToFinish.compactionStatus == Status.INITIAL)
           lastStartToFinish.compactionStatus = Status.LOOKING_FOR_FINISH;
         else if (lastStartToFinish.lastFinish > lastStartToFinish.lastStart)
@@ -249,8 +249,6 @@ public class SortedLogRecovery {
         break;
       if (key.tid != tid)
         break;
-      // log.info("Replaying " + key);
-      // log.info(value);
       if (key.event == MUTATION) {
         mr.receive(value.mutations.get(0));
       } else if (key.event == MANY_MUTATIONS) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
index 1d385d9..bc77ffb 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
@@ -21,14 +21,16 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
@@ -37,7 +39,9 @@ import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.impl.KeyExtent;
 import org.apache.accumulo.core.protobuf.ProtobufUtil;
 import org.apache.accumulo.core.replication.ReplicationConfigurationUtil;
+import org.apache.accumulo.core.util.SimpleThreadPool;
 import org.apache.accumulo.core.util.UtilWaitThread;
+import org.apache.accumulo.server.TabletLevel;
 import org.apache.accumulo.server.conf.TableConfiguration;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.replication.StatusUtil;
@@ -72,20 +76,22 @@ public class TabletServerLogger {
 
   private final TabletServer tserver;
 
-  // The current log set: always updated to a new set with every change of loggers
-  private final List<DfsLogger> loggers = new ArrayList<DfsLogger>();
+  // The current logger
+  private DfsLogger currentLog = null;
+  private final SynchronousQueue<Object> nextLog = new SynchronousQueue<>();
+  private ThreadPoolExecutor nextLogMaker;
 
-  // The current generation of logSet.
-  // Because multiple threads can be using a log set at one time, a log
+  // The current generation of logs.
+  // Because multiple threads can be using a log at one time, a log
   // failure is likely to affect multiple threads, who will all attempt to
-  // create a new logSet. This will cause many unnecessary updates to the
+  // create a new log. This will cause many unnecessary updates to the
   // metadata table.
   // We'll use this generational counter to determine if another thread has
-  // already fetched a new logSet.
-  private AtomicInteger logSetId = new AtomicInteger();
+  // already fetched a new log.
+  private final AtomicInteger logId = new AtomicInteger();
 
   // Use a ReadWriteLock to allow multiple threads to use the log set, but obtain a write lock to change them
-  private final ReentrantReadWriteLock logSetLock = new ReentrantReadWriteLock();
+  private final ReentrantReadWriteLock logIdLock = new ReentrantReadWriteLock();
 
   private final AtomicInteger seqGen = new AtomicInteger();
 
@@ -146,62 +152,66 @@ public class TabletServerLogger {
     this.flushCounter = flushCounter;
   }
 
-  private int initializeLoggers(final List<DfsLogger> copy) throws IOException {
-    final int[] result = {-1};
-    testLockAndRun(logSetLock, new TestCallWithWriteLock() {
+  private DfsLogger initializeLoggers(final AtomicInteger logIdOut) throws IOException {
+    final AtomicReference<DfsLogger> result = new AtomicReference<DfsLogger>();
+    testLockAndRun(logIdLock, new TestCallWithWriteLock() {
       @Override
       boolean test() {
-        copy.clear();
-        copy.addAll(loggers);
-        if (!loggers.isEmpty())
-          result[0] = logSetId.get();
-        return loggers.isEmpty();
+        result.set(currentLog);
+        if (currentLog != null)
+          logIdOut.set(logId.get());
+        return currentLog == null;
       }
 
       @Override
       void withWriteLock() throws IOException {
         try {
-          createLoggers();
-          copy.clear();
-          copy.addAll(loggers);
-          if (copy.size() > 0)
-            result[0] = logSetId.get();
+          createLogger();
+          result.set(currentLog);
+          if (currentLog != null)
+            logIdOut.set(logId.get());
           else
-            result[0] = -1;
+            logIdOut.set(-1);
         } catch (IOException e) {
           log.error("Unable to create loggers", e);
         }
       }
     });
-    return result[0];
+    return result.get();
   }
 
-  public void getLogFiles(Set<String> loggersOut) {
-    logSetLock.readLock().lock();
+  public String getLogFile() {
+    logIdLock.readLock().lock();
     try {
-      for (DfsLogger logger : loggers) {
-        loggersOut.add(logger.getFileName());
-      }
+      return currentLog.getFileName();
     } finally {
-      logSetLock.readLock().unlock();
+      logIdLock.readLock().unlock();
     }
   }
 
-  synchronized private void createLoggers() throws IOException {
-    if (!logSetLock.isWriteLockedByCurrentThread()) {
+  synchronized private void createLogger() throws IOException {
+    if (!logIdLock.isWriteLockedByCurrentThread()) {
       throw new IllegalStateException("createLoggers should be called with write lock held!");
     }
 
-    if (loggers.size() != 0) {
-      throw new IllegalStateException("createLoggers should not be called when loggers.size() is " + loggers.size());
+    if (currentLog != null) {
+      throw new IllegalStateException("createLoggers should not be called when current log is set");
     }
 
     try {
-      DfsLogger alog = new DfsLogger(tserver.getServerConfig(), syncCounter, flushCounter);
-      alog.open(tserver.getClientAddressString());
-      loggers.add(alog);
-      logSetId.incrementAndGet();
-      return;
+      startLogMaker();
+      Object next = nextLog.take();
+      if (next instanceof Exception) {
+        throw (Exception)next;
+      }
+      if (next instanceof DfsLogger) {
+        currentLog = (DfsLogger)next;
+        logId.incrementAndGet();
+        log.info("Using next log " + currentLog.getFileName());
+        return;
+      } else {
+        throw new RuntimeException("Error: unexpected type seen: " + next);
+      }
     } catch (Exception t) {
       walErrors.put(System.currentTimeMillis(), "");
       if (walErrors.size() >= HALT_AFTER_ERROR_COUNT) {
@@ -211,22 +221,63 @@ public class TabletServerLogger {
     }
   }
 
+  private synchronized void startLogMaker() {
+    if (nextLogMaker != null) {
+      return;
+    }
+    nextLogMaker = new SimpleThreadPool(1, "WALog creator");
+    nextLogMaker.submit(new Runnable() {
+      @Override
+      public void run() {
+        while (!nextLogMaker.isShutdown()) {
+          try {
+            log.debug("Creating next WAL");
+            DfsLogger alog = new DfsLogger(tserver.getServerConfig(), syncCounter, flushCounter);
+            alog.open(tserver.getClientAddressString());
+            log.debug("Created next WAL " + alog.getFileName());
+            while (!nextLog.offer(alog, 12, TimeUnit.HOURS)) {
+              log.info("Our WAL was not used for 12 hours: " + alog.getFileName());
+            }
+          } catch (Exception t) {
+            log.error("{}", t.getMessage(), t);
+            try {
+              nextLog.offer(t, 12, TimeUnit.HOURS);
+            } catch (InterruptedException ex) {
+              // ignore
+            }
+          }
+        }
+      }
+    });
+  }
+
+  public void resetLoggers() throws IOException {
+    logIdLock.writeLock().lock();
+    try {
+      close();
+    } finally {
+      logIdLock.writeLock().unlock();
+    }
+  }
+
   synchronized private void close() throws IOException {
-    if (!logSetLock.isWriteLockedByCurrentThread()) {
+    if (!logIdLock.isWriteLockedByCurrentThread()) {
       throw new IllegalStateException("close should be called with write lock held!");
     }
     try {
-      for (DfsLogger logger : loggers) {
+      if (null != currentLog) {
         try {
-          logger.close();
+          currentLog.close();
         } catch (DfsLogger.LogClosedException ex) {
           // ignore
         } catch (Throwable ex) {
-          log.error("Unable to cleanly close log " + logger.getFileName() + ": " + ex, ex);
+          log.error("Unable to cleanly close log " + currentLog.getFileName() + ": " + ex, ex);
+        } finally {
+          this.tserver.walogClosed(currentLog);
         }
+        currentLog = null;
+        logSizeEstimate.set(0);
       }
-      loggers.clear();
-      logSizeEstimate.set(0);
     } catch (Throwable t) {
       throw new IOException(t);
     }
@@ -243,7 +294,7 @@ public class TabletServerLogger {
 
   private int write(final Collection<CommitSession> sessions, boolean mincFinish, Writer writer) throws IOException {
     // Work very hard not to lock this during calls to the outside world
-    int currentLogSet = logSetId.get();
+    int currentLogId = logId.get();
 
     int seq = -1;
     int attempt = 1;
@@ -251,20 +302,22 @@ public class TabletServerLogger {
     while (!success) {
       try {
         // get a reference to the loggers that no other thread can touch
-        ArrayList<DfsLogger> copy = new ArrayList<DfsLogger>();
-        currentLogSet = initializeLoggers(copy);
+        DfsLogger copy = null;
+        AtomicInteger currentId = new AtomicInteger(-1);
+        copy = initializeLoggers(currentId);
+        currentLogId = currentId.get();
 
         // add the logger to the log set for the memory in the tablet,
         // update the metadata table if we've never used this tablet
 
-        if (currentLogSet == logSetId.get()) {
+        if (currentLogId == logId.get()) {
           for (CommitSession commitSession : sessions) {
             if (commitSession.beginUpdatingLogsUsed(copy, mincFinish)) {
               try {
                 // Scribble out a tablet definition and then write to the metadata table
                 defineTablet(commitSession);
-                if (currentLogSet == logSetId.get())
-                  tserver.addLoggersToMetadata(copy, commitSession.getExtent(), commitSession.getLogId());
+                if (currentLogId == logId.get())
+                  tserver.addLoggersToMetadata(copy, TabletLevel.getLevel(commitSession.getExtent()));
               } finally {
                 commitSession.finishUpdatingLogsUsed();
               }
@@ -272,39 +325,29 @@ public class TabletServerLogger {
               // Need to release
               KeyExtent extent = commitSession.getExtent();
               if (ReplicationConfigurationUtil.isEnabled(extent, tserver.getTableConfiguration(extent))) {
-                Set<String> logs = new HashSet<String>();
-                for (DfsLogger logger : copy) {
-                  logs.add(logger.getFileName());
-                }
-                Status status = StatusUtil.fileCreated(System.currentTimeMillis());
-                log.debug("Writing " + ProtobufUtil.toString(status) + " to metadata table for " + logs);
+                Status status = StatusUtil.openWithUnknownLength(System.currentTimeMillis());
+                log.debug("Writing " + ProtobufUtil.toString(status) + " to metadata table for " + copy.getFileName());
                 // Got some new WALs, note this in the metadata table
-                ReplicationTableUtil.updateFiles(tserver, commitSession.getExtent(), logs, status);
+                ReplicationTableUtil.updateFiles(tserver, commitSession.getExtent(), copy.getFileName(), status);
               }
             }
           }
         }
 
         // Make sure that the logs haven't changed out from underneath our copy
-        if (currentLogSet == logSetId.get()) {
+        if (currentLogId == logId.get()) {
 
           // write the mutation to the logs
           seq = seqGen.incrementAndGet();
           if (seq < 0)
             throw new RuntimeException("Logger sequence generator wrapped!  Onos!!!11!eleven");
-          ArrayList<LoggerOperation> queuedOperations = new ArrayList<LoggerOperation>(copy.size());
-          for (DfsLogger wal : copy) {
-            LoggerOperation lop = writer.write(wal, seq);
-            if (lop != null)
-              queuedOperations.add(lop);
-          }
-
-          for (LoggerOperation lop : queuedOperations) {
+          LoggerOperation lop = writer.write(copy, seq);
+          if (lop != null) {
             lop.await();
           }
 
           // double-check: did the log set change?
-          success = (currentLogSet == logSetId.get());
+          success = (currentLogId == logId.get());
         }
       } catch (DfsLogger.LogClosedException ex) {
         log.debug("Logs closed while writing, retrying " + attempt);
@@ -319,13 +362,13 @@ public class TabletServerLogger {
       // Some sort of write failure occurred. Grab the write lock and reset the logs.
       // But since multiple threads will attempt it, only attempt the reset when
       // the logs haven't changed.
-      final int finalCurrent = currentLogSet;
+      final int finalCurrent = currentLogId;
       if (!success) {
-        testLockAndRun(logSetLock, new TestCallWithWriteLock() {
+        testLockAndRun(logIdLock, new TestCallWithWriteLock() {
 
           @Override
           boolean test() {
-            return finalCurrent == logSetId.get();
+            return finalCurrent == logId.get();
           }
 
           @Override
@@ -338,7 +381,7 @@ public class TabletServerLogger {
     }
     // if the log gets too big, reset it .. grab the write lock first
     logSizeEstimate.addAndGet(4 * 3); // event, tid, seq overhead
-    testLockAndRun(logSetLock, new TestCallWithWriteLock() {
+    testLockAndRun(logIdLock, new TestCallWithWriteLock() {
       @Override
       boolean test() {
         return logSizeEstimate.get() > maxSize;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CommitSession.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CommitSession.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CommitSession.java
index d908f1d..dee705c 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CommitSession.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/CommitSession.java
@@ -16,7 +16,6 @@
  */
 package org.apache.accumulo.tserver.tablet;
 
-import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.accumulo.core.data.Mutation;
@@ -86,7 +85,7 @@ public class CommitSession {
     return committer;
   }
 
-  public boolean beginUpdatingLogsUsed(ArrayList<DfsLogger> copy, boolean mincFinish) {
+  public boolean beginUpdatingLogsUsed(DfsLogger copy, boolean mincFinish) {
     return committer.beginUpdatingLogsUsed(memTable, copy, mincFinish);
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/DatafileManager.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/DatafileManager.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/DatafileManager.java
index db1b418..ab15ccc 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/DatafileManager.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/DatafileManager.java
@@ -424,7 +424,9 @@ class DatafileManager {
         if (log.isDebugEnabled()) {
           log.debug("Recording that data has been ingested into " + tablet.getExtent() + " using " + logFileOnly);
         }
-        ReplicationTableUtil.updateFiles(tablet.getTabletServer(), tablet.getExtent(), logFileOnly, StatusUtil.openWithUnknownLength());
+        for (String logFile : logFileOnly) {
+          ReplicationTableUtil.updateFiles(tablet.getTabletServer(), tablet.getExtent(), logFile, StatusUtil.openWithUnknownLength());
+        }
       }
     } finally {
       tablet.finishClearingUnusedLogs();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
index 1f4625b..17864be 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java
@@ -37,6 +37,7 @@ import java.util.PriorityQueue;
 import java.util.Set;
 import java.util.SortedMap;
 import java.util.TreeMap;
+import java.util.concurrent.ConcurrentSkipListSet;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.AtomicReference;
@@ -200,7 +201,7 @@ public class Tablet implements TabletCommitter {
   }
 
   // stores info about user initiated major compaction that is waiting on a minor compaction to finish
-  private CompactionWaitInfo compactionWaitInfo = new CompactionWaitInfo();
+  private final CompactionWaitInfo compactionWaitInfo = new CompactionWaitInfo();
 
   static enum CompactionState {
     WAITING_TO_START, IN_PROGRESS
@@ -627,8 +628,8 @@ public class Tablet implements TabletCommitter {
           // the WAL isn't closed (WRT replication Status) and thus we're safe to update its progress.
           Status status = StatusUtil.openWithUnknownLength();
           for (LogEntry logEntry : logEntries) {
-            log.debug("Writing updated status to metadata table for " + logEntry.logSet + " " + ProtobufUtil.toString(status));
-            ReplicationTableUtil.updateFiles(tabletServer, extent, logEntry.logSet, status);
+            log.debug("Writing updated status to metadata table for " + logEntry.filename + " " + ProtobufUtil.toString(status));
+            ReplicationTableUtil.updateFiles(tabletServer, extent, logEntry.filename, status);
           }
         }
 
@@ -640,11 +641,9 @@ public class Tablet implements TabletCommitter {
         }
       }
       // make some closed references that represent the recovered logs
-      currentLogs = new HashSet<DfsLogger>();
+      currentLogs = new ConcurrentSkipListSet<DfsLogger>();
       for (LogEntry logEntry : logEntries) {
-        for (String log : logEntry.logSet) {
-          currentLogs.add(new DfsLogger(tabletServer.getServerConfig(), log, logEntry.getColumnQualifier().toString()));
-        }
+        currentLogs.add(new DfsLogger(tabletServer.getServerConfig(), logEntry.filename, logEntry.getColumnQualifier().toString()));
       }
 
       log.info("Write-Ahead Log recovery complete for " + this.extent + " (" + count[0] + " mutations applied, " + getTabletMemory().getNumEntries()
@@ -935,7 +934,9 @@ public class Tablet implements TabletCommitter {
 
     long count = 0;
 
+    String oldName = Thread.currentThread().getName();
     try {
+      Thread.currentThread().setName("Minor compacting " + this.extent);
       Span span = Trace.start("write");
       CompactionStats stats;
       try {
@@ -966,6 +967,7 @@ public class Tablet implements TabletCommitter {
       failed = true;
       throw new RuntimeException(e);
     } finally {
+      Thread.currentThread().setName(oldName);
       try {
         getTabletMemory().finalizeMinC();
       } catch (Throwable t) {
@@ -990,7 +992,7 @@ public class Tablet implements TabletCommitter {
   private synchronized MinorCompactionTask prepareForMinC(long flushId, MinorCompactionReason mincReason) {
     CommitSession oldCommitSession = getTabletMemory().prepareForMinC();
     otherLogs = currentLogs;
-    currentLogs = new HashSet<DfsLogger>();
+    currentLogs = new ConcurrentSkipListSet<DfsLogger>();
 
     FileRef mergeFile = null;
     if (mincReason != MinorCompactionReason.RECOVERY) {
@@ -2374,14 +2376,11 @@ public class Tablet implements TabletCommitter {
     }
   }
 
-  private Set<DfsLogger> currentLogs = new HashSet<DfsLogger>();
+  private ConcurrentSkipListSet<DfsLogger> currentLogs = new ConcurrentSkipListSet<DfsLogger>();
 
-  public synchronized Set<String> getCurrentLogFiles() {
-    Set<String> result = new HashSet<String>();
-    for (DfsLogger log : currentLogs) {
-      result.add(log.getFileName());
-    }
-    return result;
+  // currentLogs may be updated while a tablet is otherwise locked
+  public Set<DfsLogger> getCurrentLogFiles() {
+    return new HashSet<DfsLogger>(currentLogs);
   }
 
   Set<String> beginClearingUnusedLogs() {
@@ -2440,13 +2439,13 @@ public class Tablet implements TabletCommitter {
   // this lock is basically used to synchronize writing of log info to metadata
   private final ReentrantLock logLock = new ReentrantLock();
 
-  public synchronized int getLogCount() {
+  public int getLogCount() {
     return currentLogs.size();
   }
 
   // don't release the lock if this method returns true for success; instead, the caller should clean up by calling finishUpdatingLogsUsed()
   @Override
-  public boolean beginUpdatingLogsUsed(InMemoryMap memTable, Collection<DfsLogger> more, boolean mincFinish) {
+  public boolean beginUpdatingLogsUsed(InMemoryMap memTable, DfsLogger more, boolean mincFinish) {
 
     boolean releaseLock = true;
 
@@ -2483,28 +2482,26 @@ public class Tablet implements TabletCommitter {
 
         int numAdded = 0;
         int numContained = 0;
-        for (DfsLogger logger : more) {
-          if (addToOther) {
-            if (otherLogs.add(logger))
-              numAdded++;
+        if (addToOther) {
+          if (otherLogs.add(more))
+            numAdded++;
 
-            if (currentLogs.contains(logger))
-              numContained++;
-          } else {
-            if (currentLogs.add(logger))
-              numAdded++;
+          if (currentLogs.contains(more))
+            numContained++;
+        } else {
+          if (currentLogs.add(more))
+            numAdded++;
 
-            if (otherLogs.contains(logger))
-              numContained++;
-          }
+          if (otherLogs.contains(more))
+            numContained++;
         }
 
-        if (numAdded > 0 && numAdded != more.size()) {
+        if (numAdded > 0 && numAdded != 1) {
           // expect to add all or none
           throw new IllegalArgumentException("Added subset of logs " + extent + " " + more + " " + currentLogs);
         }
 
-        if (numContained > 0 && numContained != more.size()) {
+        if (numContained > 0 && numContained != 1) {
           // expect to contain all or none
           throw new IllegalArgumentException("Other logs contained subset of logs " + extent + " " + more + " " + otherLogs);
         }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletCommitter.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletCommitter.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletCommitter.java
index c7e3a66..934ce20 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletCommitter.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/TabletCommitter.java
@@ -16,7 +16,6 @@
  */
 package org.apache.accumulo.tserver.tablet;
 
-import java.util.Collection;
 import java.util.List;
 
 import org.apache.accumulo.core.client.Durability;
@@ -38,7 +37,7 @@ public interface TabletCommitter {
   /**
    * If this method returns true, the caller must call {@link #finishUpdatingLogsUsed()} to clean up
    */
-  boolean beginUpdatingLogsUsed(InMemoryMap memTable, Collection<DfsLogger> copy, boolean mincFinish);
+  boolean beginUpdatingLogsUsed(InMemoryMap memTable, DfsLogger copy, boolean mincFinish);
 
   void finishUpdatingLogsUsed();
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/server/tserver/src/test/java/org/apache/accumulo/tserver/log/LogEntryTest.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/log/LogEntryTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/log/LogEntryTest.java
new file mode 100644
index 0000000..44058d3
--- /dev/null
+++ b/server/tserver/src/test/java/org/apache/accumulo/tserver/log/LogEntryTest.java
@@ -0,0 +1,56 @@
+/*
+ * 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.tserver.log;
+
+import static org.junit.Assert.assertEquals;
+
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.KeyExtent;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.tabletserver.log.LogEntry;
+import org.apache.hadoop.io.Text;
+import org.junit.Test;
+
+public class LogEntryTest {
+
+  @Test
+  public void test() throws Exception {
+    KeyExtent extent = new KeyExtent(new Text("1"), null, new Text(""));
+    long ts = 12345678L;
+    String server = "localhost:1234";
+    String filename = "default/foo";
+    LogEntry entry = new LogEntry(extent, ts, server, filename);
+    assertEquals(extent, entry.extent);
+    assertEquals(server, entry.server);
+    assertEquals(filename, entry.filename);
+    assertEquals(ts, entry.timestamp);
+    assertEquals("1<; default/foo", entry.toString());
+    assertEquals(new Text("log"), entry.getColumnFamily());
+    assertEquals(new Text("localhost:1234/default/foo"), entry.getColumnQualifier());
+    LogEntry copy = LogEntry.fromBytes(entry.toBytes());
+    assertEquals(entry.toString(), copy.toString());
+    Key key = new Key(new Text("1<"), new Text("log"), new Text("localhost:1234/default/foo"));
+    key.setTimestamp(ts);
+    LogEntry copy2 = LogEntry.fromKeyValue(key, entry.getValue());
+    assertEquals(entry.toString(), copy2.toString());
+    assertEquals(entry.timestamp, copy2.timestamp);
+    assertEquals("foo", entry.getUniqueID());
+    assertEquals("localhost:1234/default/foo", entry.getName());
+    assertEquals(new Value("default/foo".getBytes()), entry.getValue());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java b/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java
index d0de29f..1186c68 100644
--- a/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java
+++ b/test/src/main/java/org/apache/accumulo/test/performance/thrift/NullTserver.java
@@ -202,9 +202,6 @@ public class NullTserver {
     }
 
     @Override
-    public void removeLogs(TInfo tinfo, TCredentials credentials, List<String> filenames) throws TException {}
-
-    @Override
     public List<ActiveCompaction> getActiveCompactions(TInfo tinfo, TCredentials credentials) throws ThriftSecurityException, TException {
       return new ArrayList<ActiveCompaction>();
     }
@@ -231,6 +228,9 @@ public class NullTserver {
     public List<String> getActiveLogs(TInfo tinfo, TCredentials credentials) throws TException {
       return null;
     }
+
+    @Override
+    public void removeLogs(TInfo tinfo, TCredentials credentials, List<String> filenames) throws TException { }
   }
 
   static class Opts extends Help {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/test/src/test/java/org/apache/accumulo/proxy/ProxyDurabilityIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/proxy/ProxyDurabilityIT.java b/test/src/test/java/org/apache/accumulo/proxy/ProxyDurabilityIT.java
index 404a8fd..81e25cc 100644
--- a/test/src/test/java/org/apache/accumulo/proxy/ProxyDurabilityIT.java
+++ b/test/src/test/java/org/apache/accumulo/proxy/ProxyDurabilityIT.java
@@ -60,6 +60,11 @@ import com.google.common.net.HostAndPort;
 public class ProxyDurabilityIT extends ConfigurableMacIT {
 
   @Override
+  protected int defaultTimeoutSeconds() {
+    return 60;
+  }
+
+  @Override
   public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
     hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
     cfg.setProperty(Property.INSTANCE_ZK_TIMEOUT, "10s");
@@ -111,7 +116,7 @@ public class ProxyDurabilityIT extends ConfigurableMacIT {
     assertEquals(0, count(tableName));
 
     ConditionalWriterOptions cfg = new ConditionalWriterOptions();
-    cfg.setDurability(Durability.LOG);
+    cfg.setDurability(Durability.SYNC);
     String cwriter = client.createConditionalWriter(login, tableName, cfg);
     ConditionalUpdates updates = new ConditionalUpdates();
     updates.addToConditions(new Condition(new Column(bytes("cf"), bytes("cq"), bytes(""))));
@@ -120,7 +125,7 @@ public class ProxyDurabilityIT extends ConfigurableMacIT {
     assertEquals(ConditionalStatus.ACCEPTED, status.get(bytes("row")));
     assertEquals(1, count(tableName));
     restartTServer();
-    assertEquals(0, count(tableName));
+    assertEquals(1, count(tableName));
 
     proxyServer.stop();
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/test/src/test/java/org/apache/accumulo/test/BadDeleteMarkersCreatedIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/BadDeleteMarkersCreatedIT.java b/test/src/test/java/org/apache/accumulo/test/BadDeleteMarkersCreatedIT.java
index 25337b2..0dcdf42 100644
--- a/test/src/test/java/org/apache/accumulo/test/BadDeleteMarkersCreatedIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/BadDeleteMarkersCreatedIT.java
@@ -54,7 +54,7 @@ public class BadDeleteMarkersCreatedIT extends AccumuloClusterIT {
 
   @Override
   public int defaultTimeoutSeconds() {
-    return 60;
+    return 120;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/test/src/test/java/org/apache/accumulo/test/BalanceIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/BalanceIT.java b/test/src/test/java/org/apache/accumulo/test/BalanceIT.java
index f793925..8703f18 100644
--- a/test/src/test/java/org/apache/accumulo/test/BalanceIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/BalanceIT.java
@@ -20,25 +20,33 @@ import java.util.SortedSet;
 import java.util.TreeSet;
 
 import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.test.functional.ConfigurableMacIT;
+import org.apache.accumulo.harness.AccumuloClusterIT;
 import org.apache.hadoop.io.Text;
 import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
-public class BalanceIT extends ConfigurableMacIT {
+public class BalanceIT extends AccumuloClusterIT {
+  private static final Logger log = LoggerFactory.getLogger(BalanceIT.class);
 
-  @Test(timeout = 60 * 1000)
+  @Override
+  public int defaultTimeoutSeconds() {
+    return 60;
+  }
+
+  @Test
   public void testBalance() throws Exception {
     String tableName = getUniqueNames(1)[0];
     Connector c = getConnector();
-    System.out.println("Creating table");
+    log.info("Creating table");
     c.tableOperations().create(tableName);
     SortedSet<Text> splits = new TreeSet<Text>();
     for (int i = 0; i < 10; i++) {
       splits.add(new Text("" + i));
     }
-    System.out.println("Adding splits");
+    log.info("Adding splits");
     c.tableOperations().addSplits(tableName, splits);
-    System.out.println("Waiting for balance");
+    log.info("Waiting for balance");
     c.instanceOperations().waitForBalance();
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/test/src/test/java/org/apache/accumulo/test/CleanWalIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/CleanWalIT.java b/test/src/test/java/org/apache/accumulo/test/CleanWalIT.java
index f553be8..fcad293 100644
--- a/test/src/test/java/org/apache/accumulo/test/CleanWalIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/CleanWalIT.java
@@ -128,6 +128,7 @@ public class CleanWalIT extends AccumuloClusterIT {
   private int countLogs(String tableName, Connector conn) throws TableNotFoundException {
     Scanner scanner = conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
     scanner.fetchColumnFamily(MetadataSchema.TabletsSection.LogColumnFamily.NAME);
+    scanner.setRange(MetadataSchema.TabletsSection.getRange());
     int count = 0;
     for (Entry<Key,Value> entry : scanner) {
       log.debug("Saw " + entry.getKey() + "=" + entry.getValue());

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/test/src/test/java/org/apache/accumulo/test/ConditionalWriterIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/ConditionalWriterIT.java b/test/src/test/java/org/apache/accumulo/test/ConditionalWriterIT.java
index b7637a6..65be396 100644
--- a/test/src/test/java/org/apache/accumulo/test/ConditionalWriterIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/ConditionalWriterIT.java
@@ -1294,6 +1294,7 @@ public class ConditionalWriterIT extends AccumuloClusterIT {
     conn.tableOperations().create(tableName);
 
     DistributedTrace.enable("localhost", "testTrace", mac.getClientConfig());
+    UtilWaitThread.sleep(1000);
     Span root = Trace.on("traceTest");
     ConditionalWriter cw = conn.createConditionalWriter(tableName, new ConditionalWriterConfig());
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/test/src/test/java/org/apache/accumulo/test/GarbageCollectWALIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/GarbageCollectWALIT.java b/test/src/test/java/org/apache/accumulo/test/GarbageCollectWALIT.java
new file mode 100644
index 0000000..0324e4a
--- /dev/null
+++ b/test/src/test/java/org/apache/accumulo/test/GarbageCollectWALIT.java
@@ -0,0 +1,81 @@
+/*
+ * 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.test;
+
+import static org.junit.Assert.assertEquals;
+
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.fate.util.UtilWaitThread;
+import org.apache.accumulo.minicluster.ServerType;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloClusterImpl;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.test.functional.ConfigurableMacIT;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RawLocalFileSystem;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.junit.Test;
+
+import com.google.common.collect.Iterators;
+
+public class GarbageCollectWALIT extends ConfigurableMacIT {
+
+  @Override
+  protected void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
+    cfg.setProperty(Property.INSTANCE_ZK_HOST, "5s");
+    cfg.setProperty(Property.GC_CYCLE_START, "1s");
+    cfg.setProperty(Property.GC_CYCLE_DELAY, "1s");
+    cfg.setNumTservers(1);
+    hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
+  }
+
+  @Test(timeout = 2 * 60 * 1000)
+  public void test() throws Exception {
+    // not yet, please
+    String tableName = getUniqueNames(1)[0];
+    cluster.getClusterControl().stop(ServerType.GARBAGE_COLLECTOR);
+    Connector c = getConnector();
+    c.tableOperations().create(tableName);
+    // count the number of WALs in the filesystem
+    assertEquals(2, countWALsInFS(cluster));
+    cluster.getClusterControl().stop(ServerType.TABLET_SERVER);
+    cluster.getClusterControl().start(ServerType.GARBAGE_COLLECTOR);
+    cluster.getClusterControl().start(ServerType.TABLET_SERVER);
+    Iterators.size(c.createScanner(MetadataTable.NAME, Authorizations.EMPTY).iterator());
+    // let GC run
+    UtilWaitThread.sleep(3 * 5 * 1000);
+    assertEquals(2, countWALsInFS(cluster));
+  }
+
+  private int countWALsInFS(MiniAccumuloClusterImpl cluster) throws Exception {
+    FileSystem fs = cluster.getFileSystem();
+    RemoteIterator<LocatedFileStatus> iterator = fs.listFiles(new Path(cluster.getConfig().getAccumuloDir() + "/wal"), true);
+    int result = 0;
+    while (iterator.hasNext()) {
+      LocatedFileStatus next = iterator.next();
+      if (!next.isDirectory()) {
+        result++;
+      }
+    }
+    return result;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/test/src/test/java/org/apache/accumulo/test/MissingWalHeaderCompletesRecoveryIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/MissingWalHeaderCompletesRecoveryIT.java b/test/src/test/java/org/apache/accumulo/test/MissingWalHeaderCompletesRecoveryIT.java
index b78a311..27f1f69 100644
--- a/test/src/test/java/org/apache/accumulo/test/MissingWalHeaderCompletesRecoveryIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/MissingWalHeaderCompletesRecoveryIT.java
@@ -19,7 +19,6 @@ package org.apache.accumulo.test;
 import static java.nio.charset.StandardCharsets.UTF_8;
 
 import java.io.File;
-import java.util.Collections;
 import java.util.UUID;
 
 import org.apache.accumulo.core.client.BatchWriter;
@@ -27,6 +26,7 @@ import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema;
@@ -127,11 +127,7 @@ public class MissingWalHeaderCompletesRecoveryIT extends ConfigurableMacIT {
     String tableId = conn.tableOperations().tableIdMap().get(tableName);
     Assert.assertNotNull("Table ID was null", tableId);
 
-    LogEntry logEntry = new LogEntry();
-    logEntry.server = "127.0.0.1:12345";
-    logEntry.filename = emptyWalog.toURI().toString();
-    logEntry.tabletId = 10;
-    logEntry.logSet = Collections.singleton(logEntry.filename);
+    LogEntry logEntry = new LogEntry(new KeyExtent(new Text(tableId), null, null), 0, "127.0.0.1:12345", emptyWalog.toURI().toString());
 
     log.info("Taking {} offline", tableName);
     conn.tableOperations().offline(tableName, true);
@@ -186,11 +182,7 @@ public class MissingWalHeaderCompletesRecoveryIT extends ConfigurableMacIT {
     String tableId = conn.tableOperations().tableIdMap().get(tableName);
     Assert.assertNotNull("Table ID was null", tableId);
 
-    LogEntry logEntry = new LogEntry();
-    logEntry.server = "127.0.0.1:12345";
-    logEntry.filename = partialHeaderWalog.toURI().toString();
-    logEntry.tabletId = 10;
-    logEntry.logSet = Collections.singleton(logEntry.filename);
+    LogEntry logEntry = new LogEntry(null, 0, "127.0.0.1:12345", partialHeaderWalog.toURI().toString());
 
     log.info("Taking {} offline", tableName);
     conn.tableOperations().offline(tableName, true);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/test/src/test/java/org/apache/accumulo/test/NoMutationRecoveryIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/NoMutationRecoveryIT.java b/test/src/test/java/org/apache/accumulo/test/NoMutationRecoveryIT.java
deleted file mode 100644
index 6a9975c..0000000
--- a/test/src/test/java/org/apache/accumulo/test/NoMutationRecoveryIT.java
+++ /dev/null
@@ -1,178 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.test;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.util.Map.Entry;
-
-import org.apache.accumulo.cluster.ClusterControl;
-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.Scanner;
-import org.apache.accumulo.core.data.Key;
-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.metadata.MetadataTable;
-import org.apache.accumulo.core.metadata.RootTable;
-import org.apache.accumulo.core.metadata.schema.MetadataSchema;
-import org.apache.accumulo.core.security.Authorizations;
-import org.apache.accumulo.core.security.TablePermission;
-import org.apache.accumulo.harness.AccumuloClusterIT;
-import org.apache.accumulo.minicluster.ServerType;
-import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
-import org.apache.accumulo.test.functional.FunctionalTestUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.RawLocalFileSystem;
-import org.apache.hadoop.io.Text;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-// Verify that a recovery of a log without any mutations removes the log reference
-public class NoMutationRecoveryIT extends AccumuloClusterIT {
-  private static final Logger log = LoggerFactory.getLogger(NoMutationRecoveryIT.class);
-
-  @Override
-  public int defaultTimeoutSeconds() {
-    return 10 * 60;
-  }
-
-  @Override
-  public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
-    cfg.setNumTservers(1);
-    hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
-  }
-
-  @Before
-  public void takeTraceTableOffline() throws Exception {
-    Connector conn = getConnector();
-    if (conn.tableOperations().exists("trace")) {
-      conn.tableOperations().offline("trace", true);
-    }
-  }
-
-  @After
-  public void takeTraceTableOnline() throws Exception {
-    Connector conn = getConnector();
-    if (conn.tableOperations().exists("trace")) {
-      conn.tableOperations().online("trace", true);
-    }
-  }
-
-  public boolean equals(Entry<Key,Value> a, Entry<Key,Value> b) {
-    // comparison, without timestamp
-    Key akey = a.getKey();
-    Key bkey = b.getKey();
-    log.info("Comparing {} to {}", akey.toStringNoTruncate(), bkey.toStringNoTruncate());
-    return akey.compareTo(bkey, PartialKey.ROW_COLFAM_COLQUAL_COLVIS) == 0 && a.getValue().equals(b.getValue());
-  }
-
-  @Test
-  public void test() throws Exception {
-    Connector conn = getConnector();
-    final String table = getUniqueNames(1)[0];
-    conn.tableOperations().create(table);
-    String tableId = conn.tableOperations().tableIdMap().get(table);
-
-    log.info("Created {} with id {}", table, tableId);
-
-    // Add a record to the table
-    update(conn, table, new Text("row"), new Text("cf"), new Text("cq"), new Value("value".getBytes()));
-
-    // Get the WAL reference used by the table we just added the update to
-    Entry<Key,Value> logRef = getLogRef(conn, MetadataTable.NAME);
-
-    log.info("Log reference in metadata table {} {}", logRef.getKey().toStringNoTruncate(), logRef.getValue());
-
-    // Flush the record to disk
-    conn.tableOperations().flush(table, null, null, true);
-
-    Range range = Range.prefix(tableId);
-    log.info("Fetching WAL references over " + table);
-    assertEquals("should not have any refs", 0, FunctionalTestUtils.count(getLogRefs(conn, MetadataTable.NAME, range)));
-
-    // Grant permission to the admin user to write to the Metadata table
-    conn.securityOperations().grantTablePermission(conn.whoami(), MetadataTable.NAME, TablePermission.WRITE);
-
-    // Add the wal record back to the metadata table
-    update(conn, MetadataTable.NAME, logRef);
-
-    // Assert that we can get the bogus update back out again
-    assertTrue(equals(logRef, getLogRef(conn, MetadataTable.NAME)));
-
-    conn.tableOperations().flush(MetadataTable.NAME, null, null, true);
-    conn.tableOperations().flush(RootTable.NAME, null, null, true);
-
-    ClusterControl control = cluster.getClusterControl();
-    control.stopAllServers(ServerType.TABLET_SERVER);
-    control.startAllServers(ServerType.TABLET_SERVER);
-
-    // Verify that we can read the original record we wrote
-    Scanner s = conn.createScanner(table, Authorizations.EMPTY);
-    int count = 0;
-    for (Entry<Key,Value> e : s) {
-      assertEquals(e.getKey().getRow().toString(), "row");
-      assertEquals(e.getKey().getColumnFamily().toString(), "cf");
-      assertEquals(e.getKey().getColumnQualifier().toString(), "cq");
-      assertEquals(e.getValue().toString(), "value");
-      count++;
-    }
-    assertEquals(1, count);
-
-    // Verify that the bogus log reference we wrote it gone
-    for (Entry<Key,Value> ref : getLogRefs(conn, MetadataTable.NAME)) {
-      assertFalse("Unexpected found reference to bogus log entry: " + ref.getKey().toStringNoTruncate() + " " + ref.getValue(), equals(ref, logRef));
-    }
-  }
-
-  private void update(Connector conn, String name, Entry<Key,Value> logRef) throws Exception {
-    Key k = logRef.getKey();
-    update(conn, name, k.getRow(), k.getColumnFamily(), k.getColumnQualifier(), logRef.getValue());
-  }
-
-  private Iterable<Entry<Key,Value>> getLogRefs(Connector conn, String table) throws Exception {
-    return getLogRefs(conn, table, new Range());
-  }
-
-  private Iterable<Entry<Key,Value>> getLogRefs(Connector conn, String table, Range r) throws Exception {
-    Scanner s = conn.createScanner(table, Authorizations.EMPTY);
-    s.fetchColumnFamily(MetadataSchema.TabletsSection.LogColumnFamily.NAME);
-    s.setRange(r);
-    return s;
-  }
-
-  private Entry<Key,Value> getLogRef(Connector conn, String table) throws Exception {
-    return getLogRefs(conn, table).iterator().next();
-  }
-
-  private void update(Connector conn, String table, Text row, Text cf, Text cq, Value value) throws Exception {
-    BatchWriter bw = conn.createBatchWriter(table, new BatchWriterConfig());
-    Mutation m = new Mutation(row);
-    m.put(cf, cq, value);
-    bw.addMutation(m);
-    bw.close();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java b/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
index 8b37169..5c5b95d 100644
--- a/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
@@ -349,7 +349,7 @@ public class ShellServerIT extends SharedMiniClusterIT {
     ts.exec("config -t " + table2 + " -np", true, "345M", true);
     ts.exec("getsplits -t " + table2, true, "row5", true);
     ts.exec("constraint --list -t " + table2, true, "VisibilityConstraint=2", true);
-    ts.exec("onlinetable " + table, true);
+    ts.exec("online " + table, true);
     ts.exec("deletetable -f " + table, true);
     ts.exec("deletetable -f " + table2, true);
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/test/src/test/java/org/apache/accumulo/test/UnusedWALIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/UnusedWALIT.java b/test/src/test/java/org/apache/accumulo/test/UnusedWALIT.java
new file mode 100644
index 0000000..03d783c
--- /dev/null
+++ b/test/src/test/java/org/apache/accumulo/test/UnusedWALIT.java
@@ -0,0 +1,144 @@
+/*
+ * 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.test;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.Map.Entry;
+
+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.Scanner;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.CurrentLogsSection;
+import org.apache.accumulo.core.security.Authorizations;
+import org.apache.accumulo.minicluster.ServerType;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.test.functional.ConfigurableMacIT;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.RawLocalFileSystem;
+import org.junit.Test;
+
+import com.google.common.collect.Iterators;
+
+// When reviewing the changes for ACCUMULO-3423, kturner suggested
+// "tablets will now have log references that contain no data,
+// so it may be marked with 3 WALs, the first with data, the 2nd without, a 3rd with data.
+// It would be useful to have an IT that will test this situation.
+public class UnusedWALIT extends ConfigurableMacIT {
+
+  @Override
+  protected void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
+    final long logSize = 1024 * 1024 * 10;
+    cfg.setProperty(Property.INSTANCE_ZK_TIMEOUT, "5s");
+    cfg.setProperty(Property.TSERV_WALOG_MAX_SIZE, Long.toString(logSize));
+    cfg.setNumTservers(1);
+    // use raw local file system so walogs sync and flush will work
+    hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
+    hadoopCoreSite.set("fs.namenode.fs-limits.min-block-size", Long.toString(logSize));
+  }
+
+  @Test(timeout = 2 * 60 * 1000)
+  public void test() throws Exception {
+    // don't want this bad boy cleaning up walog entries
+    getCluster().getClusterControl().stop(ServerType.GARBAGE_COLLECTOR);
+
+    // make two tables
+    String[] tableNames = getUniqueNames(2);
+    String bigTable = tableNames[0];
+    String lilTable = tableNames[1];
+    Connector c = getConnector();
+    c.tableOperations().create(bigTable);
+    c.tableOperations().create(lilTable);
+
+    // put some data in a log that should be replayed for both tables
+    writeSomeData(c, bigTable, 0, 10, 0, 10);
+    scanSomeData(c, bigTable, 0, 10, 0, 10);
+    writeSomeData(c, lilTable, 0, 1, 0, 1);
+    scanSomeData(c, lilTable, 0, 1, 0, 1);
+    assertEquals(1, getWALCount(c));
+
+    // roll the logs by pushing data into bigTable
+    writeSomeData(c, bigTable, 0, 3000, 0, 1000);
+    assertEquals(2, getWALCount(c));
+
+    // put some data in the latest log
+    writeSomeData(c, lilTable, 1, 10, 0, 10);
+    scanSomeData(c, lilTable, 1, 10, 0, 10);
+
+    // bounce the tserver
+    getCluster().getClusterControl().stop(ServerType.TABLET_SERVER);
+    getCluster().getClusterControl().start(ServerType.TABLET_SERVER);
+
+    // wait for the metadata table to be online
+    Iterators.size(c.createScanner(MetadataTable.NAME, Authorizations.EMPTY).iterator());
+
+    // check our two sets of data in different logs
+    scanSomeData(c, lilTable, 0, 1, 0, 1);
+    scanSomeData(c, lilTable, 1, 10, 0, 10);
+  }
+
+  private void scanSomeData(Connector c, String table, int startRow, int rowCount, int startCol, int colCount) throws Exception {
+    Scanner s = c.createScanner(table, Authorizations.EMPTY);
+    s.setRange(new Range(Integer.toHexString(startRow), Integer.toHexString(startRow + rowCount)));
+    int row = startRow;
+    int col = startCol;
+    for (Entry<Key,Value> entry : s) {
+      assertEquals(row, Integer.parseInt(entry.getKey().getRow().toString(), 16));
+      assertEquals(col++, Integer.parseInt(entry.getKey().getColumnQualifier().toString(), 16));
+      if (col == startCol + colCount) {
+        col = startCol;
+        row++;
+        if (row == startRow + rowCount) {
+          break;
+        }
+      }
+    }
+    assertEquals(row, startRow + rowCount);
+  }
+
+  private int getWALCount(Connector c) throws Exception {
+    Scanner s = c.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
+    s.setRange(CurrentLogsSection.getRange());
+    try {
+      return Iterators.size(s.iterator());
+    } finally {
+      s.close();
+    }
+  }
+
+  private void writeSomeData(Connector conn, String table, int startRow, int rowCount, int startCol, int colCount) throws Exception {
+    BatchWriterConfig config = new BatchWriterConfig();
+    config.setMaxMemory(10 * 1024 * 1024);
+    BatchWriter bw = conn.createBatchWriter(table, config);
+    for (int r = startRow; r < startRow + rowCount; r++) {
+      Mutation m = new Mutation(Integer.toHexString(r));
+      for (int c = startCol; c < startCol + colCount; c++) {
+        m.put("", Integer.toHexString(c), "");
+      }
+      bw.addMutation(m);
+    }
+    bw.close();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/test/src/test/java/org/apache/accumulo/test/VolumeIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/VolumeIT.java b/test/src/test/java/org/apache/accumulo/test/VolumeIT.java
index d9b9429..e2a0e03 100644
--- a/test/src/test/java/org/apache/accumulo/test/VolumeIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/VolumeIT.java
@@ -103,6 +103,7 @@ public class VolumeIT extends ConfigurableMacIT {
     cfg.setProperty(Property.INSTANCE_DFS_DIR, v1Uri.getPath());
     cfg.setProperty(Property.INSTANCE_DFS_URI, v1Uri.getScheme() + v1Uri.getHost());
     cfg.setProperty(Property.INSTANCE_VOLUMES, v1.toString() + "," + v2.toString());
+    cfg.setProperty(Property.INSTANCE_ZK_TIMEOUT, "5s");
 
     // use raw local file system so walogs sync and flush will work
     hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
@@ -425,6 +426,21 @@ public class VolumeIT extends ConfigurableMacIT {
       Assert.fail("Unexpected volume " + path);
     }
 
+    Text path = new Text();
+    for (String table : new String[]{RootTable.NAME, MetadataTable.NAME}) {
+      Scanner meta = conn.createScanner(table, Authorizations.EMPTY);
+      meta.setRange(MetadataSchema.CurrentLogsSection.getRange());
+      outer: for (Entry<Key,Value> entry : meta) {
+        MetadataSchema.CurrentLogsSection.getPath(entry.getKey(), path);
+        for (int i = 0; i < paths.length; i++) {
+          if (path.toString().startsWith(paths[i].toString())) {
+            continue outer;
+          }
+        }
+        Assert.fail("Unexpected volume " + path);
+      }
+    }
+
     // if a volume is chosen randomly for each tablet, then the probability that a volume will not be chosen for any tablet is ((num_volumes -
     // 1)/num_volumes)^num_tablets. For 100 tablets and 3 volumes the probability that only 2 volumes would be chosen is 2.46e-18
 
@@ -435,6 +451,7 @@ public class VolumeIT extends ConfigurableMacIT {
     }
 
     Assert.assertEquals(200, sum);
+
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/accumulo/blob/3fdd29f5/test/src/test/java/org/apache/accumulo/test/functional/ReadWriteIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/ReadWriteIT.java b/test/src/test/java/org/apache/accumulo/test/functional/ReadWriteIT.java
index 099743d..1f3e600 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/ReadWriteIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/ReadWriteIT.java
@@ -57,6 +57,7 @@ import org.apache.accumulo.core.client.admin.TableOperations;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.client.security.tokens.KerberosToken;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Range;
@@ -72,9 +73,11 @@ import org.apache.accumulo.fate.zookeeper.ZooLock;
 import org.apache.accumulo.fate.zookeeper.ZooReader;
 import org.apache.accumulo.harness.AccumuloClusterIT;
 import org.apache.accumulo.minicluster.ServerType;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.test.TestIngest;
 import org.apache.accumulo.test.TestMultiTableIngest;
 import org.apache.accumulo.test.VerifyIngest;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
 import org.junit.Test;
@@ -84,6 +87,11 @@ import org.slf4j.LoggerFactory;
 import com.google.common.base.Charsets;
 
 public class ReadWriteIT extends AccumuloClusterIT {
+  @Override
+  public void configureMiniCluster(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
+    cfg.setProperty(Property.INSTANCE_ZK_TIMEOUT, "5s");
+  }
+
   private static final Logger log = LoggerFactory.getLogger(ReadWriteIT.class);
 
   static final int ROWS = 200000;


[31/34] accumulo git commit: Merge branch '1.7'

Posted by ec...@apache.org.
Merge branch '1.7'


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

Branch: refs/heads/master
Commit: 8b3358cb0f5f0e127ce9f66f2a39ebd9aa88f28a
Parents: 6df7169 3fdd29f
Author: Eric C. Newton <er...@gmail.com>
Authored: Fri Apr 24 18:22:34 2015 -0400
Committer: Eric C. Newton <er...@gmail.com>
Committed: Fri Apr 24 18:22:34 2015 -0400

----------------------------------------------------------------------
 .../client/impl/ReplicationOperationsImpl.java  |   4 +-
 .../org/apache/accumulo/core/conf/Property.java |   4 +-
 .../accumulo/core/metadata/RootTable.java       |   1 +
 .../core/metadata/schema/MetadataSchema.java    |  48 ++
 .../core/tabletserver/log/LogEntry.java         |  78 ++-
 .../core/metadata/MetadataTableSchemaTest.java  |  47 ++
 .../org/apache/accumulo/server/TabletLevel.java |  34 ++
 .../apache/accumulo/server/fs/VolumeUtil.java   |  22 +-
 .../apache/accumulo/server/init/Initialize.java |   1 +
 .../server/master/state/MetaDataStateStore.java |  47 +-
 .../master/state/MetaDataTableScanner.java      |   6 +-
 .../master/state/TabletLocationState.java       |   7 +
 .../server/master/state/TabletStateStore.java   |  16 +-
 .../master/state/ZooTabletStateStore.java       |  35 +-
 .../accumulo/server/replication/StatusUtil.java |  13 +
 .../accumulo/server/util/ListVolumesUsed.java   |  18 +-
 .../server/util/MasterMetadataUtil.java         |  18 +-
 .../accumulo/server/util/MetadataTableUtil.java | 239 +++++---
 .../server/util/ReplicationTableUtil.java       |  13 +-
 .../server/util/ReplicationTableUtilTest.java   |   2 +-
 .../gc/GarbageCollectWriteAheadLogs.java        | 499 +++++++---------
 .../accumulo/gc/SimpleGarbageCollector.java     |   1 -
 .../CloseWriteAheadLogReferences.java           |  23 +-
 .../gc/GarbageCollectWriteAheadLogsTest.java    | 567 -------------------
 .../CloseWriteAheadLogReferencesTest.java       | 151 +----
 .../java/org/apache/accumulo/master/Master.java |   3 +
 .../master/MasterClientServiceHandler.java      |   3 +-
 .../accumulo/master/TabletGroupWatcher.java     |  37 +-
 .../accumulo/master/replication/WorkMaker.java  |   1 +
 .../accumulo/master/state/MergeStats.java       |   3 +-
 .../master/ReplicationOperationsImplTest.java   |   9 +-
 .../apache/accumulo/master/TestMergeState.java  |   2 +-
 .../master/state/RootTabletStateStoreTest.java  |   4 +-
 .../src/main/findbugs/exclude-filter.xml        |   2 +-
 .../server/GarbageCollectionLogger.java         |   3 +-
 .../apache/accumulo/tserver/TabletServer.java   | 182 +++---
 .../apache/accumulo/tserver/log/DfsLogger.java  |  14 +-
 .../accumulo/tserver/log/SortedLogRecovery.java |   8 +-
 .../tserver/log/TabletServerLogger.java         | 187 +++---
 .../accumulo/tserver/tablet/CommitSession.java  |   3 +-
 .../tserver/tablet/DatafileManager.java         |   4 +-
 .../apache/accumulo/tserver/tablet/Tablet.java  |  59 +-
 .../tserver/tablet/TabletCommitter.java         |   3 +-
 .../accumulo/tserver/log/LogEntryTest.java      |  56 ++
 .../test/performance/thrift/NullTserver.java    |   6 +-
 .../accumulo/proxy/ProxyDurabilityIT.java       |   9 +-
 .../test/BadDeleteMarkersCreatedIT.java         |   2 +-
 .../org/apache/accumulo/test/BalanceIT.java     |  20 +-
 .../org/apache/accumulo/test/CleanWalIT.java    |   1 +
 .../accumulo/test/ConditionalWriterIT.java      |   1 +
 .../accumulo/test/GarbageCollectWALIT.java      |  81 +++
 .../MissingWalHeaderCompletesRecoveryIT.java    |  14 +-
 .../accumulo/test/NoMutationRecoveryIT.java     | 178 ------
 .../org/apache/accumulo/test/ShellServerIT.java |   2 +-
 .../org/apache/accumulo/test/UnusedWALIT.java   | 144 +++++
 .../java/org/apache/accumulo/test/VolumeIT.java |  17 +
 .../accumulo/test/functional/ReadWriteIT.java   |   8 +
 .../accumulo/test/functional/WALSunnyDayIT.java | 250 ++++++++
 .../test/functional/WatchTheWatchCountIT.java   |   2 +-
 .../test/performance/RollWALPerformanceIT.java  | 126 +++++
 ...bageCollectorCommunicatesWithTServersIT.java |  35 +-
 .../replication/MultiInstanceReplicationIT.java |   2 +-
 .../test/replication/ReplicationIT.java         | 370 ++++--------
 63 files changed, 1857 insertions(+), 1888 deletions(-)
----------------------------------------------------------------------



[07/34] accumulo git commit: ACCUMULO-3638 mostly updated file references to WALs to be Path objects

Posted by ec...@apache.org.
ACCUMULO-3638 mostly updated file references to WALs to be Path objects


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

Branch: refs/heads/master
Commit: 98c3cef8ccfccbe84a5c35ce7576a9225ee03051
Parents: 902ee7d
Author: Eric C. Newton <er...@gmail.com>
Authored: Tue Mar 10 14:07:43 2015 -0400
Committer: Eric C. Newton <er...@gmail.com>
Committed: Tue Mar 10 14:07:43 2015 -0400

----------------------------------------------------------------------
 .../server/master/state/MetaDataStateStore.java |  17 +-
 .../master/state/TabletLocationState.java       |   3 +
 .../server/master/state/TabletStateStore.java   |   7 +-
 .../master/state/ZooTabletStateStore.java       |  11 +-
 .../accumulo/server/util/MetadataTableUtil.java |  31 ++-
 .../gc/GarbageCollectWriteAheadLogs.java        |  79 +++---
 .../accumulo/master/TabletGroupWatcher.java     |  15 +-
 .../apache/accumulo/tserver/TabletServer.java   |   6 +-
 .../apache/accumulo/tserver/log/DfsLogger.java  |   6 +-
 .../accumulo/test/functional/WALSunnyDayIT.java | 240 +++++++++++++++++++
 10 files changed, 334 insertions(+), 81 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/98c3cef8/server/base/src/main/java/org/apache/accumulo/server/master/state/MetaDataStateStore.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/state/MetaDataStateStore.java b/server/base/src/main/java/org/apache/accumulo/server/master/state/MetaDataStateStore.java
index 1749904..decc8c7 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/state/MetaDataStateStore.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/state/MetaDataStateStore.java
@@ -32,6 +32,7 @@ import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema;
 import org.apache.accumulo.core.tabletserver.log.LogEntry;
 import org.apache.accumulo.server.AccumuloServerContext;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
 import org.apache.log4j.Logger;
 
@@ -123,7 +124,7 @@ public class MetaDataStateStore extends TabletStateStore {
   }
 
   @Override
-  public void unassign(Collection<TabletLocationState> tablets, Map<TServerInstance, List<String>> logsForDeadServers) throws DistributedStoreException {
+  public void unassign(Collection<TabletLocationState> tablets, Map<TServerInstance, List<Path>> logsForDeadServers) throws DistributedStoreException {
 
     BatchWriter writer = createBatchWriter();
     try {
@@ -136,10 +137,10 @@ public class MetaDataStateStore extends TabletStateStore {
           tls.future.clearFutureLocation(m);
         }
         if (logsForDeadServers != null) {
-          List<String> logs = logsForDeadServers.get(tls.futureOrCurrent());
+          List<Path> logs = logsForDeadServers.get(tls.futureOrCurrent());
           if (logs != null) {
-            for (String log : logs) {
-              LogEntry entry = new LogEntry(tls.extent, 0, tls.futureOrCurrent().hostPort(), log);
+            for (Path log : logs) {
+              LogEntry entry = new LogEntry(tls.extent, 0, tls.futureOrCurrent().hostPort(), log.toString());
               m.put(entry.getColumnFamily(), entry.getColumnQualifier(), entry.getValue());
             }
           }
@@ -163,13 +164,13 @@ public class MetaDataStateStore extends TabletStateStore {
   }
 
   @Override
-  public void markLogsAsUnused(AccumuloServerContext context, Map<TServerInstance,List<String>> logs) throws DistributedStoreException {
+  public void markLogsAsUnused(AccumuloServerContext context, Map<TServerInstance,List<Path>> logs) throws DistributedStoreException {
     BatchWriter writer = createBatchWriter();
     try {
-      for (Entry<TServerInstance,List<String>> entry : logs.entrySet()) {
+      for (Entry<TServerInstance,List<Path>> entry : logs.entrySet()) {
         Mutation m = new Mutation(MetadataSchema.CurrentLogsSection.getRowPrefix() + entry.getKey().toString());
-        for (String log : entry.getValue()) {
-          m.put(MetadataSchema.CurrentLogsSection.COLF, new Text(log), MetadataSchema.CurrentLogsSection.UNUSED);
+        for (Path log : entry.getValue()) {
+          m.put(MetadataSchema.CurrentLogsSection.COLF, new Text(log.toString()), MetadataSchema.CurrentLogsSection.UNUSED);
         }
         writer.addMutation(m);
       }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/98c3cef8/server/base/src/main/java/org/apache/accumulo/server/master/state/TabletLocationState.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/state/TabletLocationState.java b/server/base/src/main/java/org/apache/accumulo/server/master/state/TabletLocationState.java
index a222532..3ece3c9 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/state/TabletLocationState.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/state/TabletLocationState.java
@@ -22,6 +22,7 @@ import java.util.Set;
 
 import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.hadoop.io.Text;
+import org.apache.log4j.Logger;
 
 /**
  * When a tablet is assigned, we mark its future location. When the tablet is opened, we set its current location. A tablet should never have both a future and
@@ -32,6 +33,8 @@ import org.apache.hadoop.io.Text;
  */
 public class TabletLocationState {
 
+  private static final Logger log = Logger.getLogger(TabletLocationState.class);
+
   static public class BadLocationStateException extends Exception {
     private static final long serialVersionUID = 1L;
     private Text metadataTableEntry;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/98c3cef8/server/base/src/main/java/org/apache/accumulo/server/master/state/TabletStateStore.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/state/TabletStateStore.java b/server/base/src/main/java/org/apache/accumulo/server/master/state/TabletStateStore.java
index 13db05b..acc10d8 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/state/TabletStateStore.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/state/TabletStateStore.java
@@ -22,6 +22,7 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.accumulo.server.AccumuloServerContext;
+import org.apache.hadoop.fs.Path;
 
 /**
  * Interface for storing information about tablet assignments. There are three implementations:
@@ -61,9 +62,9 @@ public abstract class TabletStateStore implements Iterable<TabletLocationState>
    * @param logsForDeadServers
    *          a cache of logs in use by servers when they died
    */
-  abstract public void unassign(Collection<TabletLocationState> tablets, Map<TServerInstance, List<String>> logsForDeadServers) throws DistributedStoreException;
+  abstract public void unassign(Collection<TabletLocationState> tablets, Map<TServerInstance, List<Path>> logsForDeadServers) throws DistributedStoreException;
 
-  public static void unassign(AccumuloServerContext context, TabletLocationState tls, Map<TServerInstance, List<String>> logsForDeadServers) throws DistributedStoreException {
+  public static void unassign(AccumuloServerContext context, TabletLocationState tls, Map<TServerInstance, List<Path>> logsForDeadServers) throws DistributedStoreException {
     TabletStateStore store;
     if (tls.extent.isRootTablet()) {
       store = new ZooTabletStateStore();
@@ -90,6 +91,6 @@ public abstract class TabletStateStore implements Iterable<TabletLocationState>
   /**
    * When a server fails, its logs must be marked as unused after the log markers are moved to the tablets.
    */
-  abstract public void markLogsAsUnused(AccumuloServerContext context, Map<TServerInstance, List<String>> logs) throws DistributedStoreException;
+  abstract public void markLogsAsUnused(AccumuloServerContext context, Map<TServerInstance, List<Path>> logs) throws DistributedStoreException;
 
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/98c3cef8/server/base/src/main/java/org/apache/accumulo/server/master/state/ZooTabletStateStore.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/state/ZooTabletStateStore.java b/server/base/src/main/java/org/apache/accumulo/server/master/state/ZooTabletStateStore.java
index 66bad4e..eca8e7f 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/state/ZooTabletStateStore.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/state/ZooTabletStateStore.java
@@ -31,6 +31,7 @@ import org.apache.accumulo.core.metadata.schema.MetadataSchema;
 import org.apache.accumulo.core.tabletserver.log.LogEntry;
 import org.apache.accumulo.server.AccumuloServerContext;
 import org.apache.commons.lang.NotImplementedException;
+import org.apache.hadoop.fs.Path;
 import org.apache.log4j.Logger;
 
 import com.google.common.net.HostAndPort;
@@ -163,17 +164,17 @@ public class ZooTabletStateStore extends TabletStateStore {
   }
 
   @Override
-  public void unassign(Collection<TabletLocationState> tablets, Map<TServerInstance, List<String>> logsForDeadServers) throws DistributedStoreException {
+  public void unassign(Collection<TabletLocationState> tablets, Map<TServerInstance, List<Path>> logsForDeadServers) throws DistributedStoreException {
     if (tablets.size() != 1)
       throw new IllegalArgumentException("There is only one root tablet");
     TabletLocationState tls = tablets.iterator().next();
     if (tls.extent.compareTo(RootTable.EXTENT) != 0)
       throw new IllegalArgumentException("You can only store the root tablet location");
     if (logsForDeadServers != null) {
-      List<String> logs = logsForDeadServers.get(tls.futureOrCurrent());
+      List<Path> logs = logsForDeadServers.get(tls.futureOrCurrent());
       if (logs != null) {
-        for (String entry : logs) {
-          LogEntry logEntry = new LogEntry(RootTable.EXTENT, System.currentTimeMillis(), tls.futureOrCurrent().getLocation().toString(), entry);
+        for (Path entry : logs) {
+          LogEntry logEntry = new LogEntry(RootTable.EXTENT, System.currentTimeMillis(), tls.futureOrCurrent().getLocation().toString(), entry.toString());
           byte[] value;
           try {
             value = logEntry.toBytes();
@@ -196,7 +197,7 @@ public class ZooTabletStateStore extends TabletStateStore {
   }
 
   @Override
-  public void markLogsAsUnused(AccumuloServerContext context, Map<TServerInstance,List<String>> logs) {
+  public void markLogsAsUnused(AccumuloServerContext context, Map<TServerInstance,List<Path>> logs) {
     // the root table is not replicated, so unassigning the root tablet has removed the current log marker
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/98c3cef8/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java
----------------------------------------------------------------------
diff --git 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
index ebf4b1b..96f9c9e 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
@@ -89,7 +89,6 @@ import org.apache.accumulo.server.master.state.TServerInstance;
 import org.apache.accumulo.server.tablets.TabletTime;
 import org.apache.accumulo.server.zookeeper.ZooLock;
 import org.apache.accumulo.server.zookeeper.ZooReaderWriter;
-import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
@@ -1061,22 +1060,21 @@ public class MetadataTableUtil {
     return tabletEntries;
   }
 
-  public static void addNewLogMarker(ClientContext context, ZooLock zooLock, final TServerInstance tabletSession, final String filename, KeyExtent extent) {
+  public static void addNewLogMarker(ClientContext context, ZooLock zooLock, final TServerInstance tabletSession, final Path filename, KeyExtent extent) {
     log.debug("Adding log entry " + filename);
     if (extent.isRootTablet()) {
       retryZooKeeperUpdate(context, zooLock, new ZooOperation() {
         @Override
         public void run(IZooReaderWriter rw) throws KeeperException, InterruptedException, IOException {
           String root = ZooUtil.getRoot(HdfsZooInstance.getInstance()) + RootTable.ZROOT_TABLET_CURRENT_LOGS;
-          String[] parts = StringUtils.split(filename, '/');
-          String uniqueId = parts[parts.length - 1];
+          String uniqueId = filename.getName();
           String path = root + "/" + CurrentLogsSection.getRowPrefix() + tabletSession.toString() + uniqueId;
-          rw.putPersistentData(path, filename.getBytes(UTF_8), NodeExistsPolicy.OVERWRITE);
+          rw.putPersistentData(path, filename.toString().getBytes(UTF_8), NodeExistsPolicy.OVERWRITE);
         }
       });
     } else {
       Mutation m = new Mutation(CurrentLogsSection.getRowPrefix() + tabletSession.toString());
-      m.put(CurrentLogsSection.COLF, new Text(filename), new Value(EMPTY_BYTES));
+      m.put(CurrentLogsSection.COLF, new Text(filename.toString()), new Value(EMPTY_BYTES));
       String tableName = MetadataTable.NAME;
       if (extent.isMeta()) {
         tableName = RootTable.NAME;
@@ -1091,13 +1089,12 @@ public class MetadataTableUtil {
     }
   }
 
-  private static void removeCurrentRootLogMarker(ClientContext context, ZooLock zooLock, final TServerInstance tabletSession, final String filename) {
+  private static void removeCurrentRootLogMarker(ClientContext context, ZooLock zooLock, final TServerInstance tabletSession, final Path filename) {
     retryZooKeeperUpdate(context, zooLock, new ZooOperation() {
       @Override
       public void run(IZooReaderWriter rw) throws KeeperException, InterruptedException, IOException {
         String root = ZooUtil.getRoot(HdfsZooInstance.getInstance()) + RootTable.ZROOT_TABLET_CURRENT_LOGS;
-        String[] parts = StringUtils.split(filename, '/');
-        String uniqueId = parts[parts.length - 1];
+        String uniqueId = filename.getName();
         String path = root + "/" + CurrentLogsSection.getRowPrefix() + tabletSession.toString() + uniqueId;
         log.debug("Removing entry " + path + " from zookeeper");
         rw.recursiveDelete(path, NodeMissingPolicy.SKIP);
@@ -1105,12 +1102,12 @@ public class MetadataTableUtil {
     });
   }
 
-  public static void markLogUnused(ClientContext context, ZooLock lock, TServerInstance tabletSession, Set<String> all) throws AccumuloException {
+  public static void markLogUnused(ClientContext context, ZooLock lock, TServerInstance tabletSession, Set<Path> all) throws AccumuloException {
     try {
       BatchWriter root = context.getConnector().createBatchWriter(RootTable.NAME, null);
       BatchWriter meta = context.getConnector().createBatchWriter(MetadataTable.NAME, null);
-      for (String fname : all) {
-        Text tname = new Text(fname.getBytes(UTF_8));
+      for (Path fname : all) {
+        Text tname = new Text(fname.toString());
         Mutation m = new Mutation(MetadataSchema.CurrentLogsSection.getRowPrefix() + tabletSession.toString());
         m.putDelete(MetadataSchema.CurrentLogsSection.COLF, tname);
         root.addMutation(m);
@@ -1127,21 +1124,21 @@ public class MetadataTableUtil {
     }
   }
 
-  public static void fetchLogsForDeadServer(ClientContext context, ZooLock lock, KeyExtent extent, TServerInstance server, Map<TServerInstance,List<String>> logsForDeadServers)
+  public static void fetchLogsForDeadServer(ClientContext context, ZooLock lock, KeyExtent extent, TServerInstance server, Map<TServerInstance,List<Path>> logsForDeadServers)
       throws TableNotFoundException, AccumuloException, AccumuloSecurityException {
     // already cached
     if (logsForDeadServers.containsKey(server)) {
       return;
     }
     if (extent.isRootTablet()) {
-      final List<String> logs = new ArrayList<>();
+      final List<Path> logs = new ArrayList<>();
       retryZooKeeperUpdate(context, lock, new ZooOperation() {
         @Override
         public void run(IZooReaderWriter rw) throws KeeperException, InterruptedException, IOException {
           String root = ZooUtil.getRoot(HdfsZooInstance.getInstance()) + RootTable.ZROOT_TABLET_CURRENT_LOGS;
           logs.clear();
           for (String child : rw.getChildren(root)) {
-            logs.add(new String(rw.getData(root + "/" + child, null), UTF_8));
+            logs.add(new Path(new String(rw.getData(root + "/" + child, null), UTF_8)));
           }
         }
       });
@@ -1155,9 +1152,9 @@ public class MetadataTableUtil {
       // fetch the current logs in use, and put them in the cache
       Scanner scanner = context.getConnector().createScanner(table, Authorizations.EMPTY);
       scanner.setRange(new Range(MetadataSchema.CurrentLogsSection.getRowPrefix() + server.toString()));
-      List<String> logs = new ArrayList<>();
+      List<Path> logs = new ArrayList<>();
       for (Entry<Key,Value> entry : scanner) {
-        logs.add(entry.getKey().getColumnQualifier().toString());
+        logs.add(new Path(entry.getKey().getColumnQualifier().toString()));
       }
       logsForDeadServers.put(server, logs);
     }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/98c3cef8/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 a7703e9..444789b 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
@@ -81,6 +81,7 @@ public class GarbageCollectWriteAheadLogs {
   private final AccumuloServerContext context;
   private final VolumeManager fs;
   private final boolean useTrash;
+  private final LiveTServerSet liveServers;
 
   /**
    * Creates a new GC WAL object.
@@ -96,24 +97,26 @@ public class GarbageCollectWriteAheadLogs {
     this.context = context;
     this.fs = fs;
     this.useTrash = useTrash;
+    this.liveServers = new LiveTServerSet(context, new Listener() {
+      @Override
+      public void update(LiveTServerSet current, Set<TServerInstance> deleted, Set<TServerInstance> added) {
+        log.debug("New tablet servers noticed: " + added);
+        log.debug("Tablet servers removed: " + deleted);
+      }
+    });
+    liveServers.startListeningForTabletServerChanges();
   }
 
   public void collect(GCStatus status) {
 
     Span span = Trace.start("getCandidates");
     try {
-      LiveTServerSet liveServers = new LiveTServerSet(context, new Listener() {
-        @Override
-        public void update(LiveTServerSet current, Set<TServerInstance> deleted, Set<TServerInstance> added) {
-          log.debug("New tablet servers noticed: " + added);
-          log.debug("Tablet servers removed: " + deleted);
-        }
-      });
       Set<TServerInstance> currentServers = liveServers.getCurrentServers();
 
+
       status.currentLog.started = System.currentTimeMillis();
 
-      Map<TServerInstance, Set<String> > candidates = new HashMap<>();
+      Map<TServerInstance, Set<Path> > candidates = new HashMap<>();
       long count = getCurrent(candidates, currentServers);
       long fileScanStop = System.currentTimeMillis();
 
@@ -174,15 +177,15 @@ public class GarbageCollectWriteAheadLogs {
     }
   }
 
-  private long removeMarkers(Map<TServerInstance,Set<String>> candidates) {
+  private long removeMarkers(Map<TServerInstance,Set<Path>> candidates) {
     long result = 0;
     try {
       BatchWriter root = context.getConnector().createBatchWriter(RootTable.NAME, null);
       BatchWriter meta = context.getConnector().createBatchWriter(MetadataTable.NAME, null);
-      for (Entry<TServerInstance,Set<String>> entry : candidates.entrySet()) {
-        Mutation m = new Mutation(CurrentLogsSection.getRowPrefix() + entry.toString());
-        for (String wal : entry.getValue()) {
-          m.putDelete(CurrentLogsSection.COLF, new Text(wal));
+      for (Entry<TServerInstance,Set<Path>> entry : candidates.entrySet()) {
+        Mutation m = new Mutation(CurrentLogsSection.getRowPrefix() + entry.getKey().toString());
+        for (Path path : entry.getValue()) {
+          m.putDelete(CurrentLogsSection.COLF, new Text(path.toString()));
           result++;
         }
         root.addMutation(m);
@@ -196,11 +199,10 @@ public class GarbageCollectWriteAheadLogs {
     return result;
   }
 
-  private long removeFiles(Map<TServerInstance, Set<String> > candidates, final GCStatus status) {
-    for (Entry<TServerInstance,Set<String>> entry : candidates.entrySet()) {
-      for (String walog : entry.getValue()) {
-        log.debug("Removing WAL for offline server " + entry.getKey() + " log " + walog);
-        Path path = new Path(walog);
+  private long removeFiles(Map<TServerInstance, Set<Path> > candidates, final GCStatus status) {
+    for (Entry<TServerInstance,Set<Path>> entry : candidates.entrySet()) {
+      for (Path path : entry.getValue()) {
+        log.debug("Removing unused WAL for server " + entry.getKey() + " log " + path);
         try {
           if (!useTrash || !fs.moveToTrash(path))
             fs.deleteRecursively(path);
@@ -215,14 +217,14 @@ public class GarbageCollectWriteAheadLogs {
     return status.currentLog.deleted;
   }
 
-  private long removeMetadataEntries(Map<TServerInstance, Set<String> > candidates, GCStatus status, Set<TServerInstance> liveServers) throws IOException, KeeperException,
+  private long removeMetadataEntries(Map<TServerInstance, Set<Path> > candidates, GCStatus status, Set<TServerInstance> liveServers) throws IOException, KeeperException,
       InterruptedException {
 
     // remove any entries if there's a log reference, or a tablet is still assigned to the dead server
 
-    Map<String, TServerInstance> walToDeadServer = new HashMap<>();
-    for (Entry<TServerInstance,Set<String>> entry : candidates.entrySet()) {
-      for (String file : entry.getValue()) {
+    Map<Path, TServerInstance> walToDeadServer = new HashMap<>();
+    for (Entry<TServerInstance,Set<Path>> entry : candidates.entrySet()) {
+      for (Path file : entry.getValue()) {
         walToDeadServer.put(file, entry.getKey());
       }
     }
@@ -248,7 +250,7 @@ public class GarbageCollectWriteAheadLogs {
     return count;
   }
 
-  protected int removeReplicationEntries(Map<TServerInstance, Set<String> > candidates, GCStatus status) throws IOException, KeeperException,
+  protected int removeReplicationEntries(Map<TServerInstance, Set<Path> > candidates, GCStatus status) throws IOException, KeeperException,
   InterruptedException {
     Connector conn;
     try {
@@ -260,13 +262,13 @@ public class GarbageCollectWriteAheadLogs {
 
     int count = 0;
 
-    Iterator<Entry<TServerInstance,Set<String>>> walIter = candidates.entrySet().iterator();
+    Iterator<Entry<TServerInstance,Set<Path>>> walIter = candidates.entrySet().iterator();
 
     while (walIter.hasNext()) {
-      Entry<TServerInstance,Set<String>> wal = walIter.next();
-      Iterator<String> paths = wal.getValue().iterator();
+      Entry<TServerInstance,Set<Path>> wal = walIter.next();
+      Iterator<Path> paths = wal.getValue().iterator();
       while (paths.hasNext()) {
-        String fullPath = paths.next();
+        Path fullPath = paths.next();
         if (neededByReplication(conn, fullPath)) {
           log.debug("Removing WAL from candidate deletion as it is still needed for replication: {}", fullPath);
           // If we haven't already removed it, check to see if this WAL is
@@ -294,7 +296,7 @@ public class GarbageCollectWriteAheadLogs {
    *          The full path (URI)
    * @return True if the WAL is still needed by replication (not a candidate for deletion)
    */
-  protected boolean neededByReplication(Connector conn, String wal) {
+  protected boolean neededByReplication(Connector conn, Path wal) {
     log.info("Checking replication table for " + wal);
 
     Iterable<Entry<Key,Value>> iter = getReplicationStatusForFile(conn, wal);
@@ -317,7 +319,7 @@ public class GarbageCollectWriteAheadLogs {
     return false;
   }
 
-  protected Iterable<Entry<Key,Value>> getReplicationStatusForFile(Connector conn, String wal) {
+  protected Iterable<Entry<Key,Value>> getReplicationStatusForFile(Connector conn, Path wal) {
     Scanner metaScanner;
     try {
       metaScanner = conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
@@ -337,7 +339,7 @@ public class GarbageCollectWriteAheadLogs {
       StatusSection.limit(replScanner);
 
       // Only look for this specific WAL
-      replScanner.setRange(Range.exact(wal));
+      replScanner.setRange(Range.exact(wal.toString()));
 
       return Iterables.concat(metaScanner, replScanner);
     } catch (ReplicationTableOfflineException e) {
@@ -353,19 +355,19 @@ public class GarbageCollectWriteAheadLogs {
   /**
    * Scans log markers. The map passed in is populated with the logs for dead servers.
    *
-   * @param logsForDeadServers
+   * @param unusedLogs
    *          map of dead server to log file entries
    * @return total number of log files
    */
-  private long getCurrent(Map<TServerInstance, Set<String> > logsForDeadServers, Set<TServerInstance> currentServers) throws Exception {
-    Set<String> rootWALs = new HashSet<String>();
+  private long getCurrent(Map<TServerInstance, Set<Path> > unusedLogs, Set<TServerInstance> currentServers) throws Exception {
+    Set<Path> rootWALs = new HashSet<>();
     // Get entries in zookeeper:
     String zpath = ZooUtil.getRoot(context.getInstance()) + RootTable.ZROOT_TABLET_WALOGS;
     ZooReaderWriter zoo = ZooReaderWriter.getInstance();
     List<String> children = zoo.getChildren(zpath);
     for (String child : children) {
       LogEntry entry = LogEntry.fromBytes(zoo.getData(zpath + "/" + child, null));
-      rootWALs.add(entry.filename);
+      rootWALs.add(new Path(entry.filename));
     }
     long count = 0;
 
@@ -383,12 +385,13 @@ public class GarbageCollectWriteAheadLogs {
       CurrentLogsSection.getTabletServer(entry.getKey(), hostAndPort, sessionId);
       CurrentLogsSection.getPath(entry.getKey(), filename);
       TServerInstance tsi = new TServerInstance(HostAndPort.fromString(hostAndPort.toString()), sessionId.toString());
-      if ((!currentServers.contains(tsi) || entry.getValue().equals(CurrentLogsSection.UNUSED)) && !rootWALs.contains(filename)) {
-        Set<String> logs = logsForDeadServers.get(tsi);
+      Path path = new Path(filename.toString());
+      if ((!currentServers.contains(tsi) || (entry.getValue().equals(CurrentLogsSection.UNUSED)) && !rootWALs.contains(path))) {
+        Set<Path> logs = unusedLogs.get(tsi);
         if (logs == null) {
-          logsForDeadServers.put(tsi, logs = new HashSet<String>());
+          unusedLogs.put(tsi, logs = new HashSet<Path>());
         }
-        if (logs.add(new Path(filename.toString()).toString())) {
+        if (logs.add(path)) {
           count++;
         }
       }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/98c3cef8/server/master/src/main/java/org/apache/accumulo/master/TabletGroupWatcher.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/TabletGroupWatcher.java b/server/master/src/main/java/org/apache/accumulo/master/TabletGroupWatcher.java
index edea93f..4d237a1 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/TabletGroupWatcher.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/TabletGroupWatcher.java
@@ -161,7 +161,7 @@ class TabletGroupWatcher extends Daemon {
         List<Assignment> assigned = new ArrayList<Assignment>();
         List<TabletLocationState> assignedToDeadServers = new ArrayList<TabletLocationState>();
         Map<KeyExtent,TServerInstance> unassigned = new HashMap<KeyExtent,TServerInstance>();
-        Map<TServerInstance, List<String>> logsForDeadServers = new TreeMap<>();
+        Map<TServerInstance, List<Path>> logsForDeadServers = new TreeMap<>();
 
         MasterState masterState = master.getMasterState();
         int[] counts = new int[TabletState.values().length];
@@ -204,8 +204,9 @@ class TabletGroupWatcher extends Daemon {
           TabletGoalState goal = this.master.getGoalState(tls, mergeStats.getMergeInfo());
           TServerInstance server = tls.getServer();
           TabletState state = tls.getState(currentTServers.keySet());
-          if (Master.log.isTraceEnabled())
-            Master.log.trace("Goal state " + goal + " current " + state);
+          if (Master.log.isTraceEnabled()) {
+            Master.log.trace("Goal state " + goal + " current " + state + " for " + tls.extent);
+          }
           stats.update(tableId, state);
           mergeStats.update(tls.extent, state, tls.chopped, !tls.walogs.isEmpty());
           sendChopRequest(mergeStats.getMergeInfo(), state, tls);
@@ -308,8 +309,10 @@ class TabletGroupWatcher extends Daemon {
 
         updateMergeState(mergeStatsCache);
 
-        Master.log.debug(String.format("[%s] sleeping for %.2f seconds", store.name(), Master.TIME_TO_WAIT_BETWEEN_SCANS / 1000.));
-        eventListener.waitForEvents(Master.TIME_TO_WAIT_BETWEEN_SCANS);
+        if (this.master.tserverSet.getCurrentServers().equals(currentTServers.keySet())) {
+          Master.log.debug(String.format("[%s] sleeping for %.2f seconds", store.name(), Master.TIME_TO_WAIT_BETWEEN_SCANS / 1000.));
+          eventListener.waitForEvents(Master.TIME_TO_WAIT_BETWEEN_SCANS);
+        }
       } catch (Exception ex) {
         Master.log.error("Error processing table state for store " + store.name(), ex);
         if (ex.getCause() != null && ex.getCause() instanceof BadLocationStateException) {
@@ -731,7 +734,7 @@ class TabletGroupWatcher extends Daemon {
       List<Assignment> assignments,
       List<Assignment> assigned,
       List<TabletLocationState> assignedToDeadServers,
-      Map<TServerInstance, List<String>> logsForDeadServers,
+      Map<TServerInstance, List<Path>> logsForDeadServers,
       Map<KeyExtent,TServerInstance> unassigned)
           throws DistributedStoreException, TException {
     if (!assignedToDeadServers.isEmpty()) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/98c3cef8/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
index 3b7ff03..ffc1c2a 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
@@ -3000,9 +3000,9 @@ public class TabletServer extends AccumuloServerContext implements Runnable {
       candidates.removeAll(tablet.getCurrentLogFiles());
     }
     try {
-      Set<String> filenames = new HashSet<>();
+      Set<Path> filenames = new HashSet<>();
       for (DfsLogger candidate : candidates) {
-        filenames.add(candidate.getFileName());
+        filenames.add(candidate.getPath());
       }
       MetadataTableUtil.markLogUnused(this, this.getLock(), this.getTabletSession(), filenames);
       synchronized (closedLogs) {
@@ -3019,7 +3019,7 @@ public class TabletServer extends AccumuloServerContext implements Runnable {
       EnumSet<TabletLevel> set = metadataTableLogs.putIfAbsent(copy, EnumSet.of(level));
       if (set == null || !set.contains(level) || level == TabletLevel.ROOT) {
         log.info("Writing log marker for level " + level + " " + copy.getFileName());
-        MetadataTableUtil.addNewLogMarker(this, this.getLock(), this.getTabletSession(), copy.getFileName(), extent);
+        MetadataTableUtil.addNewLogMarker(this, this.getLock(), this.getTabletSession(), copy.getPath(), extent);
         if (set != null) {
           set.add(level);
         }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/98c3cef8/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
index f8bcfbc..e256604 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java
@@ -475,7 +475,11 @@ public class DfsLogger implements Comparable<DfsLogger> {
   }
 
   public String getFileName() {
-    return logPath.toString();
+    return logPath;
+  }
+
+  public Path getPath() {
+    return new Path(logPath);
   }
 
   public void close() throws IOException {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/98c3cef8/test/src/test/java/org/apache/accumulo/test/functional/WALSunnyDayIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/WALSunnyDayIT.java b/test/src/test/java/org/apache/accumulo/test/functional/WALSunnyDayIT.java
new file mode 100644
index 0000000..b8e36bc
--- /dev/null
+++ b/test/src/test/java/org/apache/accumulo/test/functional/WALSunnyDayIT.java
@@ -0,0 +1,240 @@
+/*
+ * 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.test.functional;
+
+import static org.apache.accumulo.core.conf.Property.GC_CYCLE_DELAY;
+import static org.apache.accumulo.core.conf.Property.GC_CYCLE_START;
+import static org.apache.accumulo.core.conf.Property.INSTANCE_ZK_TIMEOUT;
+import static org.apache.accumulo.core.conf.Property.TSERV_WALOG_MAX_SIZE;
+import static org.apache.accumulo.core.conf.Property.TSERV_WAL_REPLICATION;
+import static org.apache.accumulo.core.security.Authorizations.EMPTY;
+import static org.apache.accumulo.minicluster.ServerType.GARBAGE_COLLECTOR;
+import static org.apache.accumulo.minicluster.ServerType.TABLET_SERVER;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Random;
+
+import org.apache.accumulo.core.client.BatchWriter;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.Scanner;
+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.Value;
+import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.RootTable;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.CurrentLogsSection;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
+import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.TabletColumnFamily;
+import org.apache.accumulo.core.util.UtilWaitThread;
+import org.apache.accumulo.core.zookeeper.ZooUtil;
+import org.apache.accumulo.master.state.SetGoalState;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloClusterControl;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloClusterImpl;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.RawLocalFileSystem;
+import org.apache.hadoop.io.Text;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.ZooKeeper;
+import org.junit.Test;
+
+import com.google.common.collect.Iterators;
+
+public class WALSunnyDayIT extends ConfigurableMacIT {
+
+  private static final Text CF = new Text(new byte[0]);
+
+  @Override
+  protected void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
+    cfg.setProperty(GC_CYCLE_DELAY, "1s");
+    cfg.setProperty(GC_CYCLE_START, "0s");
+    cfg.setProperty(TSERV_WALOG_MAX_SIZE, "1M");
+    cfg.setProperty(TSERV_WAL_REPLICATION, "1");
+    cfg.setProperty(INSTANCE_ZK_TIMEOUT, "3s");
+    cfg.setNumTservers(1);
+    hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
+  }
+
+  int countTrue(Collection<Boolean> bools) {
+    int result = 0;
+    for (Boolean b : bools) {
+      if (b.booleanValue())
+        result ++;
+    }
+    return result;
+  }
+
+  @Test
+  public void test() throws Exception {
+    MiniAccumuloClusterImpl mac = getCluster();
+    MiniAccumuloClusterControl control = mac.getClusterControl();
+    control.stop(GARBAGE_COLLECTOR);
+    Connector c = getConnector();
+    ZooKeeper zoo = new ZooKeeper(c.getInstance().getZooKeepers(), c.getInstance().getZooKeepersSessionTimeOut(), new Watcher() {
+      @Override
+      public void process(WatchedEvent event) {
+        log.info(event.toString());
+      }
+    });
+    String tableName = getUniqueNames(1)[0];
+    c.tableOperations().create(tableName);
+    writeSomeData(c, tableName, 1, 1);
+
+    // should have two markers: wal in use, and next wal
+    Map<String,Boolean> wals = getWals(c, zoo);
+    assertEquals(wals.toString(), 2, wals.size());
+    for (Boolean b : wals.values()) {
+      assertTrue("logs should be in use", b.booleanValue());
+    }
+
+    // roll log, get a new next
+    writeSomeData(c, tableName, 1000, 50);
+    Map<String,Boolean> walsAfterRoll = getWals(c, zoo);
+    assertEquals("should have 3 WALs after roll", 3, walsAfterRoll.size());
+    assertTrue("new WALs should be a superset of the old WALs", walsAfterRoll.keySet().containsAll(wals.keySet()));
+    assertEquals("all WALs should be in use", 3, countTrue(walsAfterRoll.values()));
+
+    // flush the tables
+    for (String table: new String[] { tableName, MetadataTable.NAME, RootTable.NAME} ) {
+      c.tableOperations().flush(table, null, null, true);
+    }
+    // rolled WAL is no longer in use, but needs to be GC'd
+    Map<String,Boolean> walsAfterflush = getWals(c, zoo);
+    assertEquals(walsAfterflush.toString(), 3, walsAfterflush.size());
+    assertEquals("inUse should be 2", 2, countTrue(walsAfterflush.values()));
+
+    // let the GC run for a little bit
+    control.start(GARBAGE_COLLECTOR);
+    UtilWaitThread.sleep(5 * 1000);
+    // make sure the unused WAL goes away
+    Map<String,Boolean> walsAfterGC = getWals(c, zoo);
+    assertEquals(walsAfterGC.toString(), 2, walsAfterGC.size());
+    control.stop(GARBAGE_COLLECTOR);
+    // restart the tserver, but don't run recovery on all tablets
+    control.stop(TABLET_SERVER);
+    // this delays recovery on the normal tables
+    assertEquals(0, cluster.exec(SetGoalState.class, "SAFE_MODE").waitFor());
+    control.start(TABLET_SERVER);
+
+    // wait for the metadata table to go back online
+    getRecoveryMarkers(c);
+    // allow a little time for the master to notice ASSIGNED_TO_DEAD_SERVER tablets
+    UtilWaitThread.sleep(5 * 1000);
+    Map<KeyExtent,List<String>> markers = getRecoveryMarkers(c);
+    //log.debug("markers " + markers);
+    assertEquals("one tablet should have markers", 1, markers.keySet().size());
+    assertEquals("tableId of the keyExtent should be 1", markers.keySet().iterator().next().getTableId(), new Text("1"));
+
+    // put some data in the WAL
+    assertEquals(0, cluster.exec(SetGoalState.class, "NORMAL").waitFor());
+    writeSomeData(c, tableName, 100, 100);
+
+    Map<String,Boolean> walsAfterRestart = getWals(c, zoo);
+    //log.debug("wals after " + walsAfterRestart);
+    assertEquals("used WALs after restart should be 2", 2, countTrue(walsAfterRestart.values()));
+    control.start(GARBAGE_COLLECTOR);
+    UtilWaitThread.sleep(5 * 1000);
+    Map<String,Boolean> walsAfterRestartAndGC = getWals(c, zoo);
+    assertEquals("wals left should be 2", 2, walsAfterRestartAndGC.size());
+    assertEquals("logs in use should be 2", 2, countTrue(walsAfterRestartAndGC.values()));
+  }
+
+  private void writeSomeData(Connector conn, String tableName, int row, int col) throws Exception {
+    Random rand = new Random();
+    BatchWriter bw = conn.createBatchWriter(tableName, null);
+    byte[] rowData = new byte[10];
+    byte[] cq = new byte[10];
+    byte[] value = new byte[10];
+
+    for (int r = 0; r < row; r++) {
+      rand.nextBytes(rowData);
+      Mutation m = new Mutation(rowData);
+      for (int c = 0; c < col; c++) {
+        rand.nextBytes(cq);
+        rand.nextBytes(value);
+        m.put(CF, new Text(cq), new Value(value));
+      }
+      bw.addMutation(m);
+      if (r % 100 == 0) {
+        bw.flush();
+      }
+    }
+    bw.close();
+  }
+
+  private Map<String, Boolean> getWals(Connector c, ZooKeeper zoo) throws Exception {
+    Map<String, Boolean> result = new HashMap<>();
+    Scanner root = c.createScanner(RootTable.NAME, EMPTY);
+    root.setRange(CurrentLogsSection.getRange());
+    Scanner meta = c.createScanner(MetadataTable.NAME, EMPTY);
+    meta.setRange(root.getRange());
+    Iterator<Entry<Key,Value>> both = Iterators.concat(root.iterator(), meta.iterator());
+    while (both.hasNext()) {
+      Entry<Key,Value> entry = both.next();
+      Text path = new Text();
+      CurrentLogsSection.getPath(entry.getKey(), path);
+      result.put(path.toString(), entry.getValue().get().length == 0);
+    }
+    String zpath = ZooUtil.getRoot(c.getInstance()) + RootTable.ZROOT_TABLET_CURRENT_LOGS;
+    List<String> children = zoo.getChildren(zpath, null);
+    for (String child : children) {
+      byte[] data = zoo.getData(zpath + "/" + child, null, null);
+      result.put(new String(data), true);
+    }
+    return result;
+  }
+
+  private Map<KeyExtent, List<String>> getRecoveryMarkers(Connector c) throws Exception {
+    Map<KeyExtent, List<String>> result = new HashMap<>();
+    Scanner root = c.createScanner(RootTable.NAME, EMPTY);
+    root.setRange(TabletsSection.getRange());
+    root.fetchColumnFamily(TabletsSection.LogColumnFamily.NAME);
+    TabletColumnFamily.PREV_ROW_COLUMN.fetch(root);
+
+    Scanner meta = c.createScanner(MetadataTable.NAME, EMPTY);
+    meta.setRange(TabletsSection.getRange());
+    meta.fetchColumnFamily(TabletsSection.LogColumnFamily.NAME);
+    TabletColumnFamily.PREV_ROW_COLUMN.fetch(meta);
+
+    List<String> logs = new ArrayList<>();
+    Iterator<Entry<Key,Value>> both = Iterators.concat(root.iterator(), meta.iterator());
+    while (both.hasNext()) {
+      Entry<Key,Value> entry = both.next();
+      Key key = entry.getKey();
+      if (key.getColumnFamily().equals(TabletsSection.LogColumnFamily.NAME)) {
+        logs.add(key.getColumnQualifier().toString());
+      }
+      if (TabletColumnFamily.PREV_ROW_COLUMN.hasColumns(key) && !logs.isEmpty()) {
+        KeyExtent extent = new KeyExtent(key.getRow(), entry.getValue());
+        result.put(extent, logs);
+        logs = new ArrayList<String>();
+      }
+    }
+    return result;
+  }
+
+}


[25/34] accumulo git commit: ACCUMULO-3423 finished up testing, went back to lazy metadata table references

Posted by ec...@apache.org.
ACCUMULO-3423 finished up testing, went back to lazy metadata table references


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

Branch: refs/heads/master
Commit: f43b216038157e471612eac4679696ce7d6f7835
Parents: ed7c4f6
Author: Eric C. Newton <er...@gmail.com>
Authored: Wed Apr 22 10:34:02 2015 -0400
Committer: Eric C. Newton <er...@gmail.com>
Committed: Wed Apr 22 10:34:02 2015 -0400

----------------------------------------------------------------------
 .../tserver/log/TabletServerLogger.java         | 24 +++++++++++++++-----
 .../accumulo/proxy/ProxyDurabilityIT.java       |  4 ++--
 .../org/apache/accumulo/test/UnusedWALIT.java   |  2 +-
 .../accumulo/test/functional/WALSunnyDayIT.java | 20 ++++++++--------
 4 files changed, 31 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/f43b2160/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
index cdee51b..7d16aae 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/TabletServerLogger.java
@@ -78,7 +78,7 @@ public class TabletServerLogger {
 
   // The current logger
   private DfsLogger currentLog = null;
-  private final SynchronousQueue<DfsLogger> nextLog = new SynchronousQueue<>();
+  private final SynchronousQueue<Object> nextLog = new SynchronousQueue<>();
   private ThreadPoolExecutor nextLogMaker;
 
   // The current generation of logs.
@@ -200,11 +200,18 @@ public class TabletServerLogger {
 
     try {
       startLogMaker();
-      DfsLogger next = nextLog.take();
-      log.info("Using next log " + next.getFileName());
-      currentLog = next;
-      logId.incrementAndGet();
-      return;
+      Object next = nextLog.take();
+      if (next instanceof Exception) {
+        throw (Exception)next;
+      }
+      if (next instanceof DfsLogger) {
+        currentLog = (DfsLogger)next;
+        logId.incrementAndGet();
+        log.info("Using next log " + currentLog.getFileName());
+        return;
+      } else {
+        throw new RuntimeException("Error: unexpected type seen: " + next);
+      }
     } catch (Exception t) {
       walErrors.put(System.currentTimeMillis(), "");
       if (walErrors.size() >= HALT_AFTER_ERROR_COUNT) {
@@ -233,6 +240,11 @@ public class TabletServerLogger {
             }
           } catch (Exception t) {
             log.error("{}", t.getMessage(), t);
+            try {
+              nextLog.offer(t, 12, TimeUnit.HOURS);
+            } catch (InterruptedException ex) {
+              // ignore
+            }
           }
         }
       }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f43b2160/test/src/test/java/org/apache/accumulo/proxy/ProxyDurabilityIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/proxy/ProxyDurabilityIT.java b/test/src/test/java/org/apache/accumulo/proxy/ProxyDurabilityIT.java
index 654d27d..81e25cc 100644
--- a/test/src/test/java/org/apache/accumulo/proxy/ProxyDurabilityIT.java
+++ b/test/src/test/java/org/apache/accumulo/proxy/ProxyDurabilityIT.java
@@ -116,7 +116,7 @@ public class ProxyDurabilityIT extends ConfigurableMacIT {
     assertEquals(0, count(tableName));
 
     ConditionalWriterOptions cfg = new ConditionalWriterOptions();
-    cfg.setDurability(Durability.LOG);
+    cfg.setDurability(Durability.SYNC);
     String cwriter = client.createConditionalWriter(login, tableName, cfg);
     ConditionalUpdates updates = new ConditionalUpdates();
     updates.addToConditions(new Condition(new Column(bytes("cf"), bytes("cq"), bytes(""))));
@@ -125,7 +125,7 @@ public class ProxyDurabilityIT extends ConfigurableMacIT {
     assertEquals(ConditionalStatus.ACCEPTED, status.get(bytes("row")));
     assertEquals(1, count(tableName));
     restartTServer();
-    assertEquals(0, count(tableName));
+    assertEquals(1, count(tableName));
 
     proxyServer.stop();
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f43b2160/test/src/test/java/org/apache/accumulo/test/UnusedWALIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/UnusedWALIT.java b/test/src/test/java/org/apache/accumulo/test/UnusedWALIT.java
index 3684ee1..03d783c 100644
--- a/test/src/test/java/org/apache/accumulo/test/UnusedWALIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/UnusedWALIT.java
@@ -80,7 +80,7 @@ public class UnusedWALIT extends ConfigurableMacIT {
 
     // roll the logs by pushing data into bigTable
     writeSomeData(c, bigTable, 0, 3000, 0, 1000);
-    assertEquals(3, getWALCount(c));
+    assertEquals(2, getWALCount(c));
 
     // put some data in the latest log
     writeSomeData(c, lilTable, 1, 10, 0, 10);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f43b2160/test/src/test/java/org/apache/accumulo/test/functional/WALSunnyDayIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/WALSunnyDayIT.java b/test/src/test/java/org/apache/accumulo/test/functional/WALSunnyDayIT.java
index 3f51c8d..490bd7c 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/WALSunnyDayIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/WALSunnyDayIT.java
@@ -105,9 +105,9 @@ public class WALSunnyDayIT extends ConfigurableMacIT {
     c.tableOperations().create(tableName);
     writeSomeData(c, tableName, 1, 1);
 
-    // should have two markers: wal in use, and next wal
+    // wal markers are added lazily
     Map<String,Boolean> wals = getWals(c, zoo);
-    assertEquals(wals.toString(), 2, wals.size());
+    assertEquals(wals.toString(), 1, wals.size());
     for (Boolean b : wals.values()) {
       assertTrue("logs should be in use", b.booleanValue());
     }
@@ -115,9 +115,9 @@ public class WALSunnyDayIT extends ConfigurableMacIT {
     // roll log, get a new next
     writeSomeData(c, tableName, 1000, 50);
     Map<String,Boolean> walsAfterRoll = getWals(c, zoo);
-    assertEquals("should have 3 WALs after roll", 3, walsAfterRoll.size());
+    assertEquals("should have 3 WALs after roll", 2, walsAfterRoll.size());
     assertTrue("new WALs should be a superset of the old WALs", walsAfterRoll.keySet().containsAll(wals.keySet()));
-    assertEquals("all WALs should be in use", 3, countTrue(walsAfterRoll.values()));
+    assertEquals("all WALs should be in use", 2, countTrue(walsAfterRoll.values()));
 
     // flush the tables
     for (String table: new String[] { tableName, MetadataTable.NAME, RootTable.NAME} ) {
@@ -126,15 +126,15 @@ public class WALSunnyDayIT extends ConfigurableMacIT {
     UtilWaitThread.sleep(1000);
     // rolled WAL is no longer in use, but needs to be GC'd
     Map<String,Boolean> walsAfterflush = getWals(c, zoo);
-    assertEquals(walsAfterflush.toString(), 3, walsAfterflush.size());
-    assertEquals("inUse should be 2", 2, countTrue(walsAfterflush.values()));
+    assertEquals(walsAfterflush.toString(), 2, walsAfterflush.size());
+    assertEquals("inUse should be 1", 1, countTrue(walsAfterflush.values()));
 
     // let the GC run for a little bit
     control.start(GARBAGE_COLLECTOR);
     UtilWaitThread.sleep(5 * 1000);
     // make sure the unused WAL goes away
     Map<String,Boolean> walsAfterGC = getWals(c, zoo);
-    assertEquals(walsAfterGC.toString(), 2, walsAfterGC.size());
+    assertEquals(walsAfterGC.toString(), 1, walsAfterGC.size());
     control.stop(GARBAGE_COLLECTOR);
     // restart the tserver, but don't run recovery on all tablets
     control.stop(TABLET_SERVER);
@@ -158,12 +158,12 @@ public class WALSunnyDayIT extends ConfigurableMacIT {
 
     Map<String,Boolean> walsAfterRestart = getWals(c, zoo);
     //log.debug("wals after " + walsAfterRestart);
-    assertEquals("used WALs after restart should be 2", 2, countTrue(walsAfterRestart.values()));
+    assertEquals("used WALs after restart should be 1", 1, countTrue(walsAfterRestart.values()));
     control.start(GARBAGE_COLLECTOR);
     UtilWaitThread.sleep(5 * 1000);
     Map<String,Boolean> walsAfterRestartAndGC = getWals(c, zoo);
-    assertEquals("wals left should be 2", 2, walsAfterRestartAndGC.size());
-    assertEquals("logs in use should be 2", 2, countTrue(walsAfterRestartAndGC.values()));
+    assertEquals("wals left should be 1", 1, walsAfterRestartAndGC.size());
+    assertEquals("logs in use should be 1", 1, countTrue(walsAfterRestartAndGC.values()));
   }
 
   private void verifySomeData(Connector c, String tableName, int expected) throws Exception {


[16/34] accumulo git commit: ACCUMULO-3423 merge in master from apache

Posted by ec...@apache.org.
ACCUMULO-3423 merge in master from apache


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

Branch: refs/heads/master
Commit: c71fc121e70848170f3a3a69c1bb4a0b1b77b24a
Parents: afa887b 94bd393
Author: Eric C. Newton <er...@gmail.com>
Authored: Mon Mar 30 11:38:07 2015 -0400
Committer: Eric C. Newton <er...@gmail.com>
Committed: Mon Mar 30 11:38:07 2015 -0400

----------------------------------------------------------------------
 .../accumulo/core/trace/DistributedTrace.java   |   1 +
 .../impl/MiniAccumuloClusterImpl.java           |  10 +-
 .../server/security/SecurityOperation.java      |  15 +-
 .../accumulo/master/TabletGroupWatcher.java     |   6 +
 .../master/tserverOps/ShutdownTServer.java      |  51 ++++---
 .../accumulo/tracer/AsyncSpanReceiver.java      |  13 ++
 .../accumulo/tracer/SendSpansViaThrift.java     |   5 +
 .../server/GarbageCollectionLogger.java         |   2 +-
 .../apache/accumulo/tserver/TabletServer.java   |   2 +-
 .../org/apache/accumulo/test/UnusedWALIT.java   | 144 -------------------
 .../apache/accumulo/proxy/SimpleProxyBase.java  |   8 +-
 .../accumulo/test/AssignmentThreadsIT.java      |  10 +-
 .../test/BalanceWithOfflineTableIT.java         |  90 ++++++++++++
 .../org/apache/accumulo/test/UnusedWALIT.java   | 144 +++++++++++++++++++
 .../accumulo/test/functional/ClassLoaderIT.java |   4 +-
 .../accumulo/test/functional/DurabilityIT.java  |   2 -
 .../test/functional/KerberosProxyIT.java        |  14 +-
 .../test/functional/WatchTheWatchCountIT.java   |   4 +-
 .../test/replication/ReplicationIT.java         |   4 +-
 19 files changed, 346 insertions(+), 183 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/c71fc121/server/master/src/main/java/org/apache/accumulo/master/TabletGroupWatcher.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c71fc121/server/tserver/src/main/java/org/apache/accumulo/server/GarbageCollectionLogger.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c71fc121/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c71fc121/test/src/test/java/org/apache/accumulo/test/UnusedWALIT.java
----------------------------------------------------------------------
diff --cc test/src/test/java/org/apache/accumulo/test/UnusedWALIT.java
index 0000000,0000000..3684ee1
new file mode 100644
--- /dev/null
+++ b/test/src/test/java/org/apache/accumulo/test/UnusedWALIT.java
@@@ -1,0 -1,0 +1,144 @@@
++/*
++ * 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.test;
++
++import static org.junit.Assert.assertEquals;
++
++import java.util.Map.Entry;
++
++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.Scanner;
++import org.apache.accumulo.core.conf.Property;
++import org.apache.accumulo.core.data.Key;
++import org.apache.accumulo.core.data.Mutation;
++import org.apache.accumulo.core.data.Range;
++import org.apache.accumulo.core.data.Value;
++import org.apache.accumulo.core.metadata.MetadataTable;
++import org.apache.accumulo.core.metadata.schema.MetadataSchema.CurrentLogsSection;
++import org.apache.accumulo.core.security.Authorizations;
++import org.apache.accumulo.minicluster.ServerType;
++import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
++import org.apache.accumulo.test.functional.ConfigurableMacIT;
++import org.apache.hadoop.conf.Configuration;
++import org.apache.hadoop.fs.RawLocalFileSystem;
++import org.junit.Test;
++
++import com.google.common.collect.Iterators;
++
++// When reviewing the changes for ACCUMULO-3423, kturner suggested
++// "tablets will now have log references that contain no data,
++// so it may be marked with 3 WALs, the first with data, the 2nd without, a 3rd with data.
++// It would be useful to have an IT that will test this situation.
++public class UnusedWALIT extends ConfigurableMacIT {
++
++  @Override
++  protected void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
++    final long logSize = 1024 * 1024 * 10;
++    cfg.setProperty(Property.INSTANCE_ZK_TIMEOUT, "5s");
++    cfg.setProperty(Property.TSERV_WALOG_MAX_SIZE, Long.toString(logSize));
++    cfg.setNumTservers(1);
++    // use raw local file system so walogs sync and flush will work
++    hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
++    hadoopCoreSite.set("fs.namenode.fs-limits.min-block-size", Long.toString(logSize));
++  }
++
++  @Test(timeout = 2 * 60 * 1000)
++  public void test() throws Exception {
++    // don't want this bad boy cleaning up walog entries
++    getCluster().getClusterControl().stop(ServerType.GARBAGE_COLLECTOR);
++
++    // make two tables
++    String[] tableNames = getUniqueNames(2);
++    String bigTable = tableNames[0];
++    String lilTable = tableNames[1];
++    Connector c = getConnector();
++    c.tableOperations().create(bigTable);
++    c.tableOperations().create(lilTable);
++
++    // put some data in a log that should be replayed for both tables
++    writeSomeData(c, bigTable, 0, 10, 0, 10);
++    scanSomeData(c, bigTable, 0, 10, 0, 10);
++    writeSomeData(c, lilTable, 0, 1, 0, 1);
++    scanSomeData(c, lilTable, 0, 1, 0, 1);
++    assertEquals(1, getWALCount(c));
++
++    // roll the logs by pushing data into bigTable
++    writeSomeData(c, bigTable, 0, 3000, 0, 1000);
++    assertEquals(3, getWALCount(c));
++
++    // put some data in the latest log
++    writeSomeData(c, lilTable, 1, 10, 0, 10);
++    scanSomeData(c, lilTable, 1, 10, 0, 10);
++
++    // bounce the tserver
++    getCluster().getClusterControl().stop(ServerType.TABLET_SERVER);
++    getCluster().getClusterControl().start(ServerType.TABLET_SERVER);
++
++    // wait for the metadata table to be online
++    Iterators.size(c.createScanner(MetadataTable.NAME, Authorizations.EMPTY).iterator());
++
++    // check our two sets of data in different logs
++    scanSomeData(c, lilTable, 0, 1, 0, 1);
++    scanSomeData(c, lilTable, 1, 10, 0, 10);
++  }
++
++  private void scanSomeData(Connector c, String table, int startRow, int rowCount, int startCol, int colCount) throws Exception {
++    Scanner s = c.createScanner(table, Authorizations.EMPTY);
++    s.setRange(new Range(Integer.toHexString(startRow), Integer.toHexString(startRow + rowCount)));
++    int row = startRow;
++    int col = startCol;
++    for (Entry<Key,Value> entry : s) {
++      assertEquals(row, Integer.parseInt(entry.getKey().getRow().toString(), 16));
++      assertEquals(col++, Integer.parseInt(entry.getKey().getColumnQualifier().toString(), 16));
++      if (col == startCol + colCount) {
++        col = startCol;
++        row++;
++        if (row == startRow + rowCount) {
++          break;
++        }
++      }
++    }
++    assertEquals(row, startRow + rowCount);
++  }
++
++  private int getWALCount(Connector c) throws Exception {
++    Scanner s = c.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
++    s.setRange(CurrentLogsSection.getRange());
++    try {
++      return Iterators.size(s.iterator());
++    } finally {
++      s.close();
++    }
++  }
++
++  private void writeSomeData(Connector conn, String table, int startRow, int rowCount, int startCol, int colCount) throws Exception {
++    BatchWriterConfig config = new BatchWriterConfig();
++    config.setMaxMemory(10 * 1024 * 1024);
++    BatchWriter bw = conn.createBatchWriter(table, config);
++    for (int r = startRow; r < startRow + rowCount; r++) {
++      Mutation m = new Mutation(Integer.toHexString(r));
++      for (int c = startCol; c < startCol + colCount; c++) {
++        m.put("", Integer.toHexString(c), "");
++      }
++      bw.addMutation(m);
++    }
++    bw.close();
++  }
++
++}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c71fc121/test/src/test/java/org/apache/accumulo/test/functional/WatchTheWatchCountIT.java
----------------------------------------------------------------------
diff --cc test/src/test/java/org/apache/accumulo/test/functional/WatchTheWatchCountIT.java
index 4c86172,3b1dd2f..140fd59
--- a/test/src/test/java/org/apache/accumulo/test/functional/WatchTheWatchCountIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/WatchTheWatchCountIT.java
@@@ -51,8 -51,8 +51,8 @@@ public class WatchTheWatchCountIT exten
        int n = socket.getInputStream().read(buffer);
        String response = new String(buffer, 0, n);
        long total = Long.parseLong(response.split(":")[1].trim());
-       assertTrue("Total watches was not greater than 600, but was " + total, total > 600);
-       assertTrue("Total watches was not less than 600, but was " + total, total < 675);
+       assertTrue("Total watches was not greater than 500, but was " + total, total > 500);
 -      assertTrue("Total watches was not less than 650, but was " + total, total < 600);
++      assertTrue("Total watches was not less than 675, but was " + total, total < 675);
      } finally {
        socket.close();
      }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c71fc121/test/src/test/java/org/apache/accumulo/test/replication/ReplicationIT.java
----------------------------------------------------------------------
diff --cc test/src/test/java/org/apache/accumulo/test/replication/ReplicationIT.java
index ca58a59,46e3ac1..b7a518a
--- a/test/src/test/java/org/apache/accumulo/test/replication/ReplicationIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/replication/ReplicationIT.java
@@@ -993,7 -1085,18 +993,8 @@@ public class ReplicationIT extends Conf
      }
  
      // Write some data to table1
 -    BatchWriter bw = conn.createBatchWriter(table1, new BatchWriterConfig());
 -    for (int rows = 0; rows < 2000; rows++) {
 -      Mutation m = new Mutation(Integer.toString(rows));
 -      for (int cols = 0; cols < 50; cols++) {
 -        String value = Integer.toString(cols);
 -        m.put(value, "", value);
 -      }
 -      bw.addMutation(m);
 -    }
 -
 -    bw.close();
 +    writeSomeData(conn, table1, 2000, 50);
+     conn.tableOperations().flush(table1, null, null, true);
  
      String tableId = conn.tableOperations().tableIdMap().get(table1);
      Assert.assertNotNull("Table ID was null", tableId);
@@@ -1253,7 -1397,18 +1254,8 @@@
      Assert.assertNotNull("Could not determine table id for " + table1, tableId);
  
      // Write some data to table1
 -    BatchWriter bw = conn.createBatchWriter(table1, new BatchWriterConfig());
 -    for (int rows = 0; rows < 2000; rows++) {
 -      Mutation m = new Mutation(Integer.toString(rows));
 -      for (int cols = 0; cols < 50; cols++) {
 -        String value = Integer.toString(cols);
 -        m.put(value, "", value);
 -      }
 -      bw.addMutation(m);
 -    }
 -
 -    bw.close();
 +    writeSomeData(conn, table1, 2000, 50);
+     conn.tableOperations().flush(table1, null, null, true);
  
      // Make sure the replication table exists at this point
      boolean online = ReplicationTable.isOnline(conn);