You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by el...@apache.org on 2014/05/21 03:59:20 UTC

[01/50] [abbrv] git commit: ACCUMULO-378 Rogue test class not in the right package.

Repository: accumulo
Updated Branches:
  refs/heads/ACCUMULO-378 f2c0c16db -> 417b0b332


ACCUMULO-378 Rogue test class not in the right package.


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

Branch: refs/heads/ACCUMULO-378
Commit: 18432fe36372961a5460d9909b3ca9fcf1f35c1f
Parents: 971c7b7
Author: Josh Elser <el...@apache.org>
Authored: Thu May 15 19:16:39 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Thu May 15 19:16:39 2014 -0400

----------------------------------------------------------------------
 .../ReplicationWorkAssignerHelperTest.java      | 56 --------------------
 .../ReplicationWorkAssignerHelperTest.java      | 56 ++++++++++++++++++++
 2 files changed, 56 insertions(+), 56 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/18432fe3/server/base/src/test/java/ReplicationWorkAssignerHelper/ReplicationWorkAssignerHelperTest.java
----------------------------------------------------------------------
diff --git a/server/base/src/test/java/ReplicationWorkAssignerHelper/ReplicationWorkAssignerHelperTest.java b/server/base/src/test/java/ReplicationWorkAssignerHelper/ReplicationWorkAssignerHelperTest.java
deleted file mode 100644
index 2fe71ac..0000000
--- a/server/base/src/test/java/ReplicationWorkAssignerHelper/ReplicationWorkAssignerHelperTest.java
+++ /dev/null
@@ -1,56 +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 ReplicationWorkAssignerHelper;
-
-import java.util.Map.Entry;
-import java.util.UUID;
-
-import org.apache.accumulo.core.replication.ReplicationTarget;
-import org.apache.accumulo.server.replication.ReplicationWorkAssignerHelper;
-import org.apache.hadoop.fs.Path;
-import org.apache.zookeeper.common.PathUtils;
-import org.junit.Assert;
-import org.junit.Test;
-
-/**
- * 
- */
-public class ReplicationWorkAssignerHelperTest {
-
-  @Test
-  public void createsValidZKNodeName() {
-    Path p = new Path ("/accumulo/wals/tserver+port/" + UUID.randomUUID().toString());
-    ReplicationTarget target = new ReplicationTarget("cluster1", "table1", "1");
-
-    String key = ReplicationWorkAssignerHelper.getQueueKey(p.toString(), target);
-    
-    PathUtils.validatePath(key);
-  }
-
-  @Test
-  public void queueKeySerialization() {
-    Path p = new Path("/accumulo/wals/tserver+port/" + UUID.randomUUID().toString());
-    ReplicationTarget target = new ReplicationTarget("cluster1", "table1", "1");
-
-    String key = ReplicationWorkAssignerHelper.getQueueKey(p.toString(), target);
-
-    Entry<String,ReplicationTarget> result = ReplicationWorkAssignerHelper.fromQueueKey(key);
-    Assert.assertEquals(p.toString(), result.getKey());
-    Assert.assertEquals(target, result.getValue());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/18432fe3/server/base/src/test/java/org/apache/accumulo/server/replication/ReplicationWorkAssignerHelperTest.java
----------------------------------------------------------------------
diff --git a/server/base/src/test/java/org/apache/accumulo/server/replication/ReplicationWorkAssignerHelperTest.java b/server/base/src/test/java/org/apache/accumulo/server/replication/ReplicationWorkAssignerHelperTest.java
new file mode 100644
index 0000000..47a4a0d
--- /dev/null
+++ b/server/base/src/test/java/org/apache/accumulo/server/replication/ReplicationWorkAssignerHelperTest.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.server.replication;
+
+import java.util.Map.Entry;
+import java.util.UUID;
+
+import org.apache.accumulo.core.replication.ReplicationTarget;
+import org.apache.accumulo.server.replication.ReplicationWorkAssignerHelper;
+import org.apache.hadoop.fs.Path;
+import org.apache.zookeeper.common.PathUtils;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * 
+ */
+public class ReplicationWorkAssignerHelperTest {
+
+  @Test
+  public void createsValidZKNodeName() {
+    Path p = new Path ("/accumulo/wals/tserver+port/" + UUID.randomUUID().toString());
+    ReplicationTarget target = new ReplicationTarget("cluster1", "table1", "1");
+
+    String key = ReplicationWorkAssignerHelper.getQueueKey(p.toString(), target);
+    
+    PathUtils.validatePath(key);
+  }
+
+  @Test
+  public void queueKeySerialization() {
+    Path p = new Path("/accumulo/wals/tserver+port/" + UUID.randomUUID().toString());
+    ReplicationTarget target = new ReplicationTarget("cluster1", "table1", "1");
+
+    String key = ReplicationWorkAssignerHelper.getQueueKey(p.toString(), target);
+
+    Entry<String,ReplicationTarget> result = ReplicationWorkAssignerHelper.fromQueueKey(key);
+    Assert.assertEquals(p.toString(), result.getKey());
+    Assert.assertEquals(target, result.getValue());
+  }
+
+}


[33/50] [abbrv] git commit: ACCUMULO-378 Better logging for work assigner

Posted by el...@apache.org.
ACCUMULO-378 Better logging for work assigner


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

Branch: refs/heads/ACCUMULO-378
Commit: eb4b9ce60c56ed5f0c2b2b49ae1c9b61edf55a2c
Parents: 2ff3a6c
Author: Josh Elser <el...@apache.org>
Authored: Mon May 19 13:55:41 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Mon May 19 13:55:41 2014 -0400

----------------------------------------------------------------------
 .../org/apache/accumulo/master/replication/WorkDriver.java     | 6 ++++--
 1 file changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/eb4b9ce6/server/master/src/main/java/org/apache/accumulo/master/replication/WorkDriver.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/replication/WorkDriver.java b/server/master/src/main/java/org/apache/accumulo/master/replication/WorkDriver.java
index 1b70a13..00b0480 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/replication/WorkDriver.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/replication/WorkDriver.java
@@ -37,6 +37,7 @@ public class WorkDriver extends Daemon {
   private AccumuloConfiguration conf;
 
   private WorkAssigner assigner;
+  private String assignerImplName;
 
   public WorkDriver(Master master, Connector conn) {
     super();
@@ -55,6 +56,7 @@ public class WorkDriver extends Daemon {
     }
 
     this.assigner.configure(conf, conn);
+    this.assignerImplName = assigner.getClass().getName();
     this.setName(assigner.getName());
   }
 
@@ -79,14 +81,14 @@ public class WorkDriver extends Daemon {
 
   @Override
   public void run() {
-    log.info("Starting replication work assignment thread");
+    log.info("Starting replication work assignment thread using {}", assignerImplName);
 
     while (master.stillMaster()) {
       // Assign the work using the configured implementation
       assigner.assignWork();
 
       long sleepTime = conf.getTimeInMillis(Property.REPLICATION_WORK_ASSIGNMENT_SLEEP);
-      log.debug("Sleeping {} ms", sleepTime);
+      log.debug("Sleeping {} ms before next work assignment", sleepTime);
       UtilWaitThread.sleep(sleepTime);
     }
   }


[35/50] [abbrv] git commit: ACCUMULO-2819 More test updates for the sequential work assigner

Posted by el...@apache.org.
ACCUMULO-2819 More test updates for the sequential work assigner


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

Branch: refs/heads/ACCUMULO-378
Commit: 9f779184cbfa8e3c18be137f8e0471bc0bae4491
Parents: 26a88b4
Author: Josh Elser <el...@apache.org>
Authored: Mon May 19 15:40:49 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Mon May 19 15:40:49 2014 -0400

----------------------------------------------------------------------
 .../DistributedWorkQueueWorkAssigner.java       |   6 +-
 .../master/replication/ReplicationDriver.java   |   1 -
 .../tserver/log/TabletServerLogger.java         |  10 +-
 .../test/replication/MockReplicaSystem.java     |  69 +++++++++++++
 .../test/replication/CyclicReplicationIT.java   |  27 +++--
 .../replication/ReplicationSequentialIT.java    |  60 ++++++++---
 .../test/replication/ReplicationWithGCIT.java   | 102 ++++++-------------
 .../replication/ReplicationWithMakerTest.java   |  10 +-
 test/src/test/resources/log4j.properties        |   1 +
 9 files changed, 179 insertions(+), 107 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/9f779184/server/master/src/main/java/org/apache/accumulo/master/replication/DistributedWorkQueueWorkAssigner.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/replication/DistributedWorkQueueWorkAssigner.java b/server/master/src/main/java/org/apache/accumulo/master/replication/DistributedWorkQueueWorkAssigner.java
index 84f9af5..f04f3e8 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/replication/DistributedWorkQueueWorkAssigner.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/replication/DistributedWorkQueueWorkAssigner.java
@@ -217,8 +217,6 @@ public class DistributedWorkQueueWorkAssigner extends AbstractWorkAssigner {
       return;
     }
 
-    log.info("Creating batchscanner to read Work records from the replication table");
-
     WorkSection.limit(bs);
     bs.setRanges(Collections.singleton(new Range()));
     Text buffer = new Text();
@@ -261,12 +259,12 @@ public class DistributedWorkQueueWorkAssigner extends AbstractWorkAssigner {
         }
       }
     } finally {
+      log.info("Created work entries for {} files", filesWorkWasCreatedFrom);
+
       if (null != bs) {
         bs.close();
       }
     }
-
-    log.info("Created work entries for {} files", filesWorkWasCreatedFrom);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9f779184/server/master/src/main/java/org/apache/accumulo/master/replication/ReplicationDriver.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/replication/ReplicationDriver.java b/server/master/src/main/java/org/apache/accumulo/master/replication/ReplicationDriver.java
index 75fe5f3..3069c97 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/replication/ReplicationDriver.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/replication/ReplicationDriver.java
@@ -24,7 +24,6 @@ import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.util.Daemon;
 import org.apache.accumulo.fate.util.UtilWaitThread;
 import org.apache.accumulo.master.Master;
-import org.apache.accumulo.server.replication.ReplicationTable;
 import org.apache.log4j.Logger;
 
 /**

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9f779184/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 67127f1..d8c4279 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
@@ -239,7 +239,7 @@ public class TabletServerLogger {
     return write(sessions, mincFinish, writer);
   }
 
-  private int write(Collection<CommitSession> sessions, boolean mincFinish, Writer writer) throws IOException {
+  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();
 
@@ -268,7 +268,7 @@ public class TabletServerLogger {
               }
 
               // Need to release
-              KeyExtent extent = commitSession.getExtent();tserver.getTableConfiguration(extent).getNamespaceConfiguration();
+              KeyExtent extent = commitSession.getExtent();
               if (ReplicationConfigurationUtil.isEnabled(extent, tserver.getTableConfiguration(extent))) {
                 Set<String> logs = new HashSet<String>();
                 for (DfsLogger logger : copy) {
@@ -328,6 +328,7 @@ public class TabletServerLogger {
           @Override
           void withWriteLock() throws IOException {
             close();
+            closeForReplication(sessions);
           }
         });
       }
@@ -343,11 +344,16 @@ public class TabletServerLogger {
       @Override
       void withWriteLock() throws IOException {
         close();
+        closeForReplication(sessions);
       }
     });
     return seq;
   }
 
+  protected void closeForReplication(Collection<CommitSession> sessions) {
+    // TODO We can close the WAL here for replication purposes
+  }
+
   public int defineTablet(final CommitSession commitSession) throws IOException {
     // scribble this into the metadata tablet, too.
     if (!enabled(commitSession))

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9f779184/test/src/main/java/org/apache/accumulo/test/replication/MockReplicaSystem.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/replication/MockReplicaSystem.java b/test/src/main/java/org/apache/accumulo/test/replication/MockReplicaSystem.java
new file mode 100644
index 0000000..cafd9b7
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/replication/MockReplicaSystem.java
@@ -0,0 +1,69 @@
+/*
+ * 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.replication;
+
+import org.apache.accumulo.core.client.replication.ReplicaSystem;
+import org.apache.accumulo.core.replication.ReplicationTarget;
+import org.apache.accumulo.core.replication.proto.Replication.Status;
+import org.apache.hadoop.fs.Path;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.protobuf.TextFormat;
+
+/**
+ * Fake ReplicaSystem which returns that the data was fully replicated after some sleep period (in milliseconds)
+ * <p>
+ * Default sleep amount is 0ms
+ */
+public class MockReplicaSystem implements ReplicaSystem {
+  private static final Logger log = LoggerFactory.getLogger(MockReplicaSystem.class);
+
+  private long sleep = 0;
+
+  @Override
+  public Status replicate(Path p, Status status, ReplicationTarget target) {
+    Status.Builder builder = Status.newBuilder(status);
+    if (status.getInfiniteEnd()) {
+      builder.setBegin(Long.MAX_VALUE);
+    } else {
+      builder.setBegin(status.getEnd());
+    }
+
+    try {
+      Thread.sleep(sleep);
+    } catch (InterruptedException e) {
+      log.error("Interrupted while sleeping, will report no progress", e);
+      Thread.currentThread().interrupt();
+      return status;
+    }
+    
+
+    Status newStatus = builder.build();
+    log.info("Received {} returned {}", TextFormat.shortDebugString(status), TextFormat.shortDebugString(newStatus));
+    return newStatus;
+  }
+
+  @Override
+  public void configure(String configuration) {
+    try {
+      sleep = Long.parseLong(configuration);
+    } catch (NumberFormatException e) {
+      log.warn("Could not parse {} as an integer, using default sleep of {}", configuration, sleep, e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9f779184/test/src/test/java/org/apache/accumulo/test/replication/CyclicReplicationIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/replication/CyclicReplicationIT.java b/test/src/test/java/org/apache/accumulo/test/replication/CyclicReplicationIT.java
index 75f6a3f..b30dc39 100644
--- a/test/src/test/java/org/apache/accumulo/test/replication/CyclicReplicationIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/replication/CyclicReplicationIT.java
@@ -21,6 +21,7 @@ import java.io.File;
 import java.io.FileOutputStream;
 import java.io.OutputStream;
 import java.util.Map.Entry;
+import java.util.Set;
 
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
@@ -48,6 +49,8 @@ import org.junit.Assert;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TestName;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.collect.Iterables;
 
@@ -55,6 +58,7 @@ import com.google.common.collect.Iterables;
  * 
  */
 public class CyclicReplicationIT {
+  private static final Logger log = LoggerFactory.getLogger(CyclicReplicationIT.class);
 
   @Rule
   public TestName testName = new TestName();
@@ -80,7 +84,7 @@ public class CyclicReplicationIT {
     out.close();
   }
 
-  @Test
+  @Test(timeout = 5 * 60 * 1000)
   public void dataIsNotOverReplicated() throws Exception {
     File master1Dir = createTestDir("master1"), master2Dir = createTestDir("master2");
     String password = "password";
@@ -155,9 +159,12 @@ public class CyclicReplicationIT {
       Mutation m = new Mutation("row");
       m.put("count", "", "1");
       bw.addMutation(m);
-      bw.flush();
       bw.close();
 
+      Set<String> files = connMaster1.replicationOperations().referencedFiles(master1Cluster.getInstanceName());
+
+      log.info("Found {} that need replication from master1", files);
+
       // Kill and restart the tserver to close the WAL on master1
       for (ProcessReference proc : master1Cluster.getProcesses().get(ServerType.TABLET_SERVER)) {
         master1Cluster.killProcess(ServerType.TABLET_SERVER, proc);
@@ -165,21 +172,26 @@ public class CyclicReplicationIT {
 
       master1Cluster.exec(TabletServer.class);
 
+      log.info("Restarted tserver on master1");
+
       // Sanity check that the element is there on master1
       Scanner s = connMaster1.createScanner(master1Cluster.getInstanceName(), Authorizations.EMPTY);
       Entry<Key,Value> entry = Iterables.getOnlyElement(s);
       Assert.assertEquals("1", entry.getValue().toString());
 
-      Thread.sleep(5000);
-
       // Wait for this table to replicate
-      connMaster1.replicationOperations().drain(master1Cluster.getInstanceName());
+      connMaster1.replicationOperations().drain(master1Cluster.getInstanceName(), files);
+
+      Thread.sleep(5000);
 
       // Check that the element made it to master2 only once
       s = connMaster2.createScanner(master2Cluster.getInstanceName(), Authorizations.EMPTY);
       entry = Iterables.getOnlyElement(s);
       Assert.assertEquals("1", entry.getValue().toString());
 
+      // Wait for master2 to finish replicating it back
+      files = connMaster2.replicationOperations().referencedFiles(master2Cluster.getInstanceName());
+
       // Kill and restart the tserver to close the WAL on master2
       for (ProcessReference proc : master2Cluster.getProcesses().get(ServerType.TABLET_SERVER)) {
         master2Cluster.killProcess(ServerType.TABLET_SERVER, proc);
@@ -192,10 +204,9 @@ public class CyclicReplicationIT {
       entry = Iterables.getOnlyElement(s);
       Assert.assertEquals("1", entry.getValue().toString());
 
-      Thread.sleep(5000);
+      connMaster2.replicationOperations().drain(master2Cluster.getInstanceName(), files);
 
-      // Wait for master2 to finish replicating it back
-      connMaster2.replicationOperations().drain(master2Cluster.getInstanceName());
+      Thread.sleep(5000);
 
       // Verify that the entry wasn't sent back to master1
       s = connMaster1.createScanner(master1Cluster.getInstanceName(), Authorizations.EMPTY);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9f779184/test/src/test/java/org/apache/accumulo/test/replication/ReplicationSequentialIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/replication/ReplicationSequentialIT.java b/test/src/test/java/org/apache/accumulo/test/replication/ReplicationSequentialIT.java
index 0683a57..dce4e17 100644
--- a/test/src/test/java/org/apache/accumulo/test/replication/ReplicationSequentialIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/replication/ReplicationSequentialIT.java
@@ -44,10 +44,13 @@ import org.apache.accumulo.core.replication.proto.Replication.Status;
 import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.master.replication.SequentialWorkAssigner;
+import org.apache.accumulo.minicluster.ServerType;
 import org.apache.accumulo.minicluster.impl.MiniAccumuloClusterImpl;
 import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.minicluster.impl.ProcessReference;
 import org.apache.accumulo.server.replication.ReplicationTable;
 import org.apache.accumulo.test.functional.ConfigurableMacIT;
+import org.apache.accumulo.tserver.TabletServer;
 import org.apache.accumulo.tserver.replication.AccumuloReplicaSystem;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.RawLocalFileSystem;
@@ -86,8 +89,7 @@ public class ReplicationSequentialIT extends ConfigurableMacIT {
     cfg.setProperty(Property.REPLICATION_MAX_UNIT_SIZE, "8M");
     cfg.setProperty(Property.REPLICATION_NAME, "master");
     cfg.setProperty(Property.REPLICATION_WORK_ASSIGNER, SequentialWorkAssigner.class.getName());
-    cfg.useMiniDFS(true);
-//    hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
+    hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
   }
 
   @Test(timeout = 60 * 5000)
@@ -158,6 +160,31 @@ public class ReplicationSequentialIT extends ConfigurableMacIT {
       }
     }
 
+    for (ProcessReference proc : cluster.getProcesses().get(ServerType.TABLET_SERVER)) {
+      cluster.killProcess(ServerType.TABLET_SERVER, proc);
+    }
+    cluster.exec(TabletServer.class);
+
+    log.info("TabletServer restarted");
+    for (@SuppressWarnings("unused") Entry<Key,Value> e : ReplicationTable.getScanner(connMaster)) {}
+    log.info("TabletServer is online");
+
+    log.info("");
+    log.info("Fetching metadata records:");
+    for (Entry<Key,Value> kv : connMaster.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
+      if (ReplicationSection.COLF.equals(kv.getKey().getColumnFamily())) {
+        log.info(kv.getKey().toStringNoTruncate() + " " + ProtobufUtil.toString(Status.parseFrom(kv.getValue().get())));
+      } else {
+        log.info(kv.getKey().toStringNoTruncate() + " " + kv.getValue());
+      }
+    }
+
+    log.info("");
+    log.info("Fetching replication records:");
+    for (Entry<Key,Value> kv : connMaster.createScanner(ReplicationTable.NAME, Authorizations.EMPTY)) {
+      log.info(kv.getKey().toStringNoTruncate() + " " + ProtobufUtil.toString(Status.parseFrom(kv.getValue().get())));
+    }
+
     Future<Boolean> future = executor.submit(new Callable<Boolean>() {
 
       @Override
@@ -169,12 +196,17 @@ public class ReplicationSequentialIT extends ConfigurableMacIT {
       
     });
 
-    connMaster.tableOperations().compact(masterTable, null, null, true, true);
+    try {
+      future.get(30, TimeUnit.SECONDS);
+    } catch (TimeoutException e) {
+      future.cancel(true);
+      Assert.fail("Drain did not finish within 5 seconds");
+    }
+
+    log.info("drain completed");
 
     log.info("");
-    log.info("Compaction completed");
-
-    log.debug("");
+    log.info("Fetching metadata records:");
     for (Entry<Key,Value> kv : connMaster.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
       if (ReplicationSection.COLF.equals(kv.getKey().getColumnFamily())) {
         log.info(kv.getKey().toStringNoTruncate() + " " + ProtobufUtil.toString(Status.parseFrom(kv.getValue().get())));
@@ -183,13 +215,8 @@ public class ReplicationSequentialIT extends ConfigurableMacIT {
       }
     }
 
-    try {
-      future.get(15, TimeUnit.SECONDS);
-    } catch (TimeoutException e) {
-      Assert.fail("Drain did not finish within 5 seconds");
-    }
-
     log.info("");
+    log.info("Fetching replication records:");
     for (Entry<Key,Value> kv : connMaster.createScanner(ReplicationTable.NAME, Authorizations.EMPTY)) {
       log.info(kv.getKey().toStringNoTruncate() + " " + ProtobufUtil.toString(Status.parseFrom(kv.getValue().get())));
     }
@@ -300,8 +327,13 @@ public class ReplicationSequentialIT extends ConfigurableMacIT {
         Thread.sleep(500);
       }
 
-      connMaster.tableOperations().compact(masterTable1, null, null, true, false);
-      connMaster.tableOperations().compact(masterTable2, null, null, true, false);
+      // Restart the tserver to force a close on the WAL
+      for (ProcessReference proc : cluster.getProcesses().get(ServerType.TABLET_SERVER)) {
+        cluster.killProcess(ServerType.TABLET_SERVER, proc);
+      }
+      cluster.exec(TabletServer.class);
+
+      log.info("Restarted the tserver");
 
       // Wait until we fully replicated something
       boolean fullyReplicated = false;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9f779184/test/src/test/java/org/apache/accumulo/test/replication/ReplicationWithGCIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/replication/ReplicationWithGCIT.java b/test/src/test/java/org/apache/accumulo/test/replication/ReplicationWithGCIT.java
index 101001f..23da719 100644
--- a/test/src/test/java/org/apache/accumulo/test/replication/ReplicationWithGCIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/replication/ReplicationWithGCIT.java
@@ -30,7 +30,6 @@ 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.client.replication.ReplicaSystem;
 import org.apache.accumulo.core.client.replication.ReplicaSystemFactory;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Key;
@@ -58,7 +57,6 @@ import org.apache.accumulo.server.replication.ReplicationTable;
 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;
 import org.apache.hadoop.fs.RawLocalFileSystem;
 import org.apache.hadoop.io.Text;
@@ -76,33 +74,6 @@ import com.google.protobuf.TextFormat;
 public class ReplicationWithGCIT extends ConfigurableMacIT {
   private static final Logger log = LoggerFactory.getLogger(ReplicationWithGCIT.class);
 
-  /**
-   * Fake ReplicaSystem which immediately returns that the data was fully replicated
-   */
-  public static class MockReplicaSystem implements ReplicaSystem {
-    private static final Logger log = LoggerFactory.getLogger(MockReplicaSystem.class);
-
-    public MockReplicaSystem() {}
-
-    @Override
-    public Status replicate(Path p, Status status, ReplicationTarget target) {
-      Status.Builder builder = Status.newBuilder(status);
-      if (status.getInfiniteEnd()) {
-        builder.setBegin(Long.MAX_VALUE);
-      } else {
-        builder.setBegin(status.getEnd());
-      }
-
-      Status newStatus = builder.build();
-      log.info("Received {} returned {}", TextFormat.shortDebugString(status), TextFormat.shortDebugString(newStatus));
-      return newStatus;
-    }
-
-    @Override
-    public void configure(String configuration) {}
-
-  }
-
   @Override
   public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
     cfg.setNumTservers(1);
@@ -358,10 +329,10 @@ public class ReplicationWithGCIT extends ConfigurableMacIT {
 
   }
 
-  @Test
+  @Test(timeout = 5 * 60 * 1000)
   public void replicatedStatusEntriesAreDeleted() throws Exception {
-    Connector conn = getConnector();
-    FileSystem fs = FileSystem.getLocal(new Configuration());
+    final Connector conn = getConnector();
+    log.info("Got connector to MAC");
     String table1 = "table1";
 
     // replication shouldn't exist when we begin
@@ -377,8 +348,9 @@ public class ReplicationWithGCIT extends ConfigurableMacIT {
         conn.tableOperations().setProperty(table1, Property.TABLE_REPLICATION.getKey(), "true");
         // Replicate table1 to cluster1 in the table with id of '4'
         conn.tableOperations().setProperty(table1, Property.TABLE_REPLICATION_TARGETS.getKey() + "cluster1", "4");
+        // Use the MockReplicaSystem impl and sleep for 5seconds
         conn.instanceOperations().setProperty(Property.REPLICATION_PEERS.getKey() + "cluster1",
-            ReplicaSystemFactory.getPeerConfigurationValue(MockReplicaSystem.class, null));
+            ReplicaSystemFactory.getPeerConfigurationValue(MockReplicaSystem.class, "5000"));
         attempts = 0;
       } catch (Exception e) {
         attempts--;
@@ -469,17 +441,23 @@ public class ReplicationWithGCIT extends ConfigurableMacIT {
     }
 
     /**
-     * By this point, we should have data ingested into a table, with at least one WAL as a candidate for replication. It may or may not yet be closed.
+     * By this point, we should have data ingested into a table, with at least one WAL as a candidate for replication. Compacting the table should close all
+     * open WALs, which should ensure all records we're going to replicate have entries in the replication table, and nothing will exist in the metadata table
+     * anymore
      */
 
+    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);
     }
 
+    log.info("Starting tserver");
     cluster.exec(TabletServer.class);
 
+    log.info("Waiting to read tables");
+
     // Make sure we can read all the tables (recovery complete)
     for (String table : new String[] {MetadataTable.NAME, table1}) {
       s = conn.createScanner(table, new Authorizations());
@@ -487,48 +465,39 @@ public class ReplicationWithGCIT extends ConfigurableMacIT {
       Entry<Key,Value> entry : s) {}
     }
 
-    // Need to make sure we get the entries in metadata
-    boolean foundResults = false;
-    for (int i = 0; i < 5 && !foundResults; i++) {
-      s = conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
-      s.setRange(ReplicationSection.getRange());
-      if (Iterables.size(s) > 0) {
-        foundResults = true;
-      }
-      Thread.sleep(1000);
-    }
-
-    Assert.assertTrue("Did not find any replication entries in the metadata table", foundResults);
-
+    log.info("Checking for replication entries in replication");
     // Then we need to get those records over to the replication table
-    foundResults = false;
-    for (int i = 0; i < 5 && !foundResults; i++) {
+    boolean foundResults = false;
+    for (int i = 0; i < 5; i++) {
       s = ReplicationTable.getScanner(conn);
       if (Iterables.size(s) > 0) {
         foundResults = true;
+        break;
       }
       Thread.sleep(1000);
     }
 
     Assert.assertTrue("Did not find any replication entries in the replication table", foundResults);
 
-    s = conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
-    s.setRange(ReplicationSection.getRange());
-    for (Entry<Key,Value> entry : s) {
-      String row = entry.getKey().getRow().toString();
-      Path file = new Path(row.substring(ReplicationSection.getRowPrefix().length()));
-      Assert.assertTrue(file + " did not exist when it should", fs.exists(file));
+    // 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 record after we create the record in the replication table
+    foundResults = true;
+    for (int i = 0; i < 5; i++) {
+      s = conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
+      s.setRange(ReplicationSection.getRange());
+      if (Iterables.size(s) == 0) {
+        foundResults = false;
+        break;
+      }
+      Thread.sleep(1000);
     }
 
-    /**
-     * After recovery completes, we should have unreplicated, closed Status messages. The close happens at the beginning of log recovery.
-     * The MockReplicaSystem we configured will just automatically say the data has been replicated, so this should then created replicated
-     * and closed Status messages.
-     */
+    Assert.assertFalse("Replication status messages were not cleaned up from metadata table, check why the StatusMaker didn't delete them", foundResults);
 
     /**
      * After we set the begin to Long.MAX_VALUE, the RemoveCompleteReplicationRecords class will start deleting the records which have been closed by
-     * CloseWriteAheadLogReferences (which will have been working since we restarted the tserver(s))
+     * the minor compaction and replicated by the MockReplicaSystem
      */
 
     // Wait for a bit since the GC has to run (should be running after a one second delay)
@@ -552,16 +521,5 @@ public class ReplicationWithGCIT extends ConfigurableMacIT {
     }
 
     Assert.assertEquals("Found unexpected replication records in the replication table", 0, recordsFound);
-
-    // If the replication table entries were deleted, so should the metadata table replication entries
-    s = conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
-    s.setRange(ReplicationSection.getRange());
-    recordsFound = 0;
-    for (Entry<Key,Value> entry : s) {
-      recordsFound++;
-      log.info(entry.getKey().toStringNoTruncate() + " " + Status.parseFrom(entry.getValue().get()).toString().replace("\n", ", "));
-    }
-
-    Assert.assertEquals("Found unexpected replication records in the metadata table", 0, recordsFound);
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9f779184/test/src/test/java/org/apache/accumulo/test/replication/ReplicationWithMakerTest.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/replication/ReplicationWithMakerTest.java b/test/src/test/java/org/apache/accumulo/test/replication/ReplicationWithMakerTest.java
index aee8a1e..03ac72c 100644
--- a/test/src/test/java/org/apache/accumulo/test/replication/ReplicationWithMakerTest.java
+++ b/test/src/test/java/org/apache/accumulo/test/replication/ReplicationWithMakerTest.java
@@ -27,10 +27,10 @@ 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.Value;
-import org.apache.accumulo.core.protobuf.ProtobufUtil;
 import org.apache.accumulo.core.replication.ReplicationSchema.StatusSection;
 import org.apache.accumulo.core.replication.ReplicationSchema.WorkSection;
 import org.apache.accumulo.core.replication.ReplicationTarget;
+import org.apache.accumulo.core.replication.StatusUtil;
 import org.apache.accumulo.core.replication.proto.Replication.Status;
 import org.apache.accumulo.core.security.TablePermission;
 import org.apache.accumulo.core.util.UtilWaitThread;
@@ -142,13 +142,13 @@ public class ReplicationWithMakerTest extends ConfigurableMacIT {
     Scanner s = ReplicationTable.getScanner(conn);
     StatusSection.limit(s);
     Entry<Key,Value> entry = null;
+    Status expectedStatus = StatusUtil.openWithUnknownLength();
     attempts = 5;
     // This record will move from new to new with infinite length because of the minc (flush)
     while (null == entry && attempts > 0) {
       try {
         entry = Iterables.getOnlyElement(s);
-        Status actualStatus = Status.parseFrom(entry.getValue().get());
-        if (!actualStatus.hasClosedTime() || !actualStatus.getClosed()) {
+        if (!expectedStatus.equals(Status.parseFrom(entry.getValue().get()))) {
           entry = null;
           // the master process didn't yet fire and write the new mutation, wait for it to do
           // so and try to read it again
@@ -171,9 +171,7 @@ public class ReplicationWithMakerTest extends ConfigurableMacIT {
     }
 
     Assert.assertNotNull("Could not find expected entry in replication table", entry);
-    Status actualStatus = Status.parseFrom(entry.getValue().get());
-    Assert.assertTrue("Expected to find a replication entry that is closed with infinite length: " + ProtobufUtil.toString(actualStatus),
-        actualStatus.getClosed() && actualStatus.hasClosedTime());
+    Assert.assertEquals("Expected to find a replication entry that is open with infinite length", expectedStatus, Status.parseFrom(entry.getValue().get()));
 
     // Try a couple of times to watch for the work record to be created
     boolean notFound = true;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/9f779184/test/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/test/src/test/resources/log4j.properties b/test/src/test/resources/log4j.properties
index 171d690..11ff405 100644
--- a/test/src/test/resources/log4j.properties
+++ b/test/src/test/resources/log4j.properties
@@ -19,6 +19,7 @@ log4j.appender.CA.layout=org.apache.log4j.PatternLayout
 log4j.appender.CA.layout.ConversionPattern=%d{ISO8601} [%c{2}] %-5p: %m%n
 
 log4j.logger.org.apache.accumulo.core=DEBUG
+log4j.logger.org.apache.accumulo.core.client.impl.MasterClient=INFO
 log4j.logger.org.apache.accumulo.core.client.impl.ServerClient=ERROR
 log4j.logger.org.apache.accumulo.core.util.shell.Shell.audit=off
 log4j.logger.org.apache.accumulo.core.util.shell.Shell=FATAL


[18/50] [abbrv] git commit: ACCUMULO-2812 Pass options to compiler for native lib

Posted by el...@apache.org.
ACCUMULO-2812 Pass options to compiler for native lib

  Use system default architecture options when compiling, instead of forcing
  -m64. Allow passing additional arguments (USERFLAGS) to specify additional
  compiler options. Set USERFLAGS with the build_native_library.sh's
  command-line arguments.


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

Branch: refs/heads/ACCUMULO-378
Commit: 7ccf2026913993d5ce076c05f78e3921c532b2ac
Parents: 10ce7af
Author: Christopher Tubbs <ct...@apache.org>
Authored: Thu May 15 16:01:07 2014 -0400
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Fri May 16 13:01:12 2014 -0400

----------------------------------------------------------------------
 bin/build_native_library.sh               | 1 +
 server/native/src/main/resources/Makefile | 7 ++++---
 2 files changed, 5 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/7ccf2026/bin/build_native_library.sh
----------------------------------------------------------------------
diff --git a/bin/build_native_library.sh b/bin/build_native_library.sh
index 65b2457..907ce2c 100755
--- a/bin/build_native_library.sh
+++ b/bin/build_native_library.sh
@@ -56,6 +56,7 @@ native_dir=`find "${TMP_DIR}" -maxdepth 1 -mindepth 1 -type d`
 cd "${native_dir}"
 
 # Make the native library
+export USERFLAGS=$@
 make
 
 # Make sure it didn't fail

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7ccf2026/server/native/src/main/resources/Makefile
----------------------------------------------------------------------
diff --git a/server/native/src/main/resources/Makefile b/server/native/src/main/resources/Makefile
index 33c2fe9..b9211aa 100644
--- a/server/native/src/main/resources/Makefile
+++ b/server/native/src/main/resources/Makefile
@@ -18,14 +18,15 @@ HDRS=$(wildcard nativeMap/*.h) $(wildcard javah/*.h)
 TESTSRCS=$(wildcard testNativeMap/*.cc)
 CXX=g++
 MAVERICKFLAGS=
+USERFLAGS=$(shell env | grep "^USERFLAGS=" | cut -d= -f2)
 
 ifeq ($(shell uname),Linux)
-	JAVA_HOME=$(shell env | fgrep JAVA_HOME | cut -d= -f2)
+	JAVA_HOME=$(shell env | grep "^JAVA_HOME=" | cut -d= -f2)
 	ifeq ($(strip $(JAVA_HOME)),)
 		JAVA_HOME=$(shell dirname $$(dirname $$(readlink -ef $$(which javah))))
 	endif
 	NATIVE_LIB := libaccumulo.so
-	CXXFLAGS=-m64 -g -fPIC -shared -O3 -Wall -I$(JAVA_HOME)/include -I$(JAVA_HOME)/include/linux -Ijavah
+	CXXFLAGS=-g -fPIC -shared -O3 -Wall -I$(JAVA_HOME)/include -I$(JAVA_HOME)/include/linux -Ijavah $(USERFLAGS)
 endif
 
 ifeq ($(shell uname),Darwin)
@@ -34,7 +35,7 @@ ifeq ($(shell uname),Darwin)
 ifneq (,$(findstring 10.9,$(shell sw_vers -productVersion)))
 	MAVERICKFLAGS=-stdlib=libstdc++
 endif
-	CXXFLAGS=-m64 -dynamiclib -undefined dynamic_lookup -O3 -I/System/Library/Frameworks/JavaVM.framework/Headers -I$(JAVA_HOME)/include -I$(JAVA_HOME)/include/darwin -Ijavah $(MAVERICK_FLAGS)
+	CXXFLAGS=-dynamiclib -undefined dynamic_lookup -O3 -I/System/Library/Frameworks/JavaVM.framework/Headers -I$(JAVA_HOME)/include -I$(JAVA_HOME)/include/darwin -Ijavah $(USERFLAGS) $(MAVERICK_FLAGS)
 endif
 
 all : $(NATIVE_LIB)


[48/50] [abbrv] ACCUMULO-2574 Change closedTime into createdTime

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/accumulo/blob/b53bbf08/test/src/test/java/org/apache/accumulo/test/replication/StatusCombinerMacTest.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/replication/StatusCombinerMacTest.java b/test/src/test/java/org/apache/accumulo/test/replication/StatusCombinerMacTest.java
index c889e42..e264488 100644
--- a/test/src/test/java/org/apache/accumulo/test/replication/StatusCombinerMacTest.java
+++ b/test/src/test/java/org/apache/accumulo/test/replication/StatusCombinerMacTest.java
@@ -53,9 +53,10 @@ public class StatusCombinerMacTest extends SimpleMacIT {
     ReplicationTable.create(conn);
 
     BatchWriter bw = conn.createBatchWriter(ReplicationTable.NAME, new BatchWriterConfig());
+    long createTime = System.currentTimeMillis();
     try {
       Mutation m = new Mutation("file:/accumulo/wal/HW10447.local+56808/93cdc17e-7521-44fa-87b5-37f45bcb92d3");
-      StatusSection.add(m, new Text("1"), StatusUtil.newFileValue());
+      StatusSection.add(m, new Text("1"), StatusUtil.fileCreatedValue(createTime));
       bw.addMutation(m);
     } finally {
       bw.close();
@@ -63,7 +64,7 @@ public class StatusCombinerMacTest extends SimpleMacIT {
 
     Scanner s = conn.createScanner(ReplicationTable.NAME, new Authorizations());
     Entry<Key,Value> entry = Iterables.getOnlyElement(s);
-    Assert.assertEquals(entry.getValue(), StatusUtil.newFileValue());
+    Assert.assertEquals(StatusUtil.fileCreatedValue(createTime), entry.getValue());
 
     bw = conn.createBatchWriter(ReplicationTable.NAME, new BatchWriterConfig());
     try {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b53bbf08/test/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/test/src/test/resources/log4j.properties b/test/src/test/resources/log4j.properties
index 9f991d1..5566a3b 100644
--- a/test/src/test/resources/log4j.properties
+++ b/test/src/test/resources/log4j.properties
@@ -39,6 +39,7 @@ log4j.logger.org.apache.accumulo.core.client.impl.ThriftScanner=INFO
 log4j.logger.org.apache.accumulo.fate.zookeeper.DistributedReadWriteLock=WARN
 log4j.logger.org.mortbay.log=WARN
 log4j.logger.org.apache.hadoop=WARN
+log4j.logger.org.apache.jasper=INFO
 log4j.logger.org.apache.hadoop.hdfs.server.namenode.FSNamesystem.audit=WARN
 log4j.logger.org.apache.hadoop.hdfs.server.datanode.DataNode.clienttrace=WARN
 log4j.logger.BlockStateChange=WARN
\ No newline at end of file


[31/50] [abbrv] git commit: ACCUMULO-378 More logging and more robustness when ZK is "slow"

Posted by el...@apache.org.
ACCUMULO-378 More logging and more robustness when ZK is "slow"


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

Branch: refs/heads/ACCUMULO-378
Commit: 410a5ec9ffcd42c04ea88e54ac22377ac41cbc95
Parents: e1f697d
Author: Josh Elser <el...@apache.org>
Authored: Mon May 19 13:54:10 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Mon May 19 13:54:10 2014 -0400

----------------------------------------------------------------------
 .../DistributedWorkQueueWorkAssigner.java       | 32 ++++++++++++++++----
 1 file changed, 26 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/410a5ec9/server/master/src/main/java/org/apache/accumulo/master/replication/DistributedWorkQueueWorkAssigner.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/replication/DistributedWorkQueueWorkAssigner.java b/server/master/src/main/java/org/apache/accumulo/master/replication/DistributedWorkQueueWorkAssigner.java
index e97f3ca..84f9af5 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/replication/DistributedWorkQueueWorkAssigner.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/replication/DistributedWorkQueueWorkAssigner.java
@@ -98,6 +98,7 @@ public class DistributedWorkQueueWorkAssigner extends AbstractWorkAssigner {
     }
 
     if (null == queuedWork) {
+      log.info("Reinitializing state from DistributedWorkQueue in ZooKeeper");
       initializeQueuedWork();
     }
 
@@ -108,9 +109,11 @@ public class DistributedWorkQueueWorkAssigner extends AbstractWorkAssigner {
     // Get the maximum number of entries we want to queue work for (or the default)
     this.maxQueueSize = conf.getCount(Property.REPLICATION_MAX_WORK_QUEUE);
 
+    log.info("Creating work entries from replication table");
     // Scan over the work records, adding the work to the queue
     createWork();
 
+    log.info("Cleaning up finished work entries from replication table");
     // Keep the state of the work we queued correct
     cleanupFinishedWork();
   }
@@ -182,10 +185,23 @@ public class DistributedWorkQueueWorkAssigner extends AbstractWorkAssigner {
   protected void initializeQueuedWork() {
     Preconditions.checkArgument(null == queuedWork, "Expected queuedWork to be null");
     queuedWork = new HashSet<>();
-    try {
-      queuedWork.addAll(workQueue.getWorkQueued());
-    } catch (KeeperException | InterruptedException e) {
-      throw new RuntimeException("Error reading existing queued replication work", e);
+    while (true) {
+      try {
+        queuedWork.addAll(workQueue.getWorkQueued());
+        return;
+      } catch (KeeperException e) {
+        if (KeeperException.Code.NONODE.equals(e.code())) {
+          log.warn("Could not find ZK root for replication work queue, will retry", e);
+          UtilWaitThread.sleep(500);
+          continue;
+        }
+
+        log.error("Error reading existing queued replication work from ZooKeeper", e);
+        throw new RuntimeException("Error reading existing queued replication work from ZooKeeper", e);
+      } catch (InterruptedException e) {
+        log.error("Error reading existing queued replication work from ZooKeeper", e);
+        throw new RuntimeException("Error reading existing queued replication work from ZooKeeper", e);
+      }
     }
   }
 
@@ -198,20 +214,21 @@ public class DistributedWorkQueueWorkAssigner extends AbstractWorkAssigner {
     try {
       bs = ReplicationTable.getBatchScanner(conn, 4);
     } catch (TableNotFoundException e) {
-      UtilWaitThread.sleep(1000);
       return;
     }
 
+    log.info("Creating batchscanner to read Work records from the replication table");
+
     WorkSection.limit(bs);
     bs.setRanges(Collections.singleton(new Range()));
     Text buffer = new Text();
+    long filesWorkWasCreatedFrom = 0l;
     try {
       for (Entry<Key,Value> entry : bs) {
         // If we're not working off the entries, we need to not shoot ourselves in the foot by continuing
         // to add more work entries
         if (queuedWork.size() > maxQueueSize) {
           log.warn("Queued replication work exceeds configured maximum ({}), sleeping to allow work to occur", maxQueueSize);
-          UtilWaitThread.sleep(5000);
           return;
         }
 
@@ -235,6 +252,7 @@ public class DistributedWorkQueueWorkAssigner extends AbstractWorkAssigner {
           // And, we haven't already queued this file up for work already
           if (!queuedWork.contains(key)) {
             queueWork(key, file);
+            filesWorkWasCreatedFrom++;
           } else {
             log.trace("Not re-queueing work for {}", key);
           }
@@ -247,6 +265,8 @@ public class DistributedWorkQueueWorkAssigner extends AbstractWorkAssigner {
         bs.close();
       }
     }
+
+    log.info("Created work entries for {} files", filesWorkWasCreatedFrom);
   }
 
   /**


[47/50] [abbrv] git commit: ACCUMULO-378 Metadata table has the versioning iterator set at 10.

Posted by el...@apache.org.
ACCUMULO-378 Metadata table has the versioning iterator set at 10.

Someone is a terrible person and set the versioning iterator on the metadata
table to 10 instead of 20 like it is for literally every other table.


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

Branch: refs/heads/ACCUMULO-378
Commit: 53e06a2590d7cc4bb4da373f3154a5cfbbc6a9bd
Parents: 2817f42
Author: Josh Elser <el...@apache.org>
Authored: Tue May 20 18:49:07 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Tue May 20 18:49:07 2014 -0400

----------------------------------------------------------------------
 .../server/util/ReplicationTableUtil.java       |  2 +-
 .../server/util/ReplicationTableUtilTest.java   |  2 +-
 .../replication/ReplicationTablesMacTest.java   | 94 ++++++++++++++++++++
 3 files changed, 96 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/53e06a25/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 132c5f3..45f8fea 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
@@ -108,7 +108,7 @@ public class ReplicationTableUtil {
     if (!iterators.containsKey(COMBINER_NAME)) {
       // Set our combiner and combine all columns
       // Need to set the combiner beneath versioning since we don't want to turn it off
-      IteratorSetting setting = new IteratorSetting(15, COMBINER_NAME, StatusCombiner.class);
+      IteratorSetting setting = new IteratorSetting(9, COMBINER_NAME, StatusCombiner.class);
       Combiner.setColumns(setting, Collections.singletonList(new Column(MetadataSchema.ReplicationSection.COLF)));
       try {
         tops.attachIterator(tableName, setting);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/53e06a25/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 be8e7ed..7aa53b6 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
@@ -133,7 +133,7 @@ public class ReplicationTableUtilTest {
     String myMetadataTable = "mymetadata";
     Map<String,EnumSet<IteratorScope>> iterators = new HashMap<>();
     iterators.put("vers", EnumSet.of(IteratorScope.majc, IteratorScope.minc, IteratorScope.scan));
-    IteratorSetting combiner = new IteratorSetting(15, "replcombiner", StatusCombiner.class);
+    IteratorSetting combiner = new IteratorSetting(9, "replcombiner", StatusCombiner.class);
     Combiner.setColumns(combiner, Collections.singletonList(new Column(ReplicationSection.COLF)));
 
     expect(conn.tableOperations()).andReturn(tops);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/53e06a25/test/src/test/java/org/apache/accumulo/test/replication/ReplicationTablesMacTest.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/replication/ReplicationTablesMacTest.java b/test/src/test/java/org/apache/accumulo/test/replication/ReplicationTablesMacTest.java
new file mode 100644
index 0000000..00524b8
--- /dev/null
+++ b/test/src/test/java/org/apache/accumulo/test/replication/ReplicationTablesMacTest.java
@@ -0,0 +1,94 @@
+/*
+ * 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.replication;
+
+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.data.Key;
+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.StatusUtil;
+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.minicluster.impl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.server.util.ReplicationTableUtil;
+import org.apache.accumulo.test.functional.ConfigurableMacIT;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Text;
+import org.junit.Assert;
+import org.junit.Test;
+
+import com.google.common.collect.Iterables;
+
+/**
+ * 
+ */
+public class ReplicationTablesMacTest extends ConfigurableMacIT {
+
+  @Override
+  public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
+    cfg.setNumTservers(1);
+  }
+
+  @Test
+  public void combinerWorksOnMetadata() throws Exception {
+    Connector conn = getConnector();
+
+    conn.securityOperations().grantTablePermission("root", MetadataTable.NAME, TablePermission.WRITE);
+
+    ReplicationTableUtil.configureMetadataTable(conn, MetadataTable.NAME);
+
+    Status stat1 = StatusUtil.fileCreated(100);
+    Status stat2 = StatusUtil.fileClosed();
+
+    BatchWriter bw = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
+    Mutation m = new Mutation(ReplicationSection.getRowPrefix() + "file:/accumulo/wals/tserver+port/uuid");
+    m.put(ReplicationSection.COLF, new Text("1"), ProtobufUtil.toValue(stat1));
+    bw.addMutation(m);
+    bw.close();
+
+    Scanner s = conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
+    s.setRange(ReplicationSection.getRange());
+    System.out.println("Printing metadata table");
+
+    Status actual = Status.parseFrom(Iterables.getOnlyElement(s).getValue().get());
+    Assert.assertEquals(stat1, actual);
+
+    bw = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
+    m = new Mutation(ReplicationSection.getRowPrefix() + "file:/accumulo/wals/tserver+port/uuid");
+    m.put(ReplicationSection.COLF, new Text("1"), ProtobufUtil.toValue(stat2));
+    bw.addMutation(m);
+    bw.close();
+
+    s = conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
+    s.setRange(ReplicationSection.getRange());
+
+    actual = Status.parseFrom(Iterables.getOnlyElement(s).getValue().get());
+    Status expected = Status.newBuilder().setBegin(0).setEnd(0).setClosed(true).setInfiniteEnd(true).setCreatedTime(100).build();
+
+    Assert.assertEquals(expected, actual);
+  }
+
+}


[17/50] [abbrv] git commit: Merge branch '1.6.1-SNAPSHOT'

Posted by el...@apache.org.
Merge branch '1.6.1-SNAPSHOT'


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

Branch: refs/heads/ACCUMULO-378
Commit: dc69bc80ae0413d16e5ee3681f9a9a3660c5f819
Parents: 18591bc 10ce7af
Author: Christopher Tubbs <ct...@apache.org>
Authored: Fri May 16 12:58:47 2014 -0400
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Fri May 16 12:58:47 2014 -0400

----------------------------------------------------------------------
 server/native/src/main/resources/Makefile | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------



[12/50] [abbrv] git commit: Merge branch '1.5.2-SNAPSHOT' into 1.6.1-SNAPSHOT

Posted by el...@apache.org.
Merge branch '1.5.2-SNAPSHOT' into 1.6.1-SNAPSHOT


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

Branch: refs/heads/ACCUMULO-378
Commit: 01e1181bfe53b5168cfc29b0cd3d8520daea418a
Parents: fd80bc6 f83f9f9
Author: Christopher Tubbs <ct...@apache.org>
Authored: Fri May 16 12:49:12 2014 -0400
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Fri May 16 12:49:12 2014 -0400

----------------------------------------------------------------------
 bin/generate_monitor_certificate.sh | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------



[23/50] [abbrv] git commit: ACCUMULO-2816 Ensure LocalWALRecovery looks at WAL.

Posted by el...@apache.org.
ACCUMULO-2816 Ensure LocalWALRecovery looks at WAL.

* changed to use of FileSystem api in test to avoid crc handling.
* added better error message for reading an invalid wal
* fixed v2 vs v3 magic header reading


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

Branch: refs/heads/ACCUMULO-378
Commit: 244c1ab71c4b04794cbd718fc34c7fc2216774c4
Parents: f61abc0
Author: Sean Busbey <bu...@cloudera.com>
Authored: Fri May 16 15:27:18 2014 -0400
Committer: Sean Busbey <bu...@cloudera.com>
Committed: Fri May 16 19:27:05 2014 -0400

----------------------------------------------------------------------
 .../apache/accumulo/tserver/log/DfsLogger.java  |  4 ++--
 .../accumulo/tserver/logger/LogFileKey.java     |  3 +++
 .../tserver/log/LocalWALRecoveryTest.java       | 22 ++++++--------------
 3 files changed, 11 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/244c1ab7/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 eb04f09..61c1059 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
@@ -269,7 +269,7 @@ public class DfsLogger {
     } else {
       input.seek(0);
       byte[] magicV2 = DfsLogger.LOG_FILE_HEADER_V2.getBytes();
-      byte[] magicBufferV2 = new byte[magic.length];
+      byte[] magicBufferV2 = new byte[magicV2.length];
       input.readFully(magicBufferV2);
 
       if (Arrays.equals(magicBufferV2, magicV2)) {
@@ -299,7 +299,7 @@ public class DfsLogger {
           CryptoModuleParameters params = CryptoModuleFactory.createParamsObjectFromAccumuloConfiguration(conf);
 
           input.seek(0);
-          input.readFully(magicBuffer);
+          input.readFully(magicBufferV2);
           params.setEncryptedInputStream(input);
 
           params = cryptoModule.getDecryptingInputStream(params);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/244c1ab7/server/tserver/src/main/java/org/apache/accumulo/tserver/logger/LogFileKey.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/logger/LogFileKey.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/logger/LogFileKey.java
index 1e33569..3a20e8d 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/logger/LogFileKey.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/logger/LogFileKey.java
@@ -41,6 +41,9 @@ public class LogFileKey implements WritableComparable<LogFileKey> {
   @Override
   public void readFields(DataInput in) throws IOException {
     int value = in.readByte();
+    if (value >= LogEvents.values().length) {
+      throw new IOException("Invalid LogEvent type, got ordinal " + value + ", but only know about " + LogEvents.values().length + " possible types.");
+    }
     event = LogEvents.values()[value];
     switch (event) {
       case OPEN:

http://git-wip-us.apache.org/repos/asf/accumulo/blob/244c1ab7/server/tserver/src/test/java/org/apache/accumulo/tserver/log/LocalWALRecoveryTest.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/log/LocalWALRecoveryTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/log/LocalWALRecoveryTest.java
index 99190b2..df6ec2d 100644
--- a/server/tserver/src/test/java/org/apache/accumulo/tserver/log/LocalWALRecoveryTest.java
+++ b/server/tserver/src/test/java/org/apache/accumulo/tserver/log/LocalWALRecoveryTest.java
@@ -24,9 +24,7 @@ import static org.junit.Assert.assertEquals;
 import java.io.DataInputStream;
 import java.io.EOFException;
 import java.io.File;
-import java.io.FilenameFilter;
 import java.io.IOException;
-import java.util.UUID;
 
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
@@ -36,6 +34,7 @@ import org.apache.accumulo.tserver.log.DfsLogger.DFSLoggerInputStreams;
 import org.apache.accumulo.tserver.logger.LogFileKey;
 import org.apache.accumulo.tserver.logger.LogFileValue;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.junit.Before;
@@ -69,23 +68,14 @@ public class LocalWALRecoveryTest {
 
   @Test
   public void testRecoverLocalWriteAheadLogs() throws IOException {
-    FileSystem fs = FileSystem.get(walTarget.toURI(), new Configuration());
+    Path targetPath = new Path(walTarget.toURI());
+    FileSystem fs = FileSystem.get(targetPath.toUri(), new Configuration());
     recovery.recoverLocalWriteAheadLogs(fs);
 
-    assertEquals("Wrong number of WAL files recovered.", 1, walTarget.list(new FilenameFilter() {
-      @Override
-      public boolean accept(File dir, String name) {
-        try {
-          // Filter out the CRC file
-          UUID.fromString(name);
-          return true;
-        } catch (IllegalArgumentException e) {
-          return false;
-        }
-      }
-    }).length);
+    FileStatus[] recovered = fs.listStatus(targetPath);
+    assertEquals("Wrong number of WAL files recovered.", 1, recovered.length);
 
-    final Path path = new Path(walTarget.listFiles()[0].getAbsolutePath());
+    final Path path = recovered[0].getPath();
     final VolumeManager volumeManager = VolumeManagerImpl.getLocal(folder.getRoot().getAbsolutePath());
 
     final DFSLoggerInputStreams streams = DfsLogger.readHeaderAndReturnStream(volumeManager, path, configuration);


[07/50] [abbrv] git commit: ACCUMULO-378 Only set the Replication table combiner on the status and work columns (unnecessary for order columns)

Posted by el...@apache.org.
ACCUMULO-378 Only set the Replication table combiner on the status and work columns (unnecessary for order columns)


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

Branch: refs/heads/ACCUMULO-378
Commit: 005b59f3d84963d25ee091dca0f525bd09b14042
Parents: f97f13a
Author: Josh Elser <el...@apache.org>
Authored: Thu May 15 22:06:38 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Thu May 15 22:06:38 2014 -0400

----------------------------------------------------------------------
 .../org/apache/accumulo/server/replication/ReplicationTable.java | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/005b59f3/server/base/src/main/java/org/apache/accumulo/server/replication/ReplicationTable.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/replication/ReplicationTable.java b/server/base/src/main/java/org/apache/accumulo/server/replication/ReplicationTable.java
index df73604..622ec10 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/replication/ReplicationTable.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/replication/ReplicationTable.java
@@ -16,6 +16,7 @@
  */
 package org.apache.accumulo.server.replication;
 
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.EnumSet;
 import java.util.Map;
@@ -28,6 +29,7 @@ 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.IteratorSetting;
+import org.apache.accumulo.core.client.IteratorSetting.Column;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableExistsException;
 import org.apache.accumulo.core.client.TableNotFoundException;
@@ -117,7 +119,7 @@ public class ReplicationTable extends org.apache.accumulo.core.client.replicatio
     if (!iterators.containsKey(COMBINER_NAME)) {
       // Set our combiner and combine all columns
       IteratorSetting setting = new IteratorSetting(30, COMBINER_NAME, StatusCombiner.class);
-      Combiner.setCombineAllColumns(setting, true);
+      Combiner.setColumns(setting, Arrays.asList(new Column(StatusSection.NAME), new Column(WorkSection.NAME)));
       try {
         tops.attachIterator(NAME, setting);
       } catch (AccumuloSecurityException | AccumuloException | TableNotFoundException e) {


[43/50] [abbrv] git commit: ACCUMULO-378 Fix a failed test by reducing the initial delay and period of the DWQ for replication

Posted by el...@apache.org.
ACCUMULO-378 Fix a failed test by reducing the initial delay and period of the DWQ for replication


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

Branch: refs/heads/ACCUMULO-378
Commit: c3d99ec7fe29f1fa0acce9256475a8ec4c57cfd6
Parents: 215e2b5
Author: Josh Elser <el...@apache.org>
Authored: Tue May 20 13:45:28 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Tue May 20 13:45:28 2014 -0400

----------------------------------------------------------------------
 .../org/apache/accumulo/core/conf/Property.java |  4 +++
 .../server/zookeeper/DistributedWorkQueue.java  | 11 ++++++-
 .../replication/ReplicationProcessor.java       |  2 ++
 .../tserver/replication/ReplicationWorker.java  | 21 ++++++++++++-
 .../test/replication/ReplicationWithGCIT.java   | 32 +++++++++++++++-----
 test/src/test/resources/log4j.properties        |  2 +-
 6 files changed, 61 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/c3d99ec7/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 21aa578..b1ee499 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
@@ -482,6 +482,10 @@ public enum Property {
   @Experimental
   REPLICATION_WORK_ASSIGNER("replication.work.assigner", "org.apache.accumulo.master.replication.DistributedWorkQueueWorkAssigner", PropertyType.CLASSNAME,
       "Replication WorkAssigner implementation to use"),
+  @Experimental
+  REPLICATION_WORK_PROCESSOR_DELAY("replication.work.processor.delay", "0s", PropertyType.TIMEDURATION, "Amount of time to wait before first checking for replication work"),
+  @Experimental
+  REPLICATION_WORK_PROCESSOR_PERIOD("replication.work.processor.period", "0s", PropertyType.TIMEDURATION, "Amount of time to wait before re-checking for replication work"),
 
   ;
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c3d99ec7/server/base/src/main/java/org/apache/accumulo/server/zookeeper/DistributedWorkQueue.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/zookeeper/DistributedWorkQueue.java b/server/base/src/main/java/org/apache/accumulo/server/zookeeper/DistributedWorkQueue.java
index b1907fb..29c8fcf 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/zookeeper/DistributedWorkQueue.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/zookeeper/DistributedWorkQueue.java
@@ -50,6 +50,7 @@ public class DistributedWorkQueue {
   private ZooReaderWriter zoo = ZooReaderWriter.getInstance();
   private String path;
   private AccumuloConfiguration config;
+  private long timerInitialDelay, timerPeriod;
 
   private AtomicInteger numTask = new AtomicInteger(0);
 
@@ -151,8 +152,15 @@ public class DistributedWorkQueue {
   }
   
   public DistributedWorkQueue(String path, AccumuloConfiguration config) {
+    // Preserve the old delay and period
+    this(path, config, new Random().nextInt(60*1000), 60*1000);
+  }
+
+  public DistributedWorkQueue(String path, AccumuloConfiguration config, long timerInitialDelay, long timerPeriod) {
     this.path = path;
     this.config = config;
+    this.timerInitialDelay = timerInitialDelay;
+    this.timerPeriod = timerPeriod;
   }
   
   public void startProcessing(final Processor processor, ThreadPoolExecutor executorService) throws KeeperException, InterruptedException {
@@ -196,6 +204,7 @@ public class DistributedWorkQueue {
     SimpleTimer.getInstance(config).schedule(new Runnable() {
       @Override
       public void run() {
+        log.debug("Looking for work in " + path);
         try {
           lookForWork(processor, zoo.getChildren(path));
         } catch (KeeperException e) {
@@ -204,7 +213,7 @@ public class DistributedWorkQueue {
           log.info("Interrupted looking for work", e);
         }
       }
-    }, r.nextInt(60 * 1000), 60 * 1000);
+    }, timerInitialDelay, timerPeriod);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c3d99ec7/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/ReplicationProcessor.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/ReplicationProcessor.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/ReplicationProcessor.java
index 481b3e8..149af6e 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/ReplicationProcessor.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/ReplicationProcessor.java
@@ -123,6 +123,8 @@ public class ReplicationProcessor implements Processor {
     // Replicate that sucker
     Status replicatedStatus = replica.replicate(filePath, status, target);
 
+    log.debug("Completed replication of {} to {}, with new Status [{}]", filePath, target, ProtobufUtil.toString(replicatedStatus));
+
     // If we got a different status
     if (!replicatedStatus.equals(status)) {
       // We actually did some work!

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c3d99ec7/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/ReplicationWorker.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/ReplicationWorker.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/ReplicationWorker.java
index 01258de..a223511 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/ReplicationWorker.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/ReplicationWorker.java
@@ -21,16 +21,21 @@ import java.util.concurrent.ThreadPoolExecutor;
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.conf.DefaultConfiguration;
+import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.security.SystemCredentials;
 import org.apache.accumulo.server.zookeeper.DistributedWorkQueue;
 import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * 
  */
 public class ReplicationWorker implements Runnable {
+  private static final Logger log = LoggerFactory.getLogger(ReplicationWorker.class);
 
   private Instance inst;
   private VolumeManager fs;
@@ -49,8 +54,22 @@ public class ReplicationWorker implements Runnable {
 
   @Override
   public void run() {
+    DefaultConfiguration defaultConf = DefaultConfiguration.getDefaultConfiguration();
+    long defaultDelay = defaultConf.getTimeInMillis(Property.REPLICATION_WORK_PROCESSOR_DELAY);
+    long defaultPeriod = defaultConf.getTimeInMillis(Property.REPLICATION_WORK_PROCESSOR_PERIOD);
+    long delay = conf.getTimeInMillis(Property.REPLICATION_WORK_PROCESSOR_DELAY);
+    long period = conf.getTimeInMillis(Property.REPLICATION_WORK_PROCESSOR_PERIOD);
     try {
-      new DistributedWorkQueue(ZooUtil.getRoot(inst) + Constants.ZREPLICATION_WORK_QUEUE, conf).startProcessing(new ReplicationProcessor(inst, conf, fs, SystemCredentials.get()), executor);
+      DistributedWorkQueue workQueue;
+      if (defaultDelay != delay && defaultPeriod != period) {
+        log.debug("Configuration DistributedWorkQueue with delay and period of {} and {}", delay, period);
+        workQueue = new DistributedWorkQueue(ZooUtil.getRoot(inst) + Constants.ZREPLICATION_WORK_QUEUE, conf, delay, period);
+      } else {
+        log.debug("Configuring DistributedWorkQueue with default delay and period");
+        workQueue = new DistributedWorkQueue(ZooUtil.getRoot(inst) + Constants.ZREPLICATION_WORK_QUEUE, conf);
+      }
+
+      workQueue.startProcessing(new ReplicationProcessor(inst, conf, fs, SystemCredentials.get()), executor);
     } catch (KeeperException | InterruptedException e) {
       throw new RuntimeException(e);
     }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c3d99ec7/test/src/test/java/org/apache/accumulo/test/replication/ReplicationWithGCIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/replication/ReplicationWithGCIT.java b/test/src/test/java/org/apache/accumulo/test/replication/ReplicationWithGCIT.java
index 23da719..2a7a210 100644
--- a/test/src/test/java/org/apache/accumulo/test/replication/ReplicationWithGCIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/replication/ReplicationWithGCIT.java
@@ -83,6 +83,8 @@ public class ReplicationWithGCIT extends ConfigurableMacIT {
     cfg.setProperty(Property.MASTER_REPLICATION_SCAN_INTERVAL, "1s");
     cfg.setProperty(Property.REPLICATION_WORK_ASSIGNMENT_SLEEP, "1s");
     cfg.setProperty(Property.REPLICATION_NAME, "master");
+    cfg.setProperty(Property.REPLICATION_WORK_PROCESSOR_DELAY, "1s");
+    cfg.setProperty(Property.REPLICATION_WORK_PROCESSOR_PERIOD, "1s");
     cfg.setNumTservers(1);
     hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
   }
@@ -331,6 +333,11 @@ public class ReplicationWithGCIT extends ConfigurableMacIT {
 
   @Test(timeout = 5 * 60 * 1000)
   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);
+    }
+
     final Connector conn = getConnector();
     log.info("Got connector to MAC");
     String table1 = "table1";
@@ -470,7 +477,12 @@ public class ReplicationWithGCIT extends ConfigurableMacIT {
     boolean foundResults = false;
     for (int i = 0; i < 5; i++) {
       s = ReplicationTable.getScanner(conn);
-      if (Iterables.size(s) > 0) {
+      int count = 0;
+      for (Entry<Key,Value> entry : s) {
+        count++;
+        log.info("{}={}", entry.getKey().toStringNoTruncate(), entry.getValue());
+      }
+      if (count > 0) {
         foundResults = true;
         break;
       }
@@ -479,9 +491,16 @@ public class ReplicationWithGCIT extends ConfigurableMacIT {
 
     Assert.assertTrue("Did not find any replication entries in the replication table", foundResults);
 
+    getCluster().exec(SimpleGarbageCollector.class);
+
+    // Wait for a bit since the GC has to run (should be running after a one second delay)
+    Thread.sleep(5000);
+
     // 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 record after we create the record in the replication table
+    // 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;
     for (int i = 0; i < 5; i++) {
       s = conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
@@ -493,16 +512,13 @@ public class ReplicationWithGCIT extends ConfigurableMacIT {
       Thread.sleep(1000);
     }
 
-    Assert.assertFalse("Replication status messages were not cleaned up from metadata table, check why the StatusMaker didn't delete them", foundResults);
+    Assert.assertFalse("Replication status messages were not cleaned up from metadata table", foundResults);
 
     /**
-     * After we set the begin to Long.MAX_VALUE, the RemoveCompleteReplicationRecords class will start deleting the records which have been closed by
-     * the minor compaction and replicated by the MockReplicaSystem
+     * After we close out and subsequently delete the metadata record, this will propagate to the replication table,
+     * which will cause those records to be deleted after repliation occurs
      */
 
-    // Wait for a bit since the GC has to run (should be running after a one second delay)
-    Thread.sleep(5000);
-
     int recordsFound = 0;
     for (int i = 0; i < 10; i++) {
       s = ReplicationTable.getScanner(conn);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/c3d99ec7/test/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/test/src/test/resources/log4j.properties b/test/src/test/resources/log4j.properties
index 407bc28..9f991d1 100644
--- a/test/src/test/resources/log4j.properties
+++ b/test/src/test/resources/log4j.properties
@@ -35,7 +35,7 @@ log4j.logger.org.apache.accumulo.core.file.rfile.bcfile=INFO
 log4j.logger.org.apache.accumulo.server.util.ReplicationTableUtil=TRACE
 log4j.logger.org.apache.accumulo.core.client.impl.TabletServerBatchReaderIterator=INFO
 log4j.logger.org.apache.accumulo.core.client.impl.ThriftScanner=INFO
-log4j.logger.org.apache.accumulo.server.zookeeper.DistributedWorkQueue=INFO
+#log4j.logger.org.apache.accumulo.server.zookeeper.DistributedWorkQueue=INFO
 log4j.logger.org.apache.accumulo.fate.zookeeper.DistributedReadWriteLock=WARN
 log4j.logger.org.mortbay.log=WARN
 log4j.logger.org.apache.hadoop=WARN


[02/50] [abbrv] git commit: ACCUMULO-2761: Add regex to remove everything after the last slash

Posted by el...@apache.org.
ACCUMULO-2761: Add regex to remove everything after the last slash


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

Branch: refs/heads/ACCUMULO-378
Commit: ff17423fae883f51d2a837d1eb5359437440fbc7
Parents: 0cdd5fd
Author: Dave Marion <dl...@hotmail.com>
Authored: Thu May 15 19:39:26 2014 -0400
Committer: Dave Marion <dl...@hotmail.com>
Committed: Thu May 15 19:39:26 2014 -0400

----------------------------------------------------------------------
 bin/bootstrap_hdfs.sh | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/ff17423f/bin/bootstrap_hdfs.sh
----------------------------------------------------------------------
diff --git a/bin/bootstrap_hdfs.sh b/bin/bootstrap_hdfs.sh
index 0a3f866..6f38f63 100755
--- a/bin/bootstrap_hdfs.sh
+++ b/bin/bootstrap_hdfs.sh
@@ -30,7 +30,7 @@ bin="$( cd -P "$( dirname "$SOURCE" )" && pwd )"
 #
 # Find the system context directory in HDFS
 #
-SYSTEM_CONTEXT_HDFS_DIR=$(grep -A1 "general.vfs.classpaths" "$ACCUMULO_CONF_DIR/accumulo-site.xml" | tail -1 | perl -pe 's/\s+<value>//; s/<\/value>//; print $ARGV[1]')
+SYSTEM_CONTEXT_HDFS_DIR=$(grep -A1 "general.vfs.classpaths" "$ACCUMULO_CONF_DIR/accumulo-site.xml" | tail -1 | perl -pe 's/\s+<value>//; s/<\/value>//; s|[^/]+$||; print $ARGV[1]')
 
 if [ -z "$SYSTEM_CONTEXT_HDFS_DIR" ]
 then


[32/50] [abbrv] git commit: ACCUMULO-378 Remove a bogus log message

Posted by el...@apache.org.
ACCUMULO-378 Remove a bogus log message


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

Branch: refs/heads/ACCUMULO-378
Commit: 2ff3a6cd14a609f70600ee84e985466354855913
Parents: 410a5ec
Author: Josh Elser <el...@apache.org>
Authored: Mon May 19 13:54:50 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Mon May 19 13:54:50 2014 -0400

----------------------------------------------------------------------
 .../master/replication/RemoveCompleteReplicationRecords.java       | 2 --
 1 file changed, 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/2ff3a6cd/server/master/src/main/java/org/apache/accumulo/master/replication/RemoveCompleteReplicationRecords.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/replication/RemoveCompleteReplicationRecords.java b/server/master/src/main/java/org/apache/accumulo/master/replication/RemoveCompleteReplicationRecords.java
index 5a89842..e661174 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/replication/RemoveCompleteReplicationRecords.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/replication/RemoveCompleteReplicationRecords.java
@@ -64,8 +64,6 @@ public class RemoveCompleteReplicationRecords implements Runnable {
   public void run() {
     BatchScanner bs;
     BatchWriter bw;
-    // Run over the metadata table first to reduce the likelihood of the master re-creating 
-    // status records that we are in the middle of cleaning up (does not imply double-duplication though)
     try {
       bs = ReplicationTable.getBatchScanner(conn, 4);
       bw = ReplicationTable.getBatchWriter(conn);


[41/50] [abbrv] git commit: Merge remote-tracking branch 'origin/master' into ACCUMULO-378

Posted by el...@apache.org.
Merge remote-tracking branch 'origin/master' into ACCUMULO-378


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

Branch: refs/heads/ACCUMULO-378
Commit: 1b4acdf99ba082dce58f21b1a0dcc57187a13388
Parents: d5c863d b510b76
Author: Josh Elser <el...@apache.org>
Authored: Tue May 20 10:52:19 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Tue May 20 10:52:19 2014 -0400

----------------------------------------------------------------------
 README                                          |  7 +++++++
 bin/bootstrap_hdfs.sh                           |  2 +-
 bin/build_native_library.sh                     |  1 +
 bin/generate_monitor_certificate.sh             |  6 +++---
 .../accumulo/core/util/LocalityGroupUtil.java   |  6 +++---
 .../main/asciidoc/chapters/administration.txt   | 19 +++++++++++------
 server/native/src/main/resources/Makefile       |  7 ++++---
 .../apache/accumulo/tserver/log/DfsLogger.java  |  4 ++--
 .../accumulo/tserver/logger/LogFileKey.java     |  3 +++
 .../tserver/log/LocalWALRecoveryTest.java       | 22 ++++++--------------
 .../classloader/vfs/UniqueFileReplicator.java   |  4 ++--
 11 files changed, 45 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/1b4acdf9/server/tserver/src/test/java/org/apache/accumulo/tserver/log/LocalWALRecoveryTest.java
----------------------------------------------------------------------
diff --cc server/tserver/src/test/java/org/apache/accumulo/tserver/log/LocalWALRecoveryTest.java
index a678d41,df6ec2d..1a8909e
--- a/server/tserver/src/test/java/org/apache/accumulo/tserver/log/LocalWALRecoveryTest.java
+++ b/server/tserver/src/test/java/org/apache/accumulo/tserver/log/LocalWALRecoveryTest.java
@@@ -67,25 -66,16 +66,16 @@@ public class LocalWALRecoveryTest 
      recovery.parseArgs("--dfs-wal-directory", walTarget.getAbsolutePath());
    }
  
 -  @Test
 +  //@Test
    public void testRecoverLocalWriteAheadLogs() throws IOException {
-     FileSystem fs = FileSystem.get(walTarget.toURI(), new Configuration());
+     Path targetPath = new Path(walTarget.toURI());
+     FileSystem fs = FileSystem.get(targetPath.toUri(), new Configuration());
      recovery.recoverLocalWriteAheadLogs(fs);
  
-     assertEquals("Wrong number of WAL files recovered.", 1, walTarget.list(new FilenameFilter() {
-       @Override
-       public boolean accept(File dir, String name) {
-         try {
-           // Filter out the CRC file
-           UUID.fromString(name);
-           return true;
-         } catch (IllegalArgumentException e) {
-           return false;
-         }
-       }
-     }).length);
+     FileStatus[] recovered = fs.listStatus(targetPath);
+     assertEquals("Wrong number of WAL files recovered.", 1, recovered.length);
  
-     final Path path = new Path(walTarget.listFiles()[0].getAbsolutePath());
+     final Path path = recovered[0].getPath();
      final VolumeManager volumeManager = VolumeManagerImpl.getLocal(folder.getRoot().getAbsolutePath());
  
      final DFSLoggerInputStreams streams = DfsLogger.readHeaderAndReturnStream(volumeManager, path, configuration);


[15/50] [abbrv] git commit: Merge branch '1.5.2-SNAPSHOT' into 1.6.1-SNAPSHOT -sours

Posted by el...@apache.org.
Merge branch '1.5.2-SNAPSHOT' into 1.6.1-SNAPSHOT -sours


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

Branch: refs/heads/ACCUMULO-378
Commit: e86bfc29384cc33f472cfe4f020ba4031da24608
Parents: 01e1181 c16f105
Author: Christopher Tubbs <ct...@apache.org>
Authored: Fri May 16 12:57:34 2014 -0400
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Fri May 16 12:57:34 2014 -0400

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

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



[46/50] [abbrv] git commit: ACCUMULO-378 Utility to just print the replication records repeatedly

Posted by el...@apache.org.
ACCUMULO-378 Utility to just print the replication records repeatedly


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

Branch: refs/heads/ACCUMULO-378
Commit: 2817f42d4d2758b8fe8b9789d992a2ae06ec5486
Parents: 50117b3
Author: Josh Elser <el...@apache.org>
Authored: Tue May 20 18:47:32 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Tue May 20 18:47:32 2014 -0400

----------------------------------------------------------------------
 .../replication/PrintReplicationRecords.java    | 98 ++++++++++++++++++++
 .../ReplicationTablesPrinterThread.java         | 51 ++++++++++
 2 files changed, 149 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/2817f42d/core/src/main/java/org/apache/accumulo/core/replication/PrintReplicationRecords.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/replication/PrintReplicationRecords.java b/core/src/main/java/org/apache/accumulo/core/replication/PrintReplicationRecords.java
new file mode 100644
index 0000000..60aae9c
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/replication/PrintReplicationRecords.java
@@ -0,0 +1,98 @@
+/*
+ * 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.replication;
+
+import java.io.PrintStream;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.Map.Entry;
+
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.replication.ReplicationTable;
+import org.apache.accumulo.core.data.Key;
+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.proto.Replication.Status;
+import org.apache.accumulo.core.security.Authorizations;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.protobuf.InvalidProtocolBufferException;
+
+/**
+ * 
+ */
+public class PrintReplicationRecords implements Runnable {
+  private static final Logger log = LoggerFactory.getLogger(PrintReplicationRecords.class);
+
+  private Connector conn;
+  private PrintStream out;
+  private SimpleDateFormat sdf;
+
+  public PrintReplicationRecords(Connector conn,PrintStream out) {
+    this.conn = conn;
+    this.out = out;
+    this.sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss,SSS");
+  }
+
+  @Override
+  public void run() {
+    Scanner s;
+
+    out.println(sdf.format(new Date()) + " Replication entries from metadata table");
+    out.println("------------------------------------------------------------------");
+    try {
+      s = conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
+    } catch (TableNotFoundException e) {
+      log.error("Metadata table does not exist");
+      return;
+    }
+
+    s.setRange(ReplicationSection.getRange());
+    s.fetchColumnFamily(ReplicationSection.COLF);
+    for (Entry<Key,Value> entry : s) {
+      try {
+        out.println(entry.getKey().toStringNoTruncate() + "=" + ProtobufUtil.toString(Status.parseFrom(entry.getValue().get())));
+      } catch (InvalidProtocolBufferException e) {
+        out.println(entry.getKey().toStringNoTruncate() + "= Could not deserialize Status message");
+      }
+    }
+
+    out.println();
+    out.println(sdf.format(new Date()) + "Replication entries from replication table");
+    out.println("--------------------------------------------------------------------");
+
+    try {
+      s = conn.createScanner(ReplicationTable.NAME, Authorizations.EMPTY);
+    } catch (TableNotFoundException e) {
+      log.error("Replication table does not exist");
+      return;
+    }
+
+    for (Entry<Key,Value> entry : s) {
+      try {
+        out.println(entry.getKey().toStringNoTruncate() + "=" + ProtobufUtil.toString(Status.parseFrom(entry.getValue().get())));
+      } catch (InvalidProtocolBufferException e) {
+        out.println(entry.getKey().toStringNoTruncate() + "= Could not deserialize Status message");
+      }
+    }    
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/2817f42d/test/src/main/java/org/apache/accumulo/test/replication/ReplicationTablesPrinterThread.java
----------------------------------------------------------------------
diff --git a/test/src/main/java/org/apache/accumulo/test/replication/ReplicationTablesPrinterThread.java b/test/src/main/java/org/apache/accumulo/test/replication/ReplicationTablesPrinterThread.java
new file mode 100644
index 0000000..750557c
--- /dev/null
+++ b/test/src/main/java/org/apache/accumulo/test/replication/ReplicationTablesPrinterThread.java
@@ -0,0 +1,51 @@
+/*
+ * 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.replication;
+
+import java.io.PrintStream;
+
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.replication.PrintReplicationRecords;
+import org.apache.accumulo.core.util.Daemon;
+
+/**
+ * 
+ */
+public class ReplicationTablesPrinterThread extends Daemon {
+
+  private PrintStream out;
+  private PrintReplicationRecords printer;
+
+  public ReplicationTablesPrinterThread(Connector conn, PrintStream out) {
+    printer = new PrintReplicationRecords(conn, out);
+    this.out = out;
+  }
+
+  @Override
+  public void run() {
+    while (true) {
+      printer.run();
+      out.println();
+      try {
+        Thread.sleep(1000);
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+        return;
+      }
+    }
+  }
+}


[45/50] [abbrv] git commit: ACCUMULO-2819 Add a test to ensure that an inprogress file that's fully replicated is removed.

Posted by el...@apache.org.
ACCUMULO-2819 Add a test to ensure that an inprogress file that's fully replicated is removed.

This is mainly an optimization as it would be cleaned up outside of the
work assignment loop, but it should help to make a more responsive work assignment loop.


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

Branch: refs/heads/ACCUMULO-378
Commit: 50117b35535f7701ebde58f089489dd8dbcd432d
Parents: 57dfe48
Author: Josh Elser <el...@apache.org>
Authored: Tue May 20 14:06:14 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Tue May 20 14:06:14 2014 -0400

----------------------------------------------------------------------
 .../replication/SequentialWorkAssignerTest.java | 76 ++++++++++++++++++++
 1 file changed, 76 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/50117b35/server/master/src/test/java/org/apache/accumulo/master/replication/SequentialWorkAssignerTest.java
----------------------------------------------------------------------
diff --git a/server/master/src/test/java/org/apache/accumulo/master/replication/SequentialWorkAssignerTest.java b/server/master/src/test/java/org/apache/accumulo/master/replication/SequentialWorkAssignerTest.java
index ebc540f..0820e1c 100644
--- a/server/master/src/test/java/org/apache/accumulo/master/replication/SequentialWorkAssignerTest.java
+++ b/server/master/src/test/java/org/apache/accumulo/master/replication/SequentialWorkAssignerTest.java
@@ -345,4 +345,80 @@ public class SequentialWorkAssignerTest {
     Assert.assertEquals(1, cluster1Work.size());
     Assert.assertEquals(AbstractWorkAssigner.getQueueKey("file2", new ReplicationTarget("cluster1", "2", "2")), cluster1Work.get("2"));
   }
+
+  @Test
+  public void reprocessingOfCompletedWorkRemovesWork() throws Exception {
+    ReplicationTarget target = new ReplicationTarget("cluster1", "table1", "1");
+    Text serializedTarget = target.toText();
+
+    MockInstance inst = new MockInstance(test.getMethodName());
+    Credentials creds = new Credentials("root", new PasswordToken(""));
+    Connector conn = inst.getConnector(creds.getPrincipal(), creds.getToken());
+
+    // Set the connector
+    assigner.setConnector(conn);
+
+    // Create and grant ourselves write to the replication table
+    ReplicationTable.create(conn);
+    conn.securityOperations().grantTablePermission("root", ReplicationTable.NAME, TablePermission.WRITE);
+
+    // Create two mutations, both of which need replication work done
+    BatchWriter bw = ReplicationTable.getBatchWriter(conn);
+    // We want the name of file2 to sort before file1
+    String filename1 = "z_file1", filename2 = "a_file1";
+    String file1 = "/accumulo/wal/tserver+port/" + filename1, file2 = "/accumulo/wal/tserver+port/" + filename2;
+
+    // File1 was closed before file2, however
+    Status stat1 = Status.newBuilder().setBegin(100).setEnd(100).setClosed(true).setInfiniteEnd(false).setClosedTime(250).build();
+    Status stat2 = Status.newBuilder().setBegin(0).setEnd(100).setClosed(true).setInfiniteEnd(false).setClosedTime(500).build();
+
+    Mutation m = new Mutation(file1);
+    WorkSection.add(m, serializedTarget, ProtobufUtil.toValue(stat1));
+    bw.addMutation(m);
+
+    m = new Mutation(file2);
+    WorkSection.add(m, serializedTarget, ProtobufUtil.toValue(stat2));
+    bw.addMutation(m);
+
+    m = OrderSection.createMutation(file1, stat1.getClosedTime());
+    OrderSection.add(m, new Text(target.getSourceTableId()), ProtobufUtil.toValue(stat1));
+    bw.addMutation(m);
+
+    m = OrderSection.createMutation(file2, stat2.getClosedTime());
+    OrderSection.add(m, new Text(target.getSourceTableId()), ProtobufUtil.toValue(stat2));
+    bw.addMutation(m);
+
+    bw.close();
+
+    DistributedWorkQueue workQueue = createMock(DistributedWorkQueue.class);
+
+    // Treat filename1 as we have already submitted it for replication
+    Map<String,Map<String,String>> queuedWork = new HashMap<>();
+    Map<String,String> queuedWorkForCluster = new HashMap<>();
+    queuedWorkForCluster.put(target.getSourceTableId(), AbstractWorkAssigner.getQueueKey(filename1, target));
+    queuedWork.put("cluster1", queuedWorkForCluster);
+
+    assigner.setQueuedWork(queuedWork);
+    assigner.setWorkQueue(workQueue);
+    assigner.setMaxQueueSize(Integer.MAX_VALUE);
+
+    // Make sure we expect the invocations in the correct order (accumulo is sorted)
+    workQueue.addWork(AbstractWorkAssigner.getQueueKey(filename2, target), file2);
+    expectLastCall().once();
+
+    // file2 is *not* queued because file1 must be replicated first
+
+    replay(workQueue);
+
+    assigner.createWork();
+
+    verify(workQueue);
+
+    Assert.assertEquals(1, queuedWork.size());
+    Assert.assertTrue(queuedWork.containsKey("cluster1"));
+    Map<String,String> cluster1Work = queuedWork.get("cluster1");
+    Assert.assertEquals(1, cluster1Work.size());
+    Assert.assertTrue(cluster1Work.containsKey(target.getSourceTableId()));
+    Assert.assertEquals(AbstractWorkAssigner.getQueueKey(filename2, target), cluster1Work.get(target.getSourceTableId()));
+  }
 }


[34/50] [abbrv] git commit: ACCUMULO-2799 Split the drain methods to fetch the set of files needing replication and then waiting on those files

Posted by el...@apache.org.
ACCUMULO-2799 Split the drain methods to fetch the set of files needing replication and then waiting on those files


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

Branch: refs/heads/ACCUMULO-378
Commit: 26a88b4a34a47693607a177838d335e1fb85ea2f
Parents: eb4b9ce
Author: Josh Elser <el...@apache.org>
Authored: Mon May 19 15:39:25 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Mon May 19 15:39:25 2014 -0400

----------------------------------------------------------------------
 .../client/admin/ReplicationOperations.java     |  17 +++
 .../client/impl/ReplicationOperationsImpl.java  | 110 ++++++++++++-------
 2 files changed, 90 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/26a88b4a/core/src/main/java/org/apache/accumulo/core/client/admin/ReplicationOperations.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/ReplicationOperations.java b/core/src/main/java/org/apache/accumulo/core/client/admin/ReplicationOperations.java
index 4fd7d07..1680732 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/ReplicationOperations.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/ReplicationOperations.java
@@ -16,6 +16,8 @@
  */
 package org.apache.accumulo.core.client.admin;
 
+import java.util.Set;
+
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.TableNotFoundException;
@@ -58,4 +60,19 @@ public interface ReplicationOperations {
    * @throws AccumuloSecurityException
    */
   public void drain(String tableName) throws AccumuloException, AccumuloSecurityException, TableNotFoundException;
+
+  /**
+   * Wait for a table to be fully replicated as determined by the provided tables
+   * @param tableName The table to wait for
+   * @throws AccumuloException
+   * @throws AccumuloSecurityException
+   */
+  public void drain(String tableName, Set<String> files) throws AccumuloException, AccumuloSecurityException, TableNotFoundException;
+
+  /**
+   * Get all of the referenced files for a table
+   * @param tableName
+   * @throws TableNotFoundException
+   */
+  public Set<String> referencedFiles(String tableName) throws AccumuloException, AccumuloSecurityException, TableNotFoundException;
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/26a88b4a/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 d2698bd..752952d 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
@@ -115,7 +115,14 @@ public class ReplicationOperationsImpl implements ReplicationOperations {
   public void drain(String tableName) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
     checkNotNull(tableName);
 
-    log.debug("Waiting to drain {}", tableName);
+    Set<String> wals = referencedFiles(tableName);
+
+    drain(tableName, wals);
+  }
+
+  @Override
+  public void drain(String tableName, Set<String> wals) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
+    checkNotNull(tableName);
 
     Connector conn = inst.getConnector(creds.getPrincipal(), creds.getToken());
     TableOperations tops = conn.tableOperations();
@@ -124,7 +131,7 @@ public class ReplicationOperationsImpl implements ReplicationOperations {
     }
 
     if (!conn.tableOperations().exists(tableName)) {
-      throw new IllegalArgumentException("Table does not exist: " + tableName);
+      throw new TableNotFoundException(null, tableName, null);
     }
 
     String strTableId = null;
@@ -137,41 +144,6 @@ public class ReplicationOperationsImpl implements ReplicationOperations {
 
     Text tableId = new Text(strTableId);
 
-    log.debug("Found {} id for {}", strTableId, tableName);
-
-    // Get the WALs currently referenced by the table
-    BatchScanner metaBs = conn.createBatchScanner(MetadataTable.NAME, Authorizations.EMPTY, 4); 
-    metaBs.setRanges(Collections.singleton(MetadataSchema.TabletsSection.getRange(strTableId)));
-    metaBs.fetchColumnFamily(LogColumnFamily.NAME);
-    Set<String> wals = new HashSet<>();
-    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());
-        }
-      }
-    } finally {
-      metaBs.close();
-    }
-
-    // And the WALs that need to be replicated for this table
-    metaBs = conn.createBatchScanner(MetadataTable.NAME, Authorizations.EMPTY, 4);
-    metaBs.setRanges(Collections.singleton(ReplicationSection.getRange()));
-    metaBs.fetchColumnFamily(ReplicationSection.COLF);
-    try {
-      Text buffer = new Text();
-      for (Entry<Key,Value> entry : metaBs) {
-        ReplicationSection.getTableId(entry.getKey(), buffer);
-        if (buffer.equals(tableId)) {
-          ReplicationSection.getFile(entry.getKey(), buffer);
-          wals.add(buffer.toString());
-        }
-      }
-    } finally {
-      metaBs.close();
-    }
-
     log.info("Waiting for {} to be replicated for {}", wals, tableId);
 
     log.info("Reading from metadata table");
@@ -253,4 +225,68 @@ public class ReplicationOperationsImpl implements ReplicationOperations {
 
     return true;
   }
+
+  @Override
+  public Set<String> referencedFiles(String tableName) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
+    checkNotNull(tableName);
+
+    log.debug("Collecting referenced files for replication of table {}", tableName);
+
+    Connector conn = inst.getConnector(creds.getPrincipal(), creds.getToken());
+    TableOperations tops = conn.tableOperations();
+    while (!tops.exists(ReplicationTable.NAME)) {
+      UtilWaitThread.sleep(200);
+    }
+
+    if (!conn.tableOperations().exists(tableName)) {
+      throw new TableNotFoundException(null, tableName, null);
+    }
+
+    String strTableId = null;
+    while (null == strTableId) {
+      strTableId = tops.tableIdMap().get(tableName);
+      if (null == strTableId) {
+        UtilWaitThread.sleep(200);
+      }
+    }
+
+    Text tableId = new Text(strTableId);
+
+    log.debug("Found id of {} for name {}", strTableId, tableName);
+
+    // Get the WALs currently referenced by the table
+    BatchScanner metaBs = conn.createBatchScanner(MetadataTable.NAME, Authorizations.EMPTY, 4); 
+    metaBs.setRanges(Collections.singleton(MetadataSchema.TabletsSection.getRange(strTableId)));
+    metaBs.fetchColumnFamily(LogColumnFamily.NAME);
+    Set<String> wals = new HashSet<>();
+    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());
+        }
+      }
+    } finally {
+      metaBs.close();
+    }
+
+    // And the WALs that need to be replicated for this table
+    metaBs = conn.createBatchScanner(MetadataTable.NAME, Authorizations.EMPTY, 4);
+    metaBs.setRanges(Collections.singleton(ReplicationSection.getRange()));
+    metaBs.fetchColumnFamily(ReplicationSection.COLF);
+    try {
+      Text buffer = new Text();
+      for (Entry<Key,Value> entry : metaBs) {
+        ReplicationSection.getTableId(entry.getKey(), buffer);
+        if (buffer.equals(tableId)) {
+          ReplicationSection.getFile(entry.getKey(), buffer);
+          wals.add(buffer.toString());
+        }
+      }
+    } finally {
+      metaBs.close();
+    }
+
+    return wals;
+  }
 }


[49/50] [abbrv] git commit: ACCUMULO-2574 Change closedTime into createdTime

Posted by el...@apache.org.
ACCUMULO-2574 Change closedTime into createdTime

It's rather impossible to actually konw the exact point in time when
a WAL is no longer in use by any tablets. We can know when a tablet is
done with a WAL, but many tablets may be referencing it which convolutes
things.

Instead, we can get the same intent (global ordering of WALs for a tserver)
by recording the time in which the WAL was first created as this is something
that only occurs once in the lifetime of a file.


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

Branch: refs/heads/ACCUMULO-378
Commit: b53bbf08a35cf2ad82c9638fd81a67a671da71ab
Parents: 53e06a2
Author: Josh Elser <el...@apache.org>
Authored: Tue May 20 18:49:49 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Tue May 20 18:49:49 2014 -0400

----------------------------------------------------------------------
 .../accumulo/core/replication/StatusUtil.java   |  74 ++--
 .../core/replication/proto/Replication.java     |  94 ++---
 core/src/main/protobuf/replication.proto        |   2 +-
 .../core/replication/StatusUtilTest.java        |   2 +-
 .../apache/accumulo/server/fs/VolumeUtil.java   |   2 +-
 .../server/replication/StatusCombiner.java      |  20 +-
 .../server/replication/StatusCombinerTest.java  |  32 +-
 .../server/util/ReplicationTableUtilTest.java   |   7 +-
 .../CloseWriteAheadLogReferences.java           |   2 +-
 .../gc/GarbageCollectWriteAheadLogsTest.java    |  23 +-
 .../accumulo/gc/GarbageCollectionTest.java      |   2 +-
 .../CloseWriteAheadLogReferencesTest.java       |   4 +-
 .../RemoveCompleteReplicationRecords.java       |  15 +-
 .../master/replication/StatusMaker.java         |   7 +-
 .../DistributedWorkQueueWorkAssignerTest.java   |   4 +-
 .../RemoveCompleteReplicationRecordsTest.java   |   6 +-
 .../replication/SequentialWorkAssignerTest.java |  32 +-
 .../master/replication/StatusMakerTest.java     |  18 +-
 .../master/replication/WorkMakerTest.java       |  16 +-
 .../org/apache/accumulo/tserver/Tablet.java     |   2 +-
 .../tserver/log/TabletServerLogger.java         |   6 +-
 .../test/replication/ReplicationWithGCIT.java   | 341 ++++++++++---------
 .../replication/ReplicationWithMakerTest.java   |  12 +-
 .../test/replication/StatusCombinerMacTest.java |   5 +-
 test/src/test/resources/log4j.properties        |   1 +
 25 files changed, 385 insertions(+), 344 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/b53bbf08/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 94c60ab..721f1e0 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
@@ -28,47 +28,33 @@ import com.google.protobuf.InvalidProtocolBufferException;
  */
 public class StatusUtil {
 
-  private static final Status NEW_REPLICATION_STATUS, INF_END_REPLICATION_STATUS;
-  private static final Value NEW_REPLICATION_STATUS_VALUE, INF_END_REPLICATION_STATUS_VALUE;
+  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 CLOSED_STATUS_BUILDER;
+  private static final Status.Builder CREATED_STATUS_BUILDER;
 
   static {
-    Status.Builder builder = Status.newBuilder();
-    builder.setBegin(0);
-    builder.setEnd(0);
-    builder.setInfiniteEnd(false);
-    builder.setClosed(false);
-    NEW_REPLICATION_STATUS = builder.build();
-    NEW_REPLICATION_STATUS_VALUE = ProtobufUtil.toValue(NEW_REPLICATION_STATUS);
+    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);
 
-    CLOSED_STATUS_BUILDER = Status.newBuilder();
-    CLOSED_STATUS_BUILDER.setBegin(0);
-    CLOSED_STATUS_BUILDER.setEnd(0);
-    CLOSED_STATUS_BUILDER.setInfiniteEnd(true);
-    CLOSED_STATUS_BUILDER.setClosed(true);
-
-    builder = Status.newBuilder();
+    Builder builder = Status.newBuilder();
     builder.setBegin(0);
     builder.setEnd(0);
     builder.setInfiniteEnd(true);
     builder.setClosed(false);
     INF_END_REPLICATION_STATUS = builder.build();
     INF_END_REPLICATION_STATUS_VALUE = ProtobufUtil.toValue(INF_END_REPLICATION_STATUS);
-  }
 
-  /**
-   * @return A {@link Status} which represents a file with no data that is open for writes
-   */
-  public static Status newFile() {
-    return NEW_REPLICATION_STATUS;
-  }
-
-  /**
-   * @return A {@link Value} which represent a file with no data that is open for writes
-   */
-  public static Value newFileValue() {
-    return NEW_REPLICATION_STATUS_VALUE;
+    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);
   }
 
   /**
@@ -135,19 +121,33 @@ public class StatusUtil {
   }
 
   /**
-   * @return A {@link Status} for a closed file of unspecified length, all of which needs replicating.
+   * @return A {@link Status} for a new file that was just created
    */
-  public static synchronized Status fileClosed(long timeClosed) {
+  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)
-    CLOSED_STATUS_BUILDER.setClosedTime(timeClosed);
-    return CLOSED_STATUS_BUILDER.build();
+    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 {@link Value} for a closed file of unspecified length, all of which needs replicating.
+   * @return A Value representing a closed file
    */
-  public static Value fileClosedValue(long timeClosed) {
-    return ProtobufUtil.toValue(fileClosed(timeClosed));
+  public static Value fileClosedValue() {
+    return CLOSED_STATUS_VALUE;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b53bbf08/core/src/main/java/org/apache/accumulo/core/replication/proto/Replication.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/replication/proto/Replication.java b/core/src/main/java/org/apache/accumulo/core/replication/proto/Replication.java
index d301028..2bff020 100644
--- a/core/src/main/java/org/apache/accumulo/core/replication/proto/Replication.java
+++ b/core/src/main/java/org/apache/accumulo/core/replication/proto/Replication.java
@@ -99,23 +99,23 @@ package org.apache.accumulo.core.replication.proto;
      */
     boolean getClosed();
 
-    // optional int64 closedTime = 5 [default = 0];
+    // optional int64 createdTime = 5 [default = 0];
     /**
-     * <code>optional int64 closedTime = 5 [default = 0];</code>
+     * <code>optional int64 createdTime = 5 [default = 0];</code>
      *
      * <pre>
-     * when, in ms, was the file closed?
+     * when, in ms, was the file created?
      * </pre>
      */
-    boolean hasClosedTime();
+    boolean hasCreatedTime();
     /**
-     * <code>optional int64 closedTime = 5 [default = 0];</code>
+     * <code>optional int64 createdTime = 5 [default = 0];</code>
      *
      * <pre>
-     * when, in ms, was the file closed?
+     * when, in ms, was the file created?
      * </pre>
      */
-    long getClosedTime();
+    long getCreatedTime();
   }
   /**
    * Protobuf type {@code Status}
@@ -190,7 +190,7 @@ package org.apache.accumulo.core.replication.proto;
             }
             case 40: {
               bitField0_ |= 0x00000010;
-              closedTime_ = input.readInt64();
+              createdTime_ = input.readInt64();
               break;
             }
           }
@@ -329,28 +329,28 @@ package org.apache.accumulo.core.replication.proto;
       return closed_;
     }
 
-    // optional int64 closedTime = 5 [default = 0];
-    public static final int CLOSEDTIME_FIELD_NUMBER = 5;
-    private long closedTime_;
+    // optional int64 createdTime = 5 [default = 0];
+    public static final int CREATEDTIME_FIELD_NUMBER = 5;
+    private long createdTime_;
     /**
-     * <code>optional int64 closedTime = 5 [default = 0];</code>
+     * <code>optional int64 createdTime = 5 [default = 0];</code>
      *
      * <pre>
-     * when, in ms, was the file closed?
+     * when, in ms, was the file created?
      * </pre>
      */
-    public boolean hasClosedTime() {
+    public boolean hasCreatedTime() {
       return ((bitField0_ & 0x00000010) == 0x00000010);
     }
     /**
-     * <code>optional int64 closedTime = 5 [default = 0];</code>
+     * <code>optional int64 createdTime = 5 [default = 0];</code>
      *
      * <pre>
-     * when, in ms, was the file closed?
+     * when, in ms, was the file created?
      * </pre>
      */
-    public long getClosedTime() {
-      return closedTime_;
+    public long getCreatedTime() {
+      return createdTime_;
     }
 
     private void initFields() {
@@ -358,7 +358,7 @@ package org.apache.accumulo.core.replication.proto;
       end_ = 0L;
       infiniteEnd_ = false;
       closed_ = false;
-      closedTime_ = 0L;
+      createdTime_ = 0L;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -385,7 +385,7 @@ package org.apache.accumulo.core.replication.proto;
         output.writeBool(4, closed_);
       }
       if (((bitField0_ & 0x00000010) == 0x00000010)) {
-        output.writeInt64(5, closedTime_);
+        output.writeInt64(5, createdTime_);
       }
       getUnknownFields().writeTo(output);
     }
@@ -414,7 +414,7 @@ package org.apache.accumulo.core.replication.proto;
       }
       if (((bitField0_ & 0x00000010) == 0x00000010)) {
         size += com.google.protobuf.CodedOutputStream
-          .computeInt64Size(5, closedTime_);
+          .computeInt64Size(5, createdTime_);
       }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
@@ -540,7 +540,7 @@ package org.apache.accumulo.core.replication.proto;
         bitField0_ = (bitField0_ & ~0x00000004);
         closed_ = false;
         bitField0_ = (bitField0_ & ~0x00000008);
-        closedTime_ = 0L;
+        createdTime_ = 0L;
         bitField0_ = (bitField0_ & ~0x00000010);
         return this;
       }
@@ -589,7 +589,7 @@ package org.apache.accumulo.core.replication.proto;
         if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
           to_bitField0_ |= 0x00000010;
         }
-        result.closedTime_ = closedTime_;
+        result.createdTime_ = createdTime_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -618,8 +618,8 @@ package org.apache.accumulo.core.replication.proto;
         if (other.hasClosed()) {
           setClosed(other.getClosed());
         }
-        if (other.hasClosedTime()) {
-          setClosedTime(other.getClosedTime());
+        if (other.hasCreatedTime()) {
+          setCreatedTime(other.getCreatedTime());
         }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
@@ -844,51 +844,51 @@ package org.apache.accumulo.core.replication.proto;
         return this;
       }
 
-      // optional int64 closedTime = 5 [default = 0];
-      private long closedTime_ ;
+      // optional int64 createdTime = 5 [default = 0];
+      private long createdTime_ ;
       /**
-       * <code>optional int64 closedTime = 5 [default = 0];</code>
+       * <code>optional int64 createdTime = 5 [default = 0];</code>
        *
        * <pre>
-       * when, in ms, was the file closed?
+       * when, in ms, was the file created?
        * </pre>
        */
-      public boolean hasClosedTime() {
+      public boolean hasCreatedTime() {
         return ((bitField0_ & 0x00000010) == 0x00000010);
       }
       /**
-       * <code>optional int64 closedTime = 5 [default = 0];</code>
+       * <code>optional int64 createdTime = 5 [default = 0];</code>
        *
        * <pre>
-       * when, in ms, was the file closed?
+       * when, in ms, was the file created?
        * </pre>
        */
-      public long getClosedTime() {
-        return closedTime_;
+      public long getCreatedTime() {
+        return createdTime_;
       }
       /**
-       * <code>optional int64 closedTime = 5 [default = 0];</code>
+       * <code>optional int64 createdTime = 5 [default = 0];</code>
        *
        * <pre>
-       * when, in ms, was the file closed?
+       * when, in ms, was the file created?
        * </pre>
        */
-      public Builder setClosedTime(long value) {
+      public Builder setCreatedTime(long value) {
         bitField0_ |= 0x00000010;
-        closedTime_ = value;
+        createdTime_ = value;
         onChanged();
         return this;
       }
       /**
-       * <code>optional int64 closedTime = 5 [default = 0];</code>
+       * <code>optional int64 createdTime = 5 [default = 0];</code>
        *
        * <pre>
-       * when, in ms, was the file closed?
+       * when, in ms, was the file created?
        * </pre>
        */
-      public Builder clearClosedTime() {
+      public Builder clearCreatedTime() {
         bitField0_ = (bitField0_ & ~0x00000010);
-        closedTime_ = 0L;
+        createdTime_ = 0L;
         onChanged();
         return this;
       }
@@ -918,12 +918,12 @@ package org.apache.accumulo.core.replication.proto;
       descriptor;
   static {
     java.lang.String[] descriptorData = {
-      "\n#src/main/protobuf/replication.proto\"t\n" +
+      "\n#src/main/protobuf/replication.proto\"u\n" +
       "\006Status\022\020\n\005begin\030\001 \001(\003:\0010\022\016\n\003end\030\002 \001(\003:\001" +
       "0\022\032\n\013infiniteEnd\030\003 \001(\010:\005false\022\025\n\006closed\030" +
-      "\004 \001(\010:\005false\022\025\n\nclosedTime\030\005 \001(\003:\0010B.\n*o" +
-      "rg.apache.accumulo.core.replication.prot" +
-      "oH\001"
+      "\004 \001(\010:\005false\022\026\n\013createdTime\030\005 \001(\003:\0010B.\n*" +
+      "org.apache.accumulo.core.replication.pro" +
+      "toH\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -935,7 +935,7 @@ package org.apache.accumulo.core.replication.proto;
           internal_static_Status_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_Status_descriptor,
-              new java.lang.String[] { "Begin", "End", "InfiniteEnd", "Closed", "ClosedTime", });
+              new java.lang.String[] { "Begin", "End", "InfiniteEnd", "Closed", "CreatedTime", });
           return null;
         }
       };

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b53bbf08/core/src/main/protobuf/replication.proto
----------------------------------------------------------------------
diff --git a/core/src/main/protobuf/replication.proto b/core/src/main/protobuf/replication.proto
index be801b0..7feda58 100644
--- a/core/src/main/protobuf/replication.proto
+++ b/core/src/main/protobuf/replication.proto
@@ -22,5 +22,5 @@ message Status {
 	optional int64 end = 2 [default = 0]; // offset where data is ready for replication
 	optional bool infiniteEnd = 3 [default = false]; // do we have a discrete 'end'
 	optional bool closed = 4 [default = false]; // will more data be appended to the file
-	optional int64 closedTime = 5 [default = 0]; // when, in ms, was the file closed?
+	optional int64 createdTime = 5 [default = 0]; // when, in ms, was the file created?
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b53bbf08/core/src/test/java/org/apache/accumulo/core/replication/StatusUtilTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/replication/StatusUtilTest.java b/core/src/test/java/org/apache/accumulo/core/replication/StatusUtilTest.java
index 14609f3..5a35d65 100644
--- a/core/src/test/java/org/apache/accumulo/core/replication/StatusUtilTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/replication/StatusUtilTest.java
@@ -27,7 +27,7 @@ public class StatusUtilTest {
 
   @Test
   public void newFileIsNotCompletelyReplicated() {
-    Assert.assertFalse(StatusUtil.isSafeForRemoval(StatusUtil.newFile()));
+    Assert.assertFalse(StatusUtil.isSafeForRemoval(StatusUtil.fileCreated(0l)));
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b53bbf08/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 f9d43f1..0472fa9 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
@@ -233,7 +233,7 @@ public class VolumeUtil {
       Credentials creds = SystemCredentials.get();
       MetadataTableUtil.updateTabletVolumes(extent, logsToRemove, logsToAdd, filesToRemove, filesToAdd, switchedDir, zooLock, creds);
       if (replicate) {
-        Status status = StatusUtil.fileClosed(System.currentTimeMillis());
+        Status status = StatusUtil.fileClosed();
         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
         ReplicationTableUtil.updateLogs(creds, extent, logsToRemove, status);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b53bbf08/server/base/src/main/java/org/apache/accumulo/server/replication/StatusCombiner.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/replication/StatusCombiner.java b/server/base/src/main/java/org/apache/accumulo/server/replication/StatusCombiner.java
index 694664e..ecca99e 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/replication/StatusCombiner.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/replication/StatusCombiner.java
@@ -26,6 +26,7 @@ import org.apache.accumulo.core.iterators.IteratorEnvironment;
 import org.apache.accumulo.core.iterators.SortedKeyValueIterator;
 import org.apache.accumulo.core.iterators.TypedValueCombiner;
 import org.apache.accumulo.core.iterators.ValueFormatException;
+import org.apache.accumulo.core.protobuf.ProtobufUtil;
 import org.apache.accumulo.core.replication.proto.Replication.Status;
 import org.apache.accumulo.core.replication.proto.Replication.Status.Builder;
 import org.apache.log4j.Logger;
@@ -91,7 +92,7 @@ public class StatusCombiner extends TypedValueCombiner<Status> {
       if (null == combined) {
         if (!iter.hasNext()) {
           if (log.isTraceEnabled()) {
-            log.trace("Returned single value: " + key.toStringNoTruncate() + " " + status.toString().replace("\n", ", "));
+            log.trace("Returned single value: " + key.toStringNoTruncate() + " " + ProtobufUtil.toString(status));
           }
           return status;
         } else {
@@ -104,7 +105,7 @@ public class StatusCombiner extends TypedValueCombiner<Status> {
     }
 
     if (log.isTraceEnabled()) {
-      log.trace("Combined: " + key.toStringNoTruncate() + " " + combined.build().toString().replace("\n", ", "));
+      log.trace("Combined: " + key.toStringNoTruncate() + " " + ProtobufUtil.toString(combined.build()));
     }
 
     return combined.build();
@@ -135,18 +136,19 @@ public class StatusCombiner extends TypedValueCombiner<Status> {
     // persist the infinite end
     combined.setInfiniteEnd(combined.getInfiniteEnd() | status.getInfiniteEnd());
 
-    // only set the closedTime if the new status has it defined
-    if (status.hasClosedTime()) {
-      // choose the minimum (earliest) closedTime seen
-      if (combined.hasClosedTime()) {
-        combined.setClosedTime(Math.min(combined.getClosedTime(), status.getClosedTime()));
+    // only set the createdTime if the new status has it defined
+    if (status.hasCreatedTime()) {
+      // choose the minimum (earliest) createdTime seen
+      if (combined.hasCreatedTime()) {
+        combined.setCreatedTime(Math.min(combined.getCreatedTime(), status.getCreatedTime()));
       } else {
-        combined.setClosedTime(status.getClosedTime());
+        combined.setCreatedTime(status.getCreatedTime());
       }
     }
   }
 
   private String builderToString(Builder builder) {
-    return "begin: " + builder.getBegin() + ", end: " + builder.getEnd() + ", infiniteEnd: " + builder.getInfiniteEnd() + ", closed: "+ builder.getClosed(); 
+    return "begin: " + builder.getBegin() + ", end: " + builder.getEnd() + ", infiniteEnd: " + builder.getInfiniteEnd() + ", closed: " + builder.getClosed()
+        + ", createdTime: " + builder.getCreatedTime();
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b53bbf08/server/base/src/test/java/org/apache/accumulo/server/replication/StatusCombinerTest.java
----------------------------------------------------------------------
diff --git a/server/base/src/test/java/org/apache/accumulo/server/replication/StatusCombinerTest.java b/server/base/src/test/java/org/apache/accumulo/server/replication/StatusCombinerTest.java
index d74e2c6..5bc2488 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/replication/StatusCombinerTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/replication/StatusCombinerTest.java
@@ -86,27 +86,29 @@ public class StatusCombinerTest {
 
   @Test
   public void newStatusWithNewIngest() {
-    Status orig = StatusUtil.newFile();
+    Status orig = StatusUtil.fileCreated(100);
     Status status = StatusUtil.replicatedAndIngested(10, 20);
     Status ret = combiner.typedReduce(key, Arrays.asList(orig, status).iterator());
     Assert.assertEquals(10l, ret.getBegin());
     Assert.assertEquals(20l, ret.getEnd());
+    Assert.assertEquals(100l, ret.getCreatedTime());
     Assert.assertEquals(false, ret.getClosed());
   }
 
   @Test
   public void newStatusWithNewIngestSingleBuilder() {
-    Status orig = StatusUtil.newFile();
+    Status orig = StatusUtil.fileCreated(100);
     Status status = StatusUtil.replicatedAndIngested(builder, 10, 20);
     Status ret = combiner.typedReduce(key, Arrays.asList(orig, status).iterator());
     Assert.assertEquals(10l, ret.getBegin());
     Assert.assertEquals(20l, ret.getEnd());
+    Assert.assertEquals(100l, ret.getCreatedTime());
     Assert.assertEquals(false, ret.getClosed());
   }
 
   @Test
   public void commutativeNewFile() {
-    Status newFile = StatusUtil.newFile(), firstSync = StatusUtil.ingestedUntil(100), secondSync = StatusUtil.ingestedUntil(200);
+    Status newFile = StatusUtil.fileCreated(100), firstSync = StatusUtil.ingestedUntil(100), secondSync = StatusUtil.ingestedUntil(200);
 
     Status order1 = combiner.typedReduce(key, Arrays.asList(newFile, firstSync, secondSync).iterator()), order2 = combiner.typedReduce(key,
         Arrays.asList(secondSync, firstSync, newFile).iterator());
@@ -116,7 +118,7 @@ public class StatusCombinerTest {
 
   @Test
   public void commutativeNewFileSingleBuilder() {
-    Status newFile = StatusUtil.newFile(), firstSync = StatusUtil.ingestedUntil(builder, 100), secondSync = StatusUtil.ingestedUntil(builder, 200);
+    Status newFile = StatusUtil.fileCreated(100), firstSync = StatusUtil.ingestedUntil(builder, 100), secondSync = StatusUtil.ingestedUntil(builder, 200);
 
     Status order1 = combiner.typedReduce(key, Arrays.asList(newFile, firstSync, secondSync).iterator()), order2 = combiner.typedReduce(key,
         Arrays.asList(secondSync, firstSync, newFile).iterator());
@@ -126,7 +128,7 @@ public class StatusCombinerTest {
 
   @Test
   public void commutativeNewUpdates() {
-    Status newFile = StatusUtil.newFile(), firstSync = StatusUtil.ingestedUntil(100), secondSync = StatusUtil.ingestedUntil(200);
+    Status newFile = StatusUtil.fileCreated(100), firstSync = StatusUtil.ingestedUntil(100), secondSync = StatusUtil.ingestedUntil(200);
 
     Status order1 = combiner.typedReduce(key, Arrays.asList(newFile, firstSync, secondSync).iterator()), order2 = combiner.typedReduce(key,
         Arrays.asList(newFile, secondSync, firstSync).iterator());
@@ -136,7 +138,7 @@ public class StatusCombinerTest {
 
   @Test
   public void commutativeNewUpdatesSingleBuilder() {
-    Status newFile = StatusUtil.newFile(), firstSync = StatusUtil.ingestedUntil(builder, 100), secondSync = StatusUtil.ingestedUntil(builder, 200);
+    Status newFile = StatusUtil.fileCreated(100), firstSync = StatusUtil.ingestedUntil(builder, 100), secondSync = StatusUtil.ingestedUntil(builder, 200);
 
     Status order1 = combiner.typedReduce(key, Arrays.asList(newFile, firstSync, secondSync).iterator()), order2 = combiner.typedReduce(key,
         Arrays.asList(newFile, secondSync, firstSync).iterator());
@@ -146,7 +148,7 @@ public class StatusCombinerTest {
 
   @Test
   public void commutativeWithClose() {
-    Status newFile = StatusUtil.newFile(), closed = StatusUtil.fileClosed(System.currentTimeMillis()), secondSync = StatusUtil.ingestedUntil(200);
+    Status newFile = StatusUtil.fileCreated(100), closed = StatusUtil.fileClosed(), secondSync = StatusUtil.ingestedUntil(200);
 
     Status order1 = combiner.typedReduce(key, Arrays.asList(newFile, closed, secondSync).iterator()), order2 = combiner.typedReduce(key,
         Arrays.asList(newFile, secondSync, closed).iterator());
@@ -156,7 +158,7 @@ public class StatusCombinerTest {
 
   @Test
   public void commutativeWithCloseSingleBuilder() {
-    Status newFile = StatusUtil.newFile(), closed = StatusUtil.fileClosed(System.currentTimeMillis()), secondSync = StatusUtil.ingestedUntil(builder, 200);
+    Status newFile = StatusUtil.fileCreated(100), closed = StatusUtil.fileClosed(), secondSync = StatusUtil.ingestedUntil(builder, 200);
 
     Status order1 = combiner.typedReduce(key, Arrays.asList(newFile, closed, secondSync).iterator()), order2 = combiner.typedReduce(key,
         Arrays.asList(newFile, secondSync, closed).iterator());
@@ -166,7 +168,7 @@ public class StatusCombinerTest {
 
   @Test
   public void commutativeWithMultipleUpdates() {
-    Status newFile = StatusUtil.newFile(), update1 = StatusUtil.ingestedUntil(100), update2 = StatusUtil.ingestedUntil(200), repl1 = StatusUtil.replicated(50), repl2 = StatusUtil
+    Status newFile = StatusUtil.fileCreated(100), update1 = StatusUtil.ingestedUntil(100), update2 = StatusUtil.ingestedUntil(200), repl1 = StatusUtil.replicated(50), repl2 = StatusUtil
         .replicated(150);
 
     Status order1 = combiner.typedReduce(key, Arrays.asList(newFile, update1, repl1, update2, repl2).iterator());
@@ -189,7 +191,7 @@ public class StatusCombinerTest {
 
   @Test
   public void commutativeWithMultipleUpdatesSingleBuilder() {
-    Status newFile = StatusUtil.newFile(), update1 = StatusUtil.ingestedUntil(builder, 100), update2 = StatusUtil.ingestedUntil(builder, 200), repl1 = StatusUtil
+    Status newFile = StatusUtil.fileCreated(100), update1 = StatusUtil.ingestedUntil(builder, 100), update2 = StatusUtil.ingestedUntil(builder, 200), repl1 = StatusUtil
         .replicated(builder, 50), repl2 = StatusUtil.replicated(builder, 150);
 
     Status order1 = combiner.typedReduce(key, Arrays.asList(newFile, update1, repl1, update2, repl2).iterator());
@@ -212,7 +214,7 @@ public class StatusCombinerTest {
 
   @Test
   public void duplicateStatuses() {
-    Status newFile = StatusUtil.newFile(), update1 = StatusUtil.ingestedUntil(builder, 100), update2 = StatusUtil.ingestedUntil(builder, 200), repl1 = StatusUtil
+    Status newFile = StatusUtil.fileCreated(100), update1 = StatusUtil.ingestedUntil(builder, 100), update2 = StatusUtil.ingestedUntil(builder, 200), repl1 = StatusUtil
         .replicated(builder, 50), repl2 = StatusUtil.replicated(builder, 150);
 
     Status order1 = combiner.typedReduce(key, Arrays.asList(newFile, update1, repl1, update2, repl2).iterator());
@@ -225,7 +227,7 @@ public class StatusCombinerTest {
 
   @Test
   public void fileClosedTimePropagated() {
-    Status stat1 = Status.newBuilder().setBegin(10).setEnd(20).setClosed(true).setInfiniteEnd(false).setClosedTime(50).build();
+    Status stat1 = Status.newBuilder().setBegin(10).setEnd(20).setClosed(true).setInfiniteEnd(false).setCreatedTime(50).build();
     Status stat2 = Status.newBuilder().setBegin(10).setEnd(20).setClosed(true).setInfiniteEnd(false).build();
 
     Status combined = combiner.typedReduce(key, Arrays.asList(stat1, stat2).iterator());
@@ -235,14 +237,14 @@ public class StatusCombinerTest {
 
   @Test
   public void fileClosedTimeChoosesEarliestIgnoringDefault() {
-    Status stat1 = Status.newBuilder().setBegin(10).setEnd(20).setClosed(true).setInfiniteEnd(false).setClosedTime(50).build();
-    Status stat2 = Status.newBuilder().setBegin(10).setEnd(20).setClosed(true).setInfiniteEnd(false).setClosedTime(100).build();
+    Status stat1 = Status.newBuilder().setBegin(10).setEnd(20).setClosed(true).setInfiniteEnd(false).setCreatedTime(50).build();
+    Status stat2 = Status.newBuilder().setBegin(10).setEnd(20).setClosed(true).setInfiniteEnd(false).setCreatedTime(100).build();
 
     Status combined = combiner.typedReduce(key, Arrays.asList(stat1, stat2).iterator());
 
     Assert.assertEquals(stat1, combined);
 
-    Status stat3 = Status.newBuilder().setBegin(10).setEnd(20).setClosed(true).setInfiniteEnd(false).setClosedTime(100).build();
+    Status stat3 = Status.newBuilder().setBegin(10).setEnd(20).setClosed(true).setInfiniteEnd(false).setCreatedTime(100).build();
 
     Status combined2 = combiner.typedReduce(key, Arrays.asList(combined, stat3).iterator());
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b53bbf08/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 7aa53b6..3e3332f 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
@@ -86,7 +86,8 @@ public class ReplicationTableUtilTest {
     UUID uuid = UUID.randomUUID();
     String myFile = "file:////home/user/accumulo/wal/server+port/" + uuid;
 
-    ReplicationTableUtil.updateFiles(creds, new KeyExtent(new Text("1"), null, null), Collections.singleton(myFile), StatusUtil.newFile());
+    long createdTime = System.currentTimeMillis();
+    ReplicationTableUtil.updateFiles(creds, new KeyExtent(new Text("1"), null, null), Collections.singleton(myFile), StatusUtil.fileCreated(createdTime));
 
     verify(writer);
 
@@ -101,13 +102,13 @@ public class ReplicationTableUtilTest {
 
     Assert.assertEquals(MetadataSchema.ReplicationSection.COLF, new Text(update.getColumnFamily()));
     Assert.assertEquals("1", new Text(update.getColumnQualifier()).toString());
-    Assert.assertEquals(StatusUtil.newFileValue(), new Value(update.getValue()));
+    Assert.assertEquals(StatusUtil.fileCreatedValue(createdTime), new Value(update.getValue()));
   }
 
   @Test
   public void replEntryMutation() {
     // We stopped using a WAL -- we need a reference that this WAL needs to be replicated completely
-    Status stat = StatusUtil.fileClosed(System.currentTimeMillis());
+    Status stat = Status.newBuilder().setBegin(0).setEnd(0).setInfiniteEnd(true).setCreatedTime(System.currentTimeMillis()).build();
     String file = "file:///accumulo/wal/127.0.0.1+9997" + UUID.randomUUID();
     Path filePath = new Path(file);
     Text row = new Text(filePath.toString());

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b53bbf08/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 294883e..d649c3e 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
@@ -250,7 +250,7 @@ public class CloseWriteAheadLogReferences implements Runnable {
   protected void closeWal(BatchWriter bw, Key k) throws MutationsRejectedException {
     log.debug("Closing unreferenced WAL ({}) in metadata table", k.toStringNoTruncate());
     Mutation m = new Mutation(k.getRow());
-    m.put(k.getColumnFamily(), k.getColumnQualifier(), StatusUtil.fileClosedValue(System.currentTimeMillis()));
+    m.put(k.getColumnFamily(), k.getColumnQualifier(), StatusUtil.fileClosedValue());
     bw.addMutation(m);
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b53bbf08/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
index 2faa8a2..ab877fc 100644
--- a/server/gc/src/test/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogsTest.java
+++ b/server/gc/src/test/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogsTest.java
@@ -324,7 +324,7 @@ public class GarbageCollectWriteAheadLogsTest {
 
     // 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.newFileValue()));
+    replData.add(Maps.immutableEntry(new Key("/wals/" + file1, StatusSection.NAME.toString(), "1"), StatusUtil.fileCreatedValue(System.currentTimeMillis())));
 
     ReplicationGCWAL replGC = new ReplicationGCWAL(instance, volMgr, false, replData);
 
@@ -338,7 +338,7 @@ public class GarbageCollectWriteAheadLogsTest {
     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(System.currentTimeMillis())));
+    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
@@ -360,12 +360,14 @@ public class GarbageCollectWriteAheadLogsTest {
 
     ReplicationTable.create(conn);
 
+    long file1CreateTime = System.currentTimeMillis();
+    long file2CreateTime = file1CreateTime + 50;
     BatchWriter bw = conn.createBatchWriter(ReplicationTable.NAME, new BatchWriterConfig());
     Mutation m = new Mutation("/wals/" + file1);
-    StatusSection.add(m, new Text("1"), StatusUtil.newFileValue());
+    StatusSection.add(m, new Text("1"), StatusUtil.fileCreatedValue(file1CreateTime));
     bw.addMutation(m);
     m = new Mutation("/wals/" + file2);
-    StatusSection.add(m, new Text("1"), StatusUtil.newFileValue());
+    StatusSection.add(m, new Text("1"), StatusUtil.fileCreatedValue(file2CreateTime));
     bw.addMutation(m);
 
     // These WALs are potential candidates for deletion from fs
@@ -402,14 +404,16 @@ public class GarbageCollectWriteAheadLogsTest {
 
     ReplicationTable.create(conn);
 
+    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.newFileValue());
+    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.newFileValue());
+    m.put(ReplicationSection.COLF, new Text("1"), StatusUtil.fileCreatedValue(file2CreateTime));
     bw.addMutation(m);
 
     // These WALs are potential candidates for deletion from fs
@@ -442,7 +446,7 @@ public class GarbageCollectWriteAheadLogsTest {
     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.newFileValue());
+    m.put(ReplicationSection.COLF, new Text("1"), StatusUtil.fileCreatedValue(System.currentTimeMillis()));
     bw.addMutation(m);
     bw.close();
 
@@ -460,16 +464,17 @@ public class GarbageCollectWriteAheadLogsTest {
     Connector conn = inst.getConnector("root", new PasswordToken(""));
     ReplicationTable.create(conn);
 
+    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.newFileValue());
+    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.newFileValue());
+    StatusSection.add(m, new Text("1"), StatusUtil.fileCreatedValue(walCreateTime));
     bw.addMutation(m);
     bw.close();
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b53bbf08/server/gc/src/test/java/org/apache/accumulo/gc/GarbageCollectionTest.java
----------------------------------------------------------------------
diff --git a/server/gc/src/test/java/org/apache/accumulo/gc/GarbageCollectionTest.java b/server/gc/src/test/java/org/apache/accumulo/gc/GarbageCollectionTest.java
index 4b6b4fc..51b9596 100644
--- a/server/gc/src/test/java/org/apache/accumulo/gc/GarbageCollectionTest.java
+++ b/server/gc/src/test/java/org/apache/accumulo/gc/GarbageCollectionTest.java
@@ -605,7 +605,7 @@ public class GarbageCollectionTest {
     gce.candidates.add("hdfs://foo.com:6000/accumulo/tables/2/t-00002/A000002.rf");
 
     // We replicated all of the data, but we might still write more data to the file
-    Status status = StatusUtil.newFile();
+    Status status = StatusUtil.fileCreated(System.currentTimeMillis());
     gce.filesToReplicate.put("hdfs://foo.com:6000/accumulo/tables/1/t-00001/A000001.rf", status);
 
     gca.collect(gce);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b53bbf08/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 67dcc31..66152c4 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
@@ -312,7 +312,7 @@ public class CloseWriteAheadLogReferencesTest {
     ReplicationTable.create(conn);
     BatchWriter bw = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
     Mutation m = new Mutation(ReplicationSection.getRowPrefix() + "file:/accumulo/wal/tserver+port/12345");
-    m.put(ReplicationSection.COLF, new Text("1"), StatusUtil.newFileValue());
+    m.put(ReplicationSection.COLF, new Text("1"), StatusUtil.fileCreatedValue(System.currentTimeMillis()));
     bw.addMutation(m);
     bw.close();
 
@@ -334,7 +334,7 @@ public class CloseWriteAheadLogReferencesTest {
     ReplicationTable.create(conn);
     BatchWriter bw = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
     Mutation m = new Mutation(ReplicationSection.getRowPrefix() + file);
-    m.put(ReplicationSection.COLF, new Text("1"), StatusUtil.newFileValue());
+    m.put(ReplicationSection.COLF, new Text("1"), StatusUtil.fileCreatedValue(System.currentTimeMillis()));
     bw.addMutation(m);
     bw.close();
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b53bbf08/server/master/src/main/java/org/apache/accumulo/master/replication/RemoveCompleteReplicationRecords.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/replication/RemoveCompleteReplicationRecords.java b/server/master/src/main/java/org/apache/accumulo/master/replication/RemoveCompleteReplicationRecords.java
index dd5ccd8..f7246d7 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/replication/RemoveCompleteReplicationRecords.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/replication/RemoveCompleteReplicationRecords.java
@@ -18,7 +18,6 @@ package org.apache.accumulo.master.replication;
 
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
@@ -147,7 +146,7 @@ public class RemoveCompleteReplicationRecords implements Runnable {
     }
 
     Mutation m = new Mutation(row);
-    Map<String,Long> tableToTimeClosed = new HashMap<>();
+    Map<String,Long> tableToTimeCreated = new HashMap<>();
     for (Entry<Key,Value> entry : columns.entrySet()) {
       Status status = null;
       try {
@@ -178,12 +177,12 @@ public class RemoveCompleteReplicationRecords implements Runnable {
         throw new RuntimeException("Got unexpected column");
       }
 
-      if (status.hasClosedTime()) {
-        Long timeClosed = tableToTimeClosed.get(tableId);
+      if (status.hasCreatedTime()) {
+        Long timeClosed = tableToTimeCreated.get(tableId);
         if (null == timeClosed) {
-          tableToTimeClosed.put(tableId, status.getClosedTime());
-        } else if (timeClosed != status.getClosedTime()){
-          log.warn("Found multiple values for timeClosed for {}: {} and {}", row, timeClosed, status.getClosedTime());
+          tableToTimeCreated.put(tableId, status.getCreatedTime());
+        } else if (timeClosed != status.getCreatedTime()){
+          log.warn("Found multiple values for timeClosed for {}: {} and {}", row, timeClosed, status.getCreatedTime());
         }
       }
 
@@ -194,7 +193,7 @@ public class RemoveCompleteReplicationRecords implements Runnable {
 
     List<Mutation> mutations = new ArrayList<>();
     mutations.add(m);
-    for (Entry<String,Long> entry : tableToTimeClosed.entrySet()) {
+    for (Entry<String,Long> entry : tableToTimeCreated.entrySet()) {
       log.info("Removing order mutation for table {} at {} for {}", entry.getKey(), entry.getValue(), row.toString());
       Mutation orderMutation = OrderSection.createMutation(row.toString(), entry.getValue());
       orderMutation.putDelete(OrderSection.NAME, new Text(entry.getKey()));

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b53bbf08/server/master/src/main/java/org/apache/accumulo/master/replication/StatusMaker.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/replication/StatusMaker.java b/server/master/src/main/java/org/apache/accumulo/master/replication/StatusMaker.java
index 6bc5962..c054de2 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/replication/StatusMaker.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/replication/StatusMaker.java
@@ -193,13 +193,14 @@ public class StatusMaker {
    */
   protected boolean addOrderRecord(Text file, Text tableId, Status stat, Value value) {
     try {
-      if (!stat.hasClosedTime()) {
-        log.warn("Status record ({}) for {} in table {} was written to metadata table which was closed but lacked closedTime", ProtobufUtil.toString(stat), file, tableId);
+      if (!stat.hasCreatedTime()) {
+        log.error("Status record ({}) for {} in table {} was written to metadata table which lacked createdTime", ProtobufUtil.toString(stat), file, tableId);
+        return false;
       }
 
       log.info("Creating order record for {} for {} with {}", file, tableId, ProtobufUtil.toString(stat));
 
-      Mutation m = OrderSection.createMutation(file.toString(), stat.getClosedTime());
+      Mutation m = OrderSection.createMutation(file.toString(), stat.getCreatedTime());
       OrderSection.add(m, tableId, value);
 
       try {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b53bbf08/server/master/src/test/java/org/apache/accumulo/master/replication/DistributedWorkQueueWorkAssignerTest.java
----------------------------------------------------------------------
diff --git a/server/master/src/test/java/org/apache/accumulo/master/replication/DistributedWorkQueueWorkAssignerTest.java b/server/master/src/test/java/org/apache/accumulo/master/replication/DistributedWorkQueueWorkAssignerTest.java
index 46c5691..ddd4810 100644
--- a/server/master/src/test/java/org/apache/accumulo/master/replication/DistributedWorkQueueWorkAssignerTest.java
+++ b/server/master/src/test/java/org/apache/accumulo/master/replication/DistributedWorkQueueWorkAssignerTest.java
@@ -211,11 +211,11 @@ public class DistributedWorkQueueWorkAssignerTest {
     String file1 = "/accumulo/wal/tserver+port/" + filename1, file2 = "/accumulo/wal/tserver+port/" + filename2;
 
     Mutation m = new Mutation(file1);
-    WorkSection.add(m, serializedTarget1, StatusUtil.newFileValue());
+    WorkSection.add(m, serializedTarget1, StatusUtil.fileCreatedValue(5));
     bw.addMutation(m);
 
     m = new Mutation(file2);
-    WorkSection.add(m, serializedTarget2, StatusUtil.newFileValue());
+    WorkSection.add(m, serializedTarget2, StatusUtil.fileCreatedValue(10));
     bw.addMutation(m);
 
     bw.close();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b53bbf08/server/master/src/test/java/org/apache/accumulo/master/replication/RemoveCompleteReplicationRecordsTest.java
----------------------------------------------------------------------
diff --git a/server/master/src/test/java/org/apache/accumulo/master/replication/RemoveCompleteReplicationRecordsTest.java b/server/master/src/test/java/org/apache/accumulo/master/replication/RemoveCompleteReplicationRecordsTest.java
index 373062c..1cd30f8 100644
--- a/server/master/src/test/java/org/apache/accumulo/master/replication/RemoveCompleteReplicationRecordsTest.java
+++ b/server/master/src/test/java/org/apache/accumulo/master/replication/RemoveCompleteReplicationRecordsTest.java
@@ -203,7 +203,7 @@ public class RemoveCompleteReplicationRecordsTest {
     long time = System.currentTimeMillis();
     // Write out numRecords entries to both replication and metadata tables, none of which are fully replicated
     for (int i = 0; i < numRecords; i++) {
-      builder.setClosedTime(time++);
+      builder.setCreatedTime(time++);
       String file = "/accumulo/wal/tserver+port/" + UUID.randomUUID();
       Mutation m = new Mutation(file);
       Value v = ProtobufUtil.toValue(builder.setBegin(1000*(i+1)).build()); 
@@ -224,7 +224,7 @@ public class RemoveCompleteReplicationRecordsTest {
     filesToRemove.add(fileToRemove);
     Mutation m = new Mutation(fileToRemove);
     ReplicationTarget target = new ReplicationTarget("peer1", "5", "5");
-    Value value = ProtobufUtil.toValue(builder.setBegin(10000).setEnd(10000).setClosed(true).setClosedTime(time).build());
+    Value value = ProtobufUtil.toValue(builder.setBegin(10000).setEnd(10000).setClosed(true).setCreatedTime(time).build());
     StatusSection.add(m, new Text("5"), value);
     WorkSection.add(m, target.toText(), value);
     replBw.addMutation(m);
@@ -239,7 +239,7 @@ public class RemoveCompleteReplicationRecordsTest {
     fileToRemove = "/accumulo/wal/tserver+port/" + UUID.randomUUID();
     filesToRemove.add(fileToRemove);
     m = new Mutation(fileToRemove);
-    value = ProtobufUtil.toValue(builder.setBegin(10000).setEnd(10000).setClosed(true).setClosedTime(time).build());
+    value = ProtobufUtil.toValue(builder.setBegin(10000).setEnd(10000).setClosed(true).setCreatedTime(time).build());
     target = new ReplicationTarget("peer1", "6", "6");
     StatusSection.add(m, new Text("6"), value);
     WorkSection.add(m, target.toText(), value);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b53bbf08/server/master/src/test/java/org/apache/accumulo/master/replication/SequentialWorkAssignerTest.java
----------------------------------------------------------------------
diff --git a/server/master/src/test/java/org/apache/accumulo/master/replication/SequentialWorkAssignerTest.java b/server/master/src/test/java/org/apache/accumulo/master/replication/SequentialWorkAssignerTest.java
index 0820e1c..dce0aa6 100644
--- a/server/master/src/test/java/org/apache/accumulo/master/replication/SequentialWorkAssignerTest.java
+++ b/server/master/src/test/java/org/apache/accumulo/master/replication/SequentialWorkAssignerTest.java
@@ -95,8 +95,8 @@ public class SequentialWorkAssignerTest {
     String file1 = "/accumulo/wal/tserver+port/" + filename1, file2 = "/accumulo/wal/tserver+port/" + filename2;
 
     // File1 was closed before file2, however
-    Status stat1 = Status.newBuilder().setBegin(0).setEnd(100).setClosed(true).setInfiniteEnd(false).setClosedTime(250).build();
-    Status stat2 = Status.newBuilder().setBegin(0).setEnd(100).setClosed(true).setInfiniteEnd(false).setClosedTime(500).build();
+    Status stat1 = Status.newBuilder().setBegin(0).setEnd(100).setClosed(true).setInfiniteEnd(false).setCreatedTime(250).build();
+    Status stat2 = Status.newBuilder().setBegin(0).setEnd(100).setClosed(true).setInfiniteEnd(false).setCreatedTime(500).build();
 
     Mutation m = new Mutation(file1);
     WorkSection.add(m, serializedTarget, ProtobufUtil.toValue(stat1));
@@ -106,11 +106,11 @@ public class SequentialWorkAssignerTest {
     WorkSection.add(m, serializedTarget, ProtobufUtil.toValue(stat2));
     bw.addMutation(m);
 
-    m = OrderSection.createMutation(file1, stat1.getClosedTime());
+    m = OrderSection.createMutation(file1, stat1.getCreatedTime());
     OrderSection.add(m, new Text(target.getSourceTableId()), ProtobufUtil.toValue(stat1));
     bw.addMutation(m);
 
-    m = OrderSection.createMutation(file2, stat2.getClosedTime());
+    m = OrderSection.createMutation(file2, stat2.getCreatedTime());
     OrderSection.add(m, new Text(target.getSourceTableId()), ProtobufUtil.toValue(stat2));
     bw.addMutation(m);
 
@@ -168,8 +168,8 @@ public class SequentialWorkAssignerTest {
     String file1 = "/accumulo/wal/tserver+port/" + filename1, file2 = "/accumulo/wal/tserver+port/" + filename2;
 
     // File1 was closed before file2, however
-    Status stat1 = Status.newBuilder().setBegin(0).setEnd(100).setClosed(true).setInfiniteEnd(false).setClosedTime(250).build();
-    Status stat2 = Status.newBuilder().setBegin(0).setEnd(100).setClosed(true).setInfiniteEnd(false).setClosedTime(500).build();
+    Status stat1 = Status.newBuilder().setBegin(0).setEnd(100).setClosed(true).setInfiniteEnd(false).setCreatedTime(250).build();
+    Status stat2 = Status.newBuilder().setBegin(0).setEnd(100).setClosed(true).setInfiniteEnd(false).setCreatedTime(500).build();
 
     Mutation m = new Mutation(file1);
     WorkSection.add(m, serializedTarget1, ProtobufUtil.toValue(stat1));
@@ -179,11 +179,11 @@ public class SequentialWorkAssignerTest {
     WorkSection.add(m, serializedTarget2, ProtobufUtil.toValue(stat2));
     bw.addMutation(m);
 
-    m = OrderSection.createMutation(file1, stat1.getClosedTime());
+    m = OrderSection.createMutation(file1, stat1.getCreatedTime());
     OrderSection.add(m, new Text(target1.getSourceTableId()), ProtobufUtil.toValue(stat1));
     bw.addMutation(m);
 
-    m = OrderSection.createMutation(file2, stat2.getClosedTime());
+    m = OrderSection.createMutation(file2, stat2.getCreatedTime());
     OrderSection.add(m, new Text(target2.getSourceTableId()), ProtobufUtil.toValue(stat2));
     bw.addMutation(m);
 
@@ -248,8 +248,8 @@ public class SequentialWorkAssignerTest {
     String file1 = "/accumulo/wal/tserver+port/" + filename1, file2 = "/accumulo/wal/tserver+port/" + filename2;
 
     // File1 was closed before file2, however
-    Status stat1 = Status.newBuilder().setBegin(0).setEnd(100).setClosed(true).setInfiniteEnd(false).setClosedTime(250).build();
-    Status stat2 = Status.newBuilder().setBegin(0).setEnd(100).setClosed(true).setInfiniteEnd(false).setClosedTime(500).build();
+    Status stat1 = Status.newBuilder().setBegin(0).setEnd(100).setClosed(true).setInfiniteEnd(false).setCreatedTime(250).build();
+    Status stat2 = Status.newBuilder().setBegin(0).setEnd(100).setClosed(true).setInfiniteEnd(false).setCreatedTime(500).build();
 
     Mutation m = new Mutation(file1);
     WorkSection.add(m, serializedTarget1, ProtobufUtil.toValue(stat1));
@@ -259,11 +259,11 @@ public class SequentialWorkAssignerTest {
     WorkSection.add(m, serializedTarget2, ProtobufUtil.toValue(stat2));
     bw.addMutation(m);
 
-    m = OrderSection.createMutation(file1, stat1.getClosedTime());
+    m = OrderSection.createMutation(file1, stat1.getCreatedTime());
     OrderSection.add(m, new Text(target1.getSourceTableId()), ProtobufUtil.toValue(stat1));
     bw.addMutation(m);
 
-    m = OrderSection.createMutation(file2, stat2.getClosedTime());
+    m = OrderSection.createMutation(file2, stat2.getCreatedTime());
     OrderSection.add(m, new Text(target2.getSourceTableId()), ProtobufUtil.toValue(stat2));
     bw.addMutation(m);
 
@@ -369,8 +369,8 @@ public class SequentialWorkAssignerTest {
     String file1 = "/accumulo/wal/tserver+port/" + filename1, file2 = "/accumulo/wal/tserver+port/" + filename2;
 
     // File1 was closed before file2, however
-    Status stat1 = Status.newBuilder().setBegin(100).setEnd(100).setClosed(true).setInfiniteEnd(false).setClosedTime(250).build();
-    Status stat2 = Status.newBuilder().setBegin(0).setEnd(100).setClosed(true).setInfiniteEnd(false).setClosedTime(500).build();
+    Status stat1 = Status.newBuilder().setBegin(100).setEnd(100).setClosed(true).setInfiniteEnd(false).setCreatedTime(250).build();
+    Status stat2 = Status.newBuilder().setBegin(0).setEnd(100).setClosed(true).setInfiniteEnd(false).setCreatedTime(500).build();
 
     Mutation m = new Mutation(file1);
     WorkSection.add(m, serializedTarget, ProtobufUtil.toValue(stat1));
@@ -380,11 +380,11 @@ public class SequentialWorkAssignerTest {
     WorkSection.add(m, serializedTarget, ProtobufUtil.toValue(stat2));
     bw.addMutation(m);
 
-    m = OrderSection.createMutation(file1, stat1.getClosedTime());
+    m = OrderSection.createMutation(file1, stat1.getCreatedTime());
     OrderSection.add(m, new Text(target.getSourceTableId()), ProtobufUtil.toValue(stat1));
     bw.addMutation(m);
 
-    m = OrderSection.createMutation(file2, stat2.getClosedTime());
+    m = OrderSection.createMutation(file2, stat2.getCreatedTime());
     OrderSection.add(m, new Text(target.getSourceTableId()), ProtobufUtil.toValue(stat2));
     bw.addMutation(m);
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b53bbf08/server/master/src/test/java/org/apache/accumulo/master/replication/StatusMakerTest.java
----------------------------------------------------------------------
diff --git a/server/master/src/test/java/org/apache/accumulo/master/replication/StatusMakerTest.java b/server/master/src/test/java/org/apache/accumulo/master/replication/StatusMakerTest.java
index e0fc421..4c0ab2b 100644
--- a/server/master/src/test/java/org/apache/accumulo/master/replication/StatusMakerTest.java
+++ b/server/master/src/test/java/org/apache/accumulo/master/replication/StatusMakerTest.java
@@ -78,12 +78,16 @@ public class StatusMakerTest {
     Map<String,Integer> fileToTableId = new HashMap<>();
 
     int index = 1;
+    long timeCreated = 0;
+    Map<String,Long> fileToTimeCreated = new HashMap<>();
     for (String file : files) {
       Mutation m = new Mutation(ReplicationSection.getRowPrefix() + file);
-      m.put(ReplicationSection.COLF, new Text(Integer.toString(index)), StatusUtil.newFileValue());
+      m.put(ReplicationSection.COLF, new Text(Integer.toString(index)), StatusUtil.fileCreatedValue(timeCreated));
+      fileToTimeCreated.put(file, timeCreated);
       bw.addMutation(m);
       fileToTableId.put(file, index);
       index++;
+      timeCreated++;
     }
 
     bw.close();
@@ -102,7 +106,9 @@ public class StatusMakerTest {
 
       Assert.assertTrue("Found unexpected file: " + file, files.contains(file.toString()));
       Assert.assertEquals(fileToTableId.get(file.toString()), new Integer(tableId.toString()));
-      Assert.assertEquals(StatusUtil.newFile(), Status.parseFrom(entry.getValue().get()));
+      timeCreated = fileToTimeCreated.get(file.toString());
+      Assert.assertNotNull(timeCreated);
+      Assert.assertEquals(StatusUtil.fileCreated(timeCreated), Status.parseFrom(entry.getValue().get()));
     }
   }
 
@@ -123,12 +129,14 @@ public class StatusMakerTest {
     Map<String,Integer> fileToTableId = new HashMap<>();
 
     int index = 1;
+    long timeCreated = 0;
     for (String file : files) {
       Mutation m = new Mutation(ReplicationSection.getRowPrefix() + file);
-      m.put(ReplicationSection.COLF, new Text(Integer.toString(index)), StatusUtil.newFileValue());
+      m.put(ReplicationSection.COLF, new Text(Integer.toString(index)), StatusUtil.fileCreatedValue(timeCreated));
       bw.addMutation(m);
       fileToTableId.put(file, index);
       index++;
+      timeCreated++;
     }
 
     bw.close();
@@ -160,7 +168,7 @@ public class StatusMakerTest {
         walPrefix + UUID.randomUUID());
     Map<String,Integer> fileToTableId = new HashMap<>();
 
-    Status stat = Status.newBuilder().setBegin(0).setEnd(0).setInfiniteEnd(true).setClosed(true).build();
+    Status stat = Status.newBuilder().setBegin(0).setEnd(0).setInfiniteEnd(true).setClosed(true).setCreatedTime(System.currentTimeMillis()).build();
 
     int index = 1;
     for (String file : files) {
@@ -212,7 +220,7 @@ public class StatusMakerTest {
     int index = 1;
     long time = System.currentTimeMillis();
     for (String file : files) {
-      statBuilder.setClosedTime(time++);
+      statBuilder.setCreatedTime(time++);
       Mutation m = new Mutation(ReplicationSection.getRowPrefix() + file);
       m.put(ReplicationSection.COLF, new Text(Integer.toString(index)), ProtobufUtil.toValue(statBuilder.build()));
       bw.addMutation(m);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b53bbf08/server/master/src/test/java/org/apache/accumulo/master/replication/WorkMakerTest.java
----------------------------------------------------------------------
diff --git a/server/master/src/test/java/org/apache/accumulo/master/replication/WorkMakerTest.java b/server/master/src/test/java/org/apache/accumulo/master/replication/WorkMakerTest.java
index 337aa12..0c4afc9 100644
--- a/server/master/src/test/java/org/apache/accumulo/master/replication/WorkMakerTest.java
+++ b/server/master/src/test/java/org/apache/accumulo/master/replication/WorkMakerTest.java
@@ -84,9 +84,9 @@ public class WorkMakerTest {
     String file = "hdfs://localhost:8020/accumulo/wal/123456-1234-1234-12345678";
 
     // Create a status record for a file
-    long timeClosed = System.currentTimeMillis();
+    long timeCreated = System.currentTimeMillis();
     Mutation m = new Mutation(new Path(file).toString());
-    m.put(StatusSection.NAME, new Text(tableId), StatusUtil.fileClosedValue(timeClosed));
+    m.put(StatusSection.NAME, new Text(tableId), StatusUtil.fileCreatedValue(timeCreated));
     BatchWriter bw = ReplicationTable.getBatchWriter(conn);
     bw.addMutation(m);
     bw.flush();
@@ -101,7 +101,7 @@ public class WorkMakerTest {
     // Invoke the addWorkRecord method to create a Work record from the Status record earlier
     ReplicationTarget expected = new ReplicationTarget("remote_cluster_1", "4", tableId);
     workMaker.setBatchWriter(bw);
-    workMaker.addWorkRecord(new Text(file), StatusUtil.fileClosedValue(timeClosed), ImmutableMap.of("remote_cluster_1", "4"), tableId);
+    workMaker.addWorkRecord(new Text(file), StatusUtil.fileCreatedValue(timeCreated), ImmutableMap.of("remote_cluster_1", "4"), tableId);
 
     // Scan over just the WorkSection
     s = ReplicationTable.getScanner(conn);
@@ -114,7 +114,7 @@ public class WorkMakerTest {
     Assert.assertEquals(file, workKey.getRow().toString());
     Assert.assertEquals(WorkSection.NAME, workKey.getColumnFamily());
     Assert.assertEquals(expected, actual);
-    Assert.assertEquals(workEntry.getValue(), StatusUtil.fileClosedValue(timeClosed));
+    Assert.assertEquals(workEntry.getValue(), StatusUtil.fileCreatedValue(timeCreated));
   }
 
   @Test
@@ -127,7 +127,7 @@ public class WorkMakerTest {
     String file = "hdfs://localhost:8020/accumulo/wal/123456-1234-1234-12345678";
 
     Mutation m = new Mutation(new Path(file).toString());
-    m.put(StatusSection.NAME, new Text(tableId), StatusUtil.fileClosedValue(System.currentTimeMillis()));
+    m.put(StatusSection.NAME, new Text(tableId), StatusUtil.fileCreatedValue(System.currentTimeMillis()));
     BatchWriter bw = ReplicationTable.getBatchWriter(conn);
     bw.addMutation(m);
     bw.flush();
@@ -145,7 +145,7 @@ public class WorkMakerTest {
       expectedTargets.add(new ReplicationTarget(cluster.getKey(), cluster.getValue(), tableId));
     }
     workMaker.setBatchWriter(bw);
-    workMaker.addWorkRecord(new Text(file), StatusUtil.fileClosedValue(System.currentTimeMillis()), targetClusters, tableId);
+    workMaker.addWorkRecord(new Text(file), StatusUtil.fileCreatedValue(System.currentTimeMillis()), targetClusters, tableId);
 
     s = ReplicationTable.getScanner(conn);
     WorkSection.limit(s);
@@ -175,7 +175,7 @@ public class WorkMakerTest {
     String file = "hdfs://localhost:8020/accumulo/wal/123456-1234-1234-12345678";
 
     Mutation m = new Mutation(new Path(file).toString());
-    m.put(StatusSection.NAME, new Text(tableId), StatusUtil.newFileValue());
+    m.put(StatusSection.NAME, new Text(tableId), StatusUtil.fileCreatedValue(System.currentTimeMillis()));
     BatchWriter bw = ReplicationTable.getBatchWriter(conn);
     bw.addMutation(m);
     bw.flush();
@@ -205,7 +205,7 @@ public class WorkMakerTest {
   public void closedStatusRecordsStillMakeWork() throws Exception {
     WorkMaker workMaker = new WorkMaker(conn);
 
-    Assert.assertFalse(workMaker.shouldCreateWork(StatusUtil.newFile()));
+    Assert.assertFalse(workMaker.shouldCreateWork(StatusUtil.fileCreated(System.currentTimeMillis())));
     Assert.assertTrue(workMaker.shouldCreateWork(StatusUtil.ingestedUntil(1000)));
     Assert.assertTrue(workMaker.shouldCreateWork(Status.newBuilder().setBegin(Long.MAX_VALUE).setEnd(0).setInfiniteEnd(true).setClosed(true).build()));
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b53bbf08/server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java
index 418f679..73e99e2 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java
@@ -1408,7 +1408,7 @@ public class Tablet {
 
           // Ensure that we write a record marking each WAL as requiring replication to make sure we don't abandon the data
           if (ReplicationConfigurationUtil.isEnabled(extent, tabletServer.getTableConfiguration(extent))) {
-            Status status = StatusUtil.fileClosed(System.currentTimeMillis());
+            Status status = StatusUtil.fileClosed();
             for (LogEntry logEntry : logEntries) {
               log.debug("Writing closed status to metadata table for " + logEntry.logSet + " " + ProtobufUtil.toString(status));
               ReplicationTableUtil.updateFiles(SystemCredentials.get(), extent, logEntry.logSet, status);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b53bbf08/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 d8c4279..b7b0aff 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
@@ -37,6 +37,7 @@ 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;
 import org.apache.accumulo.server.fs.VolumeManager;
@@ -274,9 +275,10 @@ public class TabletServerLogger {
                 for (DfsLogger logger : copy) {
                   logs.add(logger.getFileName());
                 }
-                log.debug("Writing " + ProtobufUtil.toString(StatusUtil.newFile()) + " to replication table for " + logs);
+                Status status = StatusUtil.fileCreated(System.currentTimeMillis());
+                log.debug("Writing " + ProtobufUtil.toString(status) + " to replication table for " + logs);
                 // Got some new WALs, note this in the replication table
-                ReplicationTableUtil.updateFiles(SystemCredentials.get(), commitSession.getExtent(), logs, StatusUtil.newFile());
+                ReplicationTableUtil.updateFiles(SystemCredentials.get(), commitSession.getExtent(), logs, status);
               }
             }
           }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b53bbf08/test/src/test/java/org/apache/accumulo/test/replication/ReplicationWithGCIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/replication/ReplicationWithGCIT.java b/test/src/test/java/org/apache/accumulo/test/replication/ReplicationWithGCIT.java
index 2a7a210..9047533 100644
--- a/test/src/test/java/org/apache/accumulo/test/replication/ReplicationWithGCIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/replication/ReplicationWithGCIT.java
@@ -227,7 +227,8 @@ public class ReplicationWithGCIT extends ConfigurableMacIT {
 
       // Fake that each one is fully replicated
       Mutation m = new Mutation(entry.getKey().getRow());
-      m.put(entry.getKey().getColumnFamily().toString(), entry.getKey().getColumnQualifier().toString(), StatusUtil.newFileValue());
+      m.put(entry.getKey().getColumnFamily().toString(), entry.getKey().getColumnQualifier().toString(),
+          StatusUtil.fileCreatedValue(System.currentTimeMillis()));
       bw.addMutation(m);
     }
     bw.close();
@@ -287,7 +288,7 @@ public class ReplicationWithGCIT extends ConfigurableMacIT {
         }
         Assert.fail("Expected all replication records in the metadata table to be closed");
       }
-      
+
       for (int i = 0; i < 10; i++) {
         allClosed = true;
 
@@ -328,7 +329,6 @@ public class ReplicationWithGCIT extends ConfigurableMacIT {
       gc.waitFor();
     }
 
-
   }
 
   @Test(timeout = 5 * 60 * 1000)
@@ -345,197 +345,210 @@ public class ReplicationWithGCIT extends ConfigurableMacIT {
     // replication shouldn't exist when we begin
     Assert.assertFalse(conn.tableOperations().exists(ReplicationTable.NAME));
 
-    // Create two tables
-    conn.tableOperations().create(table1);
+    ReplicationTablesPrinterThread thread = new ReplicationTablesPrinterThread(conn, System.out);
+    thread.start();
 
-    int attempts = 5;
-    while (attempts > 0) {
-      try {
-        // Enable replication on table1
-        conn.tableOperations().setProperty(table1, Property.TABLE_REPLICATION.getKey(), "true");
-        // Replicate table1 to cluster1 in the table with id of '4'
-        conn.tableOperations().setProperty(table1, Property.TABLE_REPLICATION_TARGETS.getKey() + "cluster1", "4");
-        // Use the MockReplicaSystem impl and sleep for 5seconds
-        conn.instanceOperations().setProperty(Property.REPLICATION_PEERS.getKey() + "cluster1",
-            ReplicaSystemFactory.getPeerConfigurationValue(MockReplicaSystem.class, "5000"));
-        attempts = 0;
-      } catch (Exception e) {
-        attempts--;
-        if (attempts <= 0) {
-          throw e;
+    try {
+      // Create two tables
+      conn.tableOperations().create(table1);
+
+      int attempts = 5;
+      while (attempts > 0) {
+        try {
+          // Enable replication on table1
+          conn.tableOperations().setProperty(table1, Property.TABLE_REPLICATION.getKey(), "true");
+          // Replicate table1 to cluster1 in the table with id of '4'
+          conn.tableOperations().setProperty(table1, Property.TABLE_REPLICATION_TARGETS.getKey() + "cluster1", "4");
+          // Use the MockReplicaSystem impl and sleep for 5seconds
+          conn.instanceOperations().setProperty(Property.REPLICATION_PEERS.getKey() + "cluster1",
+              ReplicaSystemFactory.getPeerConfigurationValue(MockReplicaSystem.class, "5000"));
+          attempts = 0;
+        } catch (Exception e) {
+          attempts--;
+          if (attempts <= 0) {
+            throw e;
+          }
+          UtilWaitThread.sleep(500);
         }
-        UtilWaitThread.sleep(500);
       }
-    }
 
-    String tableId = conn.tableOperations().tableIdMap().get(table1);
-    Assert.assertNotNull("Could not determine table id for " + table1, tableId);
+      String tableId = conn.tableOperations().tableIdMap().get(table1);
+      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);
+      // 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.addMutation(m);
-    }
 
-    bw.close();
+      bw.close();
 
-    // Make sure the replication table exists at this point
-    boolean exists = conn.tableOperations().exists(ReplicationTable.NAME);
-    attempts = 10;
-    do {
-      if (!exists) {
-        UtilWaitThread.sleep(1000);
-        exists = conn.tableOperations().exists(ReplicationTable.NAME);
-        attempts--;
-      }
-    } while (!exists && attempts > 0);
-    Assert.assertTrue("Replication table did not exist", exists);
+      // Make sure the replication table exists at this point
+      boolean exists = conn.tableOperations().exists(ReplicationTable.NAME);
+      attempts = 10;
+      do {
+        if (!exists) {
+          UtilWaitThread.sleep(1000);
+          exists = conn.tableOperations().exists(ReplicationTable.NAME);
+          attempts--;
+        }
+      } while (!exists && attempts > 0);
+      Assert.assertTrue("Replication table did not exist", exists);
+
+      // Grant ourselves the write permission for later
+      conn.securityOperations().grantTablePermission("root", ReplicationTable.NAME, TablePermission.WRITE);
+
+      // 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);
+          Text expectedColqual = new ReplicationTarget("cluster1", "4", tableId).toText();
+          Assert.assertEquals(expectedColqual, e.getKey().getColumnQualifier());
+          notFound = false;
+        } catch (NoSuchElementException e) {
+
+        } catch (IllegalArgumentException e) {
+          // Somehow we got more than one element. Log what they were
+          s = ReplicationTable.getScanner(conn);
+          for (Entry<Key,Value> content : s) {
+            log.info(content.getKey().toStringNoTruncate() + " => " + content.getValue());
+          }
+          Assert.fail("Found more than one work section entry");
+        } catch (RuntimeException e) {
+          // Catch a propagation issue, fail if it's not what we expect
+          Throwable cause = e.getCause();
+          if (cause instanceof AccumuloSecurityException) {
+            AccumuloSecurityException sec = (AccumuloSecurityException) cause;
+            switch (sec.getSecurityErrorCode()) {
+              case PERMISSION_DENIED:
+                // retry -- the grant didn't happen yet
+                log.warn("Sleeping because permission was denied");
+              default:
+                throw e;
+            }
+          } else {
+            throw e;
+          }
+        }
 
-    // Grant ourselves the write permission for later
-    conn.securityOperations().grantTablePermission("root", ReplicationTable.NAME, TablePermission.WRITE);
+        Thread.sleep(1000);
+      }
 
-    // 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);
-        Text expectedColqual = new ReplicationTarget("cluster1", "4", tableId).toText();
-        Assert.assertEquals(expectedColqual, e.getKey().getColumnQualifier());
-        notFound = false;
-      } catch (NoSuchElementException e) {
-
-      } catch (IllegalArgumentException e) {
-        // Somehow we got more than one element. Log what they were
+      if (notFound) {
         s = ReplicationTable.getScanner(conn);
         for (Entry<Key,Value> content : s) {
           log.info(content.getKey().toStringNoTruncate() + " => " + content.getValue());
         }
-        Assert.fail("Found more than one work section entry");
-      } catch (RuntimeException e) {
-        // Catch a propagation issue, fail if it's not what we expect
-        Throwable cause = e.getCause();
-        if (cause instanceof AccumuloSecurityException) {
-          AccumuloSecurityException sec = (AccumuloSecurityException) cause;
-          switch (sec.getSecurityErrorCode()) {
-            case PERMISSION_DENIED:
-              // retry -- the grant didn't happen yet
-              log.warn("Sleeping because permission was denied");
-            default:
-              throw e;
-          }
-        } else {
-          throw e;
-        }
+        Assert.assertFalse("Did not find the work entry for the status entry", notFound);
       }
 
-      Thread.sleep(1000);
-    }
-
-    if (notFound) {
-      s = ReplicationTable.getScanner(conn);
-      for (Entry<Key,Value> content : s) {
-        log.info(content.getKey().toStringNoTruncate() + " => " + content.getValue());
+      /**
+       * By this point, we should have data ingested into a table, with at least one WAL as a candidate for replication. Compacting the table should close all
+       * open WALs, which should ensure all records we're going to replicate have entries in the replication table, and nothing will exist in the metadata table
+       * anymore
+       */
+
+      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);
       }
-      Assert.assertFalse("Did not find the work entry for the status entry", notFound);
-    }
-
-    /**
-     * By this point, we should have data ingested into a table, with at least one WAL as a candidate for replication. Compacting the table should close all
-     * open WALs, which should ensure all records we're going to replicate have entries in the replication table, and nothing will exist in the metadata table
-     * anymore
-     */
-
-    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);
-    }
-
-    log.info("Starting tserver");
-    cluster.exec(TabletServer.class);
 
-    log.info("Waiting to read tables");
+      log.info("Starting tserver");
+      cluster.exec(TabletServer.class);
 
-    // Make sure we can read all the tables (recovery complete)
-    for (String table : new String[] {MetadataTable.NAME, table1}) {
-      s = conn.createScanner(table, new Authorizations());
-      for (@SuppressWarnings("unused")
-      Entry<Key,Value> entry : s) {}
-    }
+      log.info("Waiting to read tables");
 
-    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());
+      // Make sure we can read all the tables (recovery complete)
+      for (String table : new String[] {MetadataTable.NAME, table1}) {
+        s = conn.createScanner(table, new Authorizations());
+        for (@SuppressWarnings("unused")
+        Entry<Key,Value> entry : s) {}
       }
-      if (count > 0) {
-        foundResults = true;
-        break;
-      }
-      Thread.sleep(1000);
-    }
 
-    Assert.assertTrue("Did not find any replication entries in the replication table", foundResults);
-
-    getCluster().exec(SimpleGarbageCollector.class);
-
-    // Wait for a bit since the GC has to run (should be running after a one second delay)
-    Thread.sleep(5000);
-
-    // 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;
-    for (int i = 0; i < 5; i++) {
-      s = conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
-      s.setRange(ReplicationSection.getRange());
-      if (Iterables.size(s) == 0) {
-        foundResults = false;
-        break;
+      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);
       }
-      Thread.sleep(1000);
-    }
 
-    Assert.assertFalse("Replication status messages were not cleaned up from metadata table", foundResults);
+      Assert.assertTrue("Did not find any replication entries in the replication table", foundResults);
 
-    /**
-     * After we close out and subsequently delete the metadata record, this will propagate to the replication table,
-     * which will cause those records to be deleted after repliation occurs
-     */
+      getCluster().exec(SimpleGarbageCollector.class);
 
-    int recordsFound = 0;
-    for (int i = 0; i < 10; i++) {
-      s = ReplicationTable.getScanner(conn);
-      recordsFound = 0;
-      for (Entry<Key,Value> entry : s) {
-        recordsFound++;
-        log.info(entry.getKey().toStringNoTruncate() + " " + Status.parseFrom(entry.getValue().get()).toString().replace("\n", ", "));
-      }
+      // Wait for a bit since the GC has to run (should be running after a one second delay)
+      Thread.sleep(5000);
 
-      if (0 == recordsFound) {
-        break;
-      } else {
+      // 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;
+      for (int i = 0; i < 5; i++) {
+        s = conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
+        s.setRange(ReplicationSection.getRange());
+        long size = 0;
+        for (Entry<Key,Value> e : s) {
+          size++;
+          log.info("{}={}", e.getKey().toStringNoTruncate(), ProtobufUtil.toString(Status.parseFrom(e.getValue().get())));
+        }
+        if (size == 0) {
+          foundResults = false;
+          break;
+        }
         Thread.sleep(1000);
         log.info("");
       }
-    }
 
-    Assert.assertEquals("Found unexpected replication records in the replication table", 0, recordsFound);
+      Assert.assertFalse("Replication status messages were not cleaned up from metadata table", foundResults);
+
+      /**
+       * After we close out and subsequently delete the metadata record, this will propagate to the replication table, which will cause those records to be
+       * deleted after repliation occurs
+       */
+
+      int recordsFound = 0;
+      for (int i = 0; i < 10; i++) {
+        s = ReplicationTable.getScanner(conn);
+        recordsFound = 0;
+        for (Entry<Key,Value> entry : s) {
+          recordsFound++;
+          log.info(entry.getKey().toStringNoTruncate() + " " + Status.parseFrom(entry.getValue().get()).toString().replace("\n", ", "));
+        }
+
+        if (0 == recordsFound) {
+          break;
+        } else {
+          Thread.sleep(1000);
+          log.info("");
+        }
+      }
+
+      Assert.assertEquals("Found unexpected replication records in the replication table", 0, recordsFound);
+    } finally {
+      thread.join(200);
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b53bbf08/test/src/test/java/org/apache/accumulo/test/replication/ReplicationWithMakerTest.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/replication/ReplicationWithMakerTest.java b/test/src/test/java/org/apache/accumulo/test/replication/ReplicationWithMakerTest.java
index 03ac72c..70d6ca1 100644
--- a/test/src/test/java/org/apache/accumulo/test/replication/ReplicationWithMakerTest.java
+++ b/test/src/test/java/org/apache/accumulo/test/replication/ReplicationWithMakerTest.java
@@ -23,10 +23,12 @@ 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.client.replication.ReplicaSystemFactory;
 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.Value;
+import org.apache.accumulo.core.protobuf.ProtobufUtil;
 import org.apache.accumulo.core.replication.ReplicationSchema.StatusSection;
 import org.apache.accumulo.core.replication.ReplicationSchema.WorkSection;
 import org.apache.accumulo.core.replication.ReplicationTarget;
@@ -91,6 +93,8 @@ public class ReplicationWithMakerTest extends ConfigurableMacIT {
         conn.tableOperations().setProperty(table1, Property.TABLE_REPLICATION.getKey(), "true");
         // Replicate table1 to cluster1 in the table with id of '4'
         conn.tableOperations().setProperty(table1, Property.TABLE_REPLICATION_TARGETS.getKey() + "cluster1", "4");
+        conn.instanceOperations().setProperty(Property.REPLICATION_PEERS.getKey() + "cluster1",
+            ReplicaSystemFactory.getPeerConfigurationValue(MockReplicaSystem.class, "100000"));
         break;
       } catch (Exception e) {
         attempts--;
@@ -136,7 +140,7 @@ public class ReplicationWithMakerTest extends ConfigurableMacIT {
 
     // Trigger the minor compaction, waiting for it to finish.
     // This should write the entry to metadata that the file has data
-    conn.tableOperations().flush(ReplicationTable.NAME, null, null, true);
+    conn.tableOperations().flush(table1, null, null, true);
 
     // Make sure that we have one status element, should be a new file
     Scanner s = ReplicationTable.getScanner(conn);
@@ -148,7 +152,8 @@ public class ReplicationWithMakerTest extends ConfigurableMacIT {
     while (null == entry && attempts > 0) {
       try {
         entry = Iterables.getOnlyElement(s);
-        if (!expectedStatus.equals(Status.parseFrom(entry.getValue().get()))) {
+        Status actual = Status.parseFrom(entry.getValue().get());
+        if (actual.getInfiniteEnd() != expectedStatus.getInfiniteEnd()) {
           entry = null;
           // the master process didn't yet fire and write the new mutation, wait for it to do
           // so and try to read it again
@@ -171,7 +176,8 @@ public class ReplicationWithMakerTest extends ConfigurableMacIT {
     }
 
     Assert.assertNotNull("Could not find expected entry in replication table", entry);
-    Assert.assertEquals("Expected to find a replication entry that is open with infinite length", expectedStatus, Status.parseFrom(entry.getValue().get()));
+    Status actual = Status.parseFrom(entry.getValue().get());
+    Assert.assertTrue("Expected to find a replication entry that is open with infinite length: " + ProtobufUtil.toString(actual), !actual.getClosed() && actual.getInfiniteEnd());
 
     // Try a couple of times to watch for the work record to be created
     boolean notFound = true;


[50/50] [abbrv] git commit: ACCUMULO-378 Remove service port configuration in favor of random

Posted by el...@apache.org.
ACCUMULO-378 Remove service port configuration in favor of random


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

Branch: refs/heads/ACCUMULO-378
Commit: 417b0b3328f7ad81bcde36a00e15091822f32b22
Parents: b53bbf0
Author: Josh Elser <el...@apache.org>
Authored: Tue May 20 21:55:13 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Tue May 20 21:55:13 2014 -0400

----------------------------------------------------------------------
 .../org/apache/accumulo/server/util/MasterMetadataUtil.java    | 1 -
 .../apache/accumulo/test/replication/CyclicReplicationIT.java  | 2 --
 .../org/apache/accumulo/test/replication/ReplicationIT.java    | 4 ----
 .../test/replication/ReplicationPortAdvertisementIT.java       | 2 --
 .../accumulo/test/replication/ReplicationSequentialIT.java     | 6 ------
 5 files changed, 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/417b0b33/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 05e1a0b..cecfceb 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
@@ -309,7 +309,6 @@ public class MasterMetadataUtil {
     }
     if (unusedWalLogs != null) {
       for (String entry : unusedWalLogs) {
-        log.info("Removed WAL " + entry + " from " + extent, new Exception());
         m.putDelete(LogColumnFamily.NAME, new Text(entry));
       }
     }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/417b0b33/test/src/test/java/org/apache/accumulo/test/replication/CyclicReplicationIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/replication/CyclicReplicationIT.java b/test/src/test/java/org/apache/accumulo/test/replication/CyclicReplicationIT.java
index b30dc39..032e519 100644
--- a/test/src/test/java/org/apache/accumulo/test/replication/CyclicReplicationIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/replication/CyclicReplicationIT.java
@@ -105,8 +105,6 @@ public class CyclicReplicationIT {
     master2Cfg.setInstanceName("master2");
     master2Cfg.setProperty(Property.REPLICATION_NAME, master2Cfg.getInstanceName());
     master2Cfg.setProperty(Property.TSERV_WALOG_MAX_SIZE, "5M");
-    master2Cfg.setProperty(Property.MASTER_REPLICATION_COORDINATOR_PORT, "10003");
-    master2Cfg.setProperty(Property.REPLICATION_RECEIPT_SERVICE_PORT, "10004");
     master2Cfg.setProperty(Property.REPLICATION_THREADCHECK, "5m");
     master2Cfg.setProperty(Property.REPLICATION_WORK_ASSIGNMENT_SLEEP, "1s");
     master2Cfg.setProperty(Property.MASTER_REPLICATION_SCAN_INTERVAL, "1s");

http://git-wip-us.apache.org/repos/asf/accumulo/blob/417b0b33/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 91f3581..77dcceb 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
@@ -74,8 +74,6 @@ public class ReplicationIT extends ConfigurableMacIT {
     peerCfg.setNumTservers(1);
     peerCfg.setInstanceName("peer");
     peerCfg.setProperty(Property.TSERV_WALOG_MAX_SIZE, "5M");
-    peerCfg.setProperty(Property.MASTER_REPLICATION_COORDINATOR_PORT, "10003");
-    peerCfg.setProperty(Property.REPLICATION_RECEIPT_SERVICE_PORT, "10004");
     peerCfg.setProperty(Property.REPLICATION_WORK_ASSIGNMENT_SLEEP, "1s");
     peerCfg.setProperty(Property.MASTER_REPLICATION_SCAN_INTERVAL, "1s");
     peerCfg.setProperty(Property.REPLICATION_NAME, "peer");
@@ -162,8 +160,6 @@ public class ReplicationIT extends ConfigurableMacIT {
     peerCfg.setNumTservers(1);
     peerCfg.setInstanceName("peer");
     peerCfg.setProperty(Property.TSERV_WALOG_MAX_SIZE, "5M");
-    peerCfg.setProperty(Property.MASTER_REPLICATION_COORDINATOR_PORT, "10003");
-    peerCfg.setProperty(Property.REPLICATION_RECEIPT_SERVICE_PORT, "10004");
     peerCfg.setProperty(Property.REPLICATION_WORK_ASSIGNMENT_SLEEP, "1s");
     peerCfg.setProperty(Property.MASTER_REPLICATION_SCAN_INTERVAL, "1s");
     peerCfg.setProperty(Property.REPLICATION_NAME, "peer");

http://git-wip-us.apache.org/repos/asf/accumulo/blob/417b0b33/test/src/test/java/org/apache/accumulo/test/replication/ReplicationPortAdvertisementIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/replication/ReplicationPortAdvertisementIT.java b/test/src/test/java/org/apache/accumulo/test/replication/ReplicationPortAdvertisementIT.java
index a13ca70..4254154 100644
--- a/test/src/test/java/org/apache/accumulo/test/replication/ReplicationPortAdvertisementIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/replication/ReplicationPortAdvertisementIT.java
@@ -48,8 +48,6 @@ public class ReplicationPortAdvertisementIT extends ConfigurableMacIT {
   @Override
   public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
     cfg.setNumTservers(2);
-    cfg.setProperty(Property.REPLICATION_RECEIPT_SERVICE_PORT, "0");
-    cfg.setProperty(Property.MASTER_REPLICATION_COORDINATOR_PORT, "0");
   }
   
   @Test

http://git-wip-us.apache.org/repos/asf/accumulo/blob/417b0b33/test/src/test/java/org/apache/accumulo/test/replication/ReplicationSequentialIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/replication/ReplicationSequentialIT.java b/test/src/test/java/org/apache/accumulo/test/replication/ReplicationSequentialIT.java
index 296175b..2420ac6 100644
--- a/test/src/test/java/org/apache/accumulo/test/replication/ReplicationSequentialIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/replication/ReplicationSequentialIT.java
@@ -84,8 +84,6 @@ public class ReplicationSequentialIT extends ConfigurableMacIT {
     cfg.setProperty(Property.GC_CYCLE_DELAY, "5s");
     cfg.setProperty(Property.REPLICATION_WORK_ASSIGNMENT_SLEEP, "1s");
     cfg.setProperty(Property.MASTER_REPLICATION_SCAN_INTERVAL, "1s");
-    cfg.setProperty(Property.MASTER_REPLICATION_COORDINATOR_PORT, "0");
-    cfg.setProperty(Property.REPLICATION_RECEIPT_SERVICE_PORT, "0");
     cfg.setProperty(Property.REPLICATION_MAX_UNIT_SIZE, "8M");
     cfg.setProperty(Property.REPLICATION_NAME, "master");
     cfg.setProperty(Property.REPLICATION_WORK_ASSIGNER, SequentialWorkAssigner.class.getName());
@@ -99,8 +97,6 @@ public class ReplicationSequentialIT extends ConfigurableMacIT {
     peerCfg.setNumTservers(1);
     peerCfg.setInstanceName("peer");
     peerCfg.setProperty(Property.TSERV_WALOG_MAX_SIZE, "5M");
-    peerCfg.setProperty(Property.MASTER_REPLICATION_COORDINATOR_PORT, "0");
-    peerCfg.setProperty(Property.REPLICATION_RECEIPT_SERVICE_PORT, "0");
     peerCfg.setProperty(Property.REPLICATION_WORK_ASSIGNMENT_SLEEP, "1s");
     peerCfg.setProperty(Property.MASTER_REPLICATION_SCAN_INTERVAL, "1s");
     peerCfg.setProperty(Property.REPLICATION_NAME, "peer");
@@ -251,8 +247,6 @@ public class ReplicationSequentialIT extends ConfigurableMacIT {
     peerCfg.setNumTservers(1);
     peerCfg.setInstanceName("peer");
     peerCfg.setProperty(Property.TSERV_WALOG_MAX_SIZE, "5M");
-    peerCfg.setProperty(Property.MASTER_REPLICATION_COORDINATOR_PORT, "0");
-    peerCfg.setProperty(Property.REPLICATION_RECEIPT_SERVICE_PORT, "0");
     peerCfg.setProperty(Property.REPLICATION_WORK_ASSIGNMENT_SLEEP, "1s");
     peerCfg.setProperty(Property.MASTER_REPLICATION_SCAN_INTERVAL, "1s");
     peerCfg.setProperty(Property.REPLICATION_NAME, "peer");


[08/50] [abbrv] git commit: ACCUMULO-2819 Make sure that order records are deleted with the status and work records.

Posted by el...@apache.org.
ACCUMULO-2819 Make sure that order records are deleted with the status and work records.

Tests for both the removal and the (de)serialization of the Order records.


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

Branch: refs/heads/ACCUMULO-378
Commit: f312cf14dbdbce4dcd4673635f0254c78bb0fcfb
Parents: 005b59f
Author: Josh Elser <el...@apache.org>
Authored: Thu May 15 22:06:58 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Thu May 15 22:06:58 2014 -0400

----------------------------------------------------------------------
 .../core/replication/ReplicationSchema.java     | 11 ++----
 .../core/replication/ReplicationSchemaTest.java | 21 ++++++++++
 .../RemoveCompleteReplicationRecords.java       | 27 ++++++++++---
 .../master/replication/StatusMaker.java         |  3 +-
 .../RemoveCompleteReplicationRecordsTest.java   | 41 +++++++++++++++-----
 5 files changed, 79 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/f312cf14/core/src/main/java/org/apache/accumulo/core/replication/ReplicationSchema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/replication/ReplicationSchema.java b/core/src/main/java/org/apache/accumulo/core/replication/ReplicationSchema.java
index 725758e..51bd7db 100644
--- a/core/src/main/java/org/apache/accumulo/core/replication/ReplicationSchema.java
+++ b/core/src/main/java/org/apache/accumulo/core/replication/ReplicationSchema.java
@@ -178,18 +178,14 @@ public class ReplicationSchema {
     }
 
     /**
-     * Creates the Mutation for the Order section for the given file and time, adding the column
-     * as well using {@link OrderSection#add(Mutation, Text, Value)}
+     * Creates the Mutation for the Order section for the given file and time
      * @param file Filename
      * @param timeInMillis Time in millis that the file was closed
-     * @param tableId Source table id
-     * @param v Serialized Status msg as a Value
      * @return Mutation for the Order section
      */
-    public static Mutation createMutation(String file, long timeInMillis, Text tableId, Value v) {
+    public static Mutation createMutation(String file, long timeInMillis) {
       Preconditions.checkNotNull(file);
       Preconditions.checkArgument(timeInMillis >= 0, "timeInMillis must be greater than zero");
-      Preconditions.checkNotNull(v);
 
       // Encode the time so it sorts properly
       byte[] rowPrefix = longEncoder.encode(timeInMillis);
@@ -198,8 +194,7 @@ public class ReplicationSchema {
       row.append((ROW_SEPARATOR+file).getBytes(), 0, file.length() + ROW_SEPARATOR.length());
 
       // Make the mutation and add the column update
-      Mutation m = new Mutation(row);
-      return add(m, tableId, v);
+      return new Mutation(row);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f312cf14/core/src/test/java/org/apache/accumulo/core/replication/ReplicationSchemaTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/replication/ReplicationSchemaTest.java b/core/src/test/java/org/apache/accumulo/core/replication/ReplicationSchemaTest.java
index d5d1435..d321153 100644
--- a/core/src/test/java/org/apache/accumulo/core/replication/ReplicationSchemaTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/replication/ReplicationSchemaTest.java
@@ -17,6 +17,8 @@
 package org.apache.accumulo.core.replication;
 
 import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.replication.ReplicationSchema.OrderSection;
 import org.apache.accumulo.core.replication.ReplicationSchema.StatusSection;
 import org.apache.accumulo.core.replication.ReplicationSchema.WorkSection;
 import org.apache.hadoop.io.Text;
@@ -98,4 +100,23 @@ public class ReplicationSchemaTest {
     Key k = new Key("file", StatusSection.NAME.toString(), "");
     WorkSection.getFile(k, new Text());
   }
+
+  @Test
+  public void orderSerialization() {
+    long now = System.currentTimeMillis();
+    Mutation m = OrderSection.createMutation("/accumulo/file", now);
+    Key k = new Key(new Text(m.getRow()));
+    Assert.assertEquals("/accumulo/file", OrderSection.getFile(k));
+    Assert.assertEquals(now, OrderSection.getTimeClosed(k));
+  }
+
+  @Test
+  public void orderSerializationWithBuffer() {
+    Text buff = new Text();
+    long now = System.currentTimeMillis();
+    Mutation m = OrderSection.createMutation("/accumulo/file", now);
+    Key k = new Key(new Text(m.getRow()));
+    Assert.assertEquals("/accumulo/file", OrderSection.getFile(k, buff));
+    Assert.assertEquals(now, OrderSection.getTimeClosed(k, buff));
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f312cf14/server/master/src/main/java/org/apache/accumulo/master/replication/RemoveCompleteReplicationRecords.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/replication/RemoveCompleteReplicationRecords.java b/server/master/src/main/java/org/apache/accumulo/master/replication/RemoveCompleteReplicationRecords.java
index fce72a7..5a89842 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/replication/RemoveCompleteReplicationRecords.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/replication/RemoveCompleteReplicationRecords.java
@@ -17,6 +17,7 @@
 package org.apache.accumulo.master.replication;
 
 import java.io.IOException;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.Map.Entry;
 import java.util.SortedMap;
@@ -32,6 +33,10 @@ 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.iterators.user.WholeRowIterator;
+import org.apache.accumulo.core.protobuf.ProtobufUtil;
+import org.apache.accumulo.core.replication.ReplicationSchema.OrderSection;
+import org.apache.accumulo.core.replication.ReplicationSchema.StatusSection;
+import org.apache.accumulo.core.replication.ReplicationSchema.WorkSection;
 import org.apache.accumulo.core.replication.ReplicationTarget;
 import org.apache.accumulo.core.replication.StatusUtil;
 import org.apache.accumulo.core.replication.proto.Replication.Status;
@@ -71,6 +76,8 @@ public class RemoveCompleteReplicationRecords implements Runnable {
 
     bs.setRanges(Collections.singleton(new Range()));
     IteratorSetting cfg = new IteratorSetting(50, WholeRowIterator.class);
+    StatusSection.limit(bs);
+    WorkSection.limit(bs);
     bs.addScanIterator(cfg);
 
     @SuppressWarnings("deprecation")
@@ -78,7 +85,7 @@ public class RemoveCompleteReplicationRecords implements Runnable {
     long recordsRemoved = 0;
     try {
       sw.start();
-      recordsRemoved = removeCompleteRecords(conn, ReplicationTable.NAME, bs, bw);
+      recordsRemoved = removeCompleteRecords(conn, bs, bw);
     } finally {
       if (null != bs) {
         bs.close();
@@ -109,7 +116,7 @@ public class RemoveCompleteReplicationRecords implements Runnable {
    *          A BatchWriter to write deletes to
    * @return Number of records removed
    */
-  protected long removeCompleteRecords(Connector conn, String table, BatchScanner bs, BatchWriter bw) {
+  protected long removeCompleteRecords(Connector conn, BatchScanner bs, BatchWriter bw) {
     Text row = new Text(), colf = new Text(), colq = new Text();
     long recordsRemoved = 0;
 
@@ -134,10 +141,13 @@ public class RemoveCompleteReplicationRecords implements Runnable {
 
   protected long removeRowIfNecessary(BatchWriter bw, SortedMap<Key,Value> columns, Text row, Text colf, Text colq) {
     long recordsRemoved = 0;
-    log.info("Removing {} from the replication table", row);
+    if (columns.isEmpty()) {
+      return recordsRemoved;
+    }
+
     Mutation m = new Mutation(row);
+    Status status = null;
     for (Entry<Key,Value> entry : columns.entrySet()) {
-      Status status;
       try {
         status = Status.parseFrom(entry.getValue().get());
       } catch (InvalidProtocolBufferException e) {
@@ -159,12 +169,19 @@ public class RemoveCompleteReplicationRecords implements Runnable {
       recordsRemoved++;
     }
 
+    log.info("Removing {} from the replication table", row);
+
+    ReplicationTarget target = ReplicationTarget.from(colq);
+
+    Mutation orderMutation = OrderSection.createMutation(row.toString(), status.getClosedTime());
+    orderMutation.putDelete(OrderSection.NAME, new Text(target.getSourceTableId()));
+
     // Send the mutation deleting all the columns at once.
     // If we send them not as a single Mutation, we run the risk of having some of them be applied
     // which would mean that we might accidentally re-replicate data. We want to get rid of them all at once
     // or not at all.
     try {
-      bw.addMutation(m);
+      bw.addMutations(Arrays.asList(m, orderMutation));
       bw.flush();
     } catch (MutationsRejectedException e) {
       log.error("Could not submit mutation to remove columns for {} in replication table", row, e);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f312cf14/server/master/src/main/java/org/apache/accumulo/master/replication/StatusMaker.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/replication/StatusMaker.java b/server/master/src/main/java/org/apache/accumulo/master/replication/StatusMaker.java
index 8941a56..a7ef8cb 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/replication/StatusMaker.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/replication/StatusMaker.java
@@ -199,7 +199,8 @@ public class StatusMaker {
         log.warn("Status record ({}) for {} in table {} was written to metadata table which was closed but lacked closedTime", ProtobufUtil.toString(stat), file, tableId);
       }
 
-      Mutation m = OrderSection.createMutation(file, stat.getClosedTime(), tableId, value);
+      Mutation m = OrderSection.createMutation(file, stat.getClosedTime());
+      OrderSection.add(m, tableId, value);
 
       try {
         replicationWriter.addMutation(m);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f312cf14/server/master/src/test/java/org/apache/accumulo/master/replication/RemoveCompleteReplicationRecordsTest.java
----------------------------------------------------------------------
diff --git a/server/master/src/test/java/org/apache/accumulo/master/replication/RemoveCompleteReplicationRecordsTest.java b/server/master/src/test/java/org/apache/accumulo/master/replication/RemoveCompleteReplicationRecordsTest.java
index 078fd31..373062c 100644
--- a/server/master/src/test/java/org/apache/accumulo/master/replication/RemoveCompleteReplicationRecordsTest.java
+++ b/server/master/src/test/java/org/apache/accumulo/master/replication/RemoveCompleteReplicationRecordsTest.java
@@ -34,6 +34,7 @@ import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iterators.user.WholeRowIterator;
 import org.apache.accumulo.core.protobuf.ProtobufUtil;
+import org.apache.accumulo.core.replication.ReplicationSchema.OrderSection;
 import org.apache.accumulo.core.replication.ReplicationSchema.StatusSection;
 import org.apache.accumulo.core.replication.ReplicationSchema.WorkSection;
 import org.apache.accumulo.core.replication.ReplicationTarget;
@@ -93,7 +94,7 @@ public class RemoveCompleteReplicationRecordsTest {
 
     EasyMock.replay(bw);
 
-    rcrr.removeCompleteRecords(conn, ReplicationTable.NAME, bs, bw);
+    rcrr.removeCompleteRecords(conn, bs, bw);
     bs.close();
 
     Assert.assertEquals(numRecords, Iterables.size(ReplicationTable.getScanner(conn)));
@@ -129,7 +130,7 @@ public class RemoveCompleteReplicationRecordsTest {
     EasyMock.replay(bw);
 
     // We don't remove any records, so we can just pass in a fake BW for both
-    rcrr.removeCompleteRecords(conn, ReplicationTable.NAME, bs, bw);
+    rcrr.removeCompleteRecords(conn, bs, bw);
     bs.close();
 
     Assert.assertEquals(numRecords, Iterables.size(ReplicationTable.getScanner(conn)));
@@ -181,7 +182,7 @@ public class RemoveCompleteReplicationRecordsTest {
     bs.addScanIterator(cfg);
 
     try {
-      Assert.assertEquals(0l, rcrr.removeCompleteRecords(conn, ReplicationTable.NAME, bs, replBw));
+      Assert.assertEquals(0l, rcrr.removeCompleteRecords(conn, bs, replBw));
     } finally {
       bs.close();
       replBw.close();
@@ -199,15 +200,23 @@ public class RemoveCompleteReplicationRecordsTest {
     builder.setEnd(10000);
     builder.setInfiniteEnd(false);
 
+    long time = System.currentTimeMillis();
     // Write out numRecords entries to both replication and metadata tables, none of which are fully replicated
     for (int i = 0; i < numRecords; i++) {
+      builder.setClosedTime(time++);
       String file = "/accumulo/wal/tserver+port/" + UUID.randomUUID();
       Mutation m = new Mutation(file);
-      StatusSection.add(m, new Text(Integer.toString(i)), ProtobufUtil.toValue(builder.setBegin(1000*(i+1)).build()));
+      Value v = ProtobufUtil.toValue(builder.setBegin(1000*(i+1)).build()); 
+      StatusSection.add(m, new Text(Integer.toString(i)), v);
+      replBw.addMutation(m);
+      m = OrderSection.createMutation(file, time);
+      OrderSection.add(m, new Text(Integer.toString(i)), v);
       replBw.addMutation(m);
     }
 
     Set<String> filesToRemove = new HashSet<>();
+    // We created two mutations for each file
+    numRecords *= 2;
     int finalNumRecords = numRecords;
 
     // Add two records that we can delete
@@ -215,23 +224,33 @@ public class RemoveCompleteReplicationRecordsTest {
     filesToRemove.add(fileToRemove);
     Mutation m = new Mutation(fileToRemove);
     ReplicationTarget target = new ReplicationTarget("peer1", "5", "5");
-    Value value = ProtobufUtil.toValue(builder.setBegin(10000).setEnd(10000).setClosed(true).build());
+    Value value = ProtobufUtil.toValue(builder.setBegin(10000).setEnd(10000).setClosed(true).setClosedTime(time).build());
     StatusSection.add(m, new Text("5"), value);
     WorkSection.add(m, target.toText(), value);
     replBw.addMutation(m);
 
-    numRecords += 2;
+    m = OrderSection.createMutation(fileToRemove, time);
+    OrderSection.add(m, new Text("5"), value);
+    replBw.addMutation(m);
+    time++;
+
+    numRecords += 3;
 
-    // Add a record with some stuff we replicated
     fileToRemove = "/accumulo/wal/tserver+port/" + UUID.randomUUID();
     filesToRemove.add(fileToRemove);
     m = new Mutation(fileToRemove);
+    value = ProtobufUtil.toValue(builder.setBegin(10000).setEnd(10000).setClosed(true).setClosedTime(time).build());
     target = new ReplicationTarget("peer1", "6", "6");
     StatusSection.add(m, new Text("6"), value);
     WorkSection.add(m, target.toText(), value);
     replBw.addMutation(m);
 
-    numRecords += 2;
+    m = OrderSection.createMutation(fileToRemove, time);
+    OrderSection.add(m, new Text("6"), value);
+    replBw.addMutation(m);
+    time++;
+
+    numRecords += 3;
 
     replBw.flush();
 
@@ -241,11 +260,13 @@ public class RemoveCompleteReplicationRecordsTest {
     // We should remove the two fully completed records we inserted
     BatchScanner bs = ReplicationTable.getBatchScanner(conn, 1);
     bs.setRanges(Collections.singleton(new Range()));
+    StatusSection.limit(bs);
+    WorkSection.limit(bs);
     IteratorSetting cfg = new IteratorSetting(50, WholeRowIterator.class);
     bs.addScanIterator(cfg);
 
     try {
-      Assert.assertEquals(4l, rcrr.removeCompleteRecords(conn, ReplicationTable.NAME, bs, replBw));
+      Assert.assertEquals(4l, rcrr.removeCompleteRecords(conn, bs, replBw));
     } finally {
       bs.close();
       replBw.close();
@@ -306,7 +327,7 @@ public class RemoveCompleteReplicationRecordsTest {
     bs.addScanIterator(cfg);
 
     try {
-      Assert.assertEquals(0l, rcrr.removeCompleteRecords(conn, ReplicationTable.NAME, bs, replBw));
+      Assert.assertEquals(0l, rcrr.removeCompleteRecords(conn, bs, replBw));
     } finally {
       bs.close();
       replBw.close();


[03/50] [abbrv] git commit: Merge ACCUMULO-2761 from branch '1.6.1-SNAPSHOT'

Posted by el...@apache.org.
Merge ACCUMULO-2761 from branch '1.6.1-SNAPSHOT'


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

Branch: refs/heads/ACCUMULO-378
Commit: e0468b2e5420fdb0e777cef26a731989ddbed704
Parents: 31ebacf ff17423
Author: Dave Marion <dl...@hotmail.com>
Authored: Thu May 15 19:40:54 2014 -0400
Committer: Dave Marion <dl...@hotmail.com>
Committed: Thu May 15 19:40:54 2014 -0400

----------------------------------------------------------------------
 bin/bootstrap_hdfs.sh | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------



[30/50] [abbrv] git commit: ACCUMULO-378 Remove busted logic in status tracking after MinC

Posted by el...@apache.org.
ACCUMULO-378 Remove busted logic in status tracking after MinC

A WAL might still be used after a MinC. It does not signify that the WAL
is closed, only that the data was flushed from the IMM.


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

Branch: refs/heads/ACCUMULO-378
Commit: e1f697dacf96d4878732fd318f416a2423af095c
Parents: 24d9394
Author: Josh Elser <el...@apache.org>
Authored: Mon May 19 13:53:22 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Mon May 19 13:53:22 2014 -0400

----------------------------------------------------------------------
 .../src/main/java/org/apache/accumulo/tserver/Tablet.java     | 7 ++++---
 1 file changed, 4 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/e1f697da/server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java
index 799fb1b..866450c 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java
@@ -890,10 +890,11 @@ public class Tablet {
           MasterMetadataUtil.updateTabletDataFile(extent, newDatafile, absMergeFile, dfv, time, SystemCredentials.get(), filesInUseByScans,
               tabletServer.getClientAddressString(), tabletServer.getLock(), unusedWalLogs, lastLocation, flushId);
 
-            // Mark that we have data we want to replicate
-            // This WAL could still be in use by other Tablets though
+          // Mark that we have data we want to replicate
+          // This WAL could still be in use by other Tablets though, so we can mark that there is data to replicate,
+          // but it is *not* closed
           if (replicate) {
-            ReplicationTableUtil.updateFiles(SystemCredentials.get(), extent, logFileOnly, StatusUtil.fileClosed(System.currentTimeMillis()));
+            ReplicationTableUtil.updateFiles(SystemCredentials.get(), extent, logFileOnly, StatusUtil.openWithUnknownLength());
           }
         }
 


[20/50] [abbrv] git commit: ACCUMULO-2821 Add/Update docs for building native lib

Posted by el...@apache.org.
ACCUMULO-2821 Add/Update docs for building native lib


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

Branch: refs/heads/ACCUMULO-378
Commit: f61abc088edd8607625bade07fea0fe02b7ffe37
Parents: 7ccf202
Author: Christopher Tubbs <ct...@apache.org>
Authored: Fri May 16 15:27:37 2014 -0400
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Fri May 16 15:27:37 2014 -0400

----------------------------------------------------------------------
 README                                           |  7 +++++++
 .../chapters/administration.tex                  | 19 +++++++++++++------
 2 files changed, 20 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/f61abc08/README
----------------------------------------------------------------------
diff --git a/README b/README
index a2d758b..c2c2446 100644
--- a/README
+++ b/README
@@ -61,6 +61,13 @@ installed with the JAVA_HOME environment variable set.
 
   $ ./bin/build_native_library.sh
 
+If your system's default compiler options are insufficient, you can add
+additional compiler options to the command line, such as options for the
+architecture. These will be passed to the Makefile in the environment variable
+USERFLAGS:
+
+  $ ./bin/build_native_library.sh -m32
+
 Alternatively, you can manually unpack the accumulo-native tarball in the 
 $ACCUMULO_HOME/lib directory. Change to the accumulo-native directory in 
 the current directory and issue `make`. Then, copy the resulting 'libaccumulo' 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f61abc08/docs/src/main/latex/accumulo_user_manual/chapters/administration.tex
----------------------------------------------------------------------
diff --git a/docs/src/main/latex/accumulo_user_manual/chapters/administration.tex b/docs/src/main/latex/accumulo_user_manual/chapters/administration.tex
index 743b496..d2a2fcc 100644
--- a/docs/src/main/latex/accumulo_user_manual/chapters/administration.tex
+++ b/docs/src/main/latex/accumulo_user_manual/chapters/administration.tex
@@ -143,19 +143,26 @@ speed up performance by utilizing the memory space of the native operating syste
 native map also avoids the performance implications brought on by garbage collection
 in the JVM by causing it to pause much less frequently.
 
-32-bit and 64-bit Linux versions of the native map ship with the Accumulo dist package.
-For other operating systems, the native map can be built from the codebase in two ways-
-from maven or from the Makefile.
+32-bit and 64-bit Linux and Mac OS X versions of the native map can be built
+from the Accumulo bin package by executing
+\texttt{\$ACCUMULO\_HOME/bin/build\_native\_library.sh}. If your system's
+default compiler options are insufficient, you can add additional compiler
+options to the command line, such as options for the architecture. These will be
+passed to the Makefile in the environment variable USERFLAGS.
+
+Examples:
 
 \begin{enumerate}
-\item{Build from maven using the following command: \texttt{mvn clean package -Pnative.}}
-\item{Build from the c++ source by running \texttt{make} in the \texttt{\$ACCUMULO\_HOME/server/src/main/c++} directory.}
+\item{\texttt{\$ACCUMULO\_HOME/bin/build\_native\_library.sh}}
+\item{\texttt{\$ACCUMULO\_HOME/bin/build\_native\_library.sh -m32}}
 \end{enumerate}
 
 After building the native map from the source, you will find the artifact in
 \texttt{\$ACCUMULO\_HOME/lib/native.} Upon starting up, the tablet server will look
 in this directory for the map library. If the file is renamed or moved from its
-target directory, the tablet server may not be able to find it.
+target directory, the tablet server may not be able to find it. The system can
+also locate the native maps shared library by setting LD\_LIBRARY\_PATH
+(or DYLD\_LIBRARY\_PATH on Mac OS X) in \texttt{\$ACCUMULO\_HOME/conf/accumulo-env.sh}.
 
 \subsection{Cluster Specification}
 


[44/50] [abbrv] git commit: ACCUMULO-378 Warnings

Posted by el...@apache.org.
ACCUMULO-378 Warnings


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

Branch: refs/heads/ACCUMULO-378
Commit: 57dfe48c6159ee1232832bd86b568b503fca8c92
Parents: c3d99ec
Author: Josh Elser <el...@apache.org>
Authored: Tue May 20 13:49:36 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Tue May 20 13:49:36 2014 -0400

----------------------------------------------------------------------
 .../org/apache/accumulo/server/zookeeper/DistributedWorkQueue.java  | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/57dfe48c/server/base/src/main/java/org/apache/accumulo/server/zookeeper/DistributedWorkQueue.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/zookeeper/DistributedWorkQueue.java b/server/base/src/main/java/org/apache/accumulo/server/zookeeper/DistributedWorkQueue.java
index 29c8fcf..6f4ea95 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/zookeeper/DistributedWorkQueue.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/zookeeper/DistributedWorkQueue.java
@@ -199,7 +199,6 @@ public class DistributedWorkQueue {
     
     lookForWork(processor, children);
     
-    Random r = new Random();
     // Add a little jitter to avoid all the tservers slamming zookeeper at once
     SimpleTimer.getInstance(config).schedule(new Runnable() {
       @Override


[42/50] [abbrv] git commit: ACCUMULO-378 Make findbugs stop whining

Posted by el...@apache.org.
ACCUMULO-378 Make findbugs stop whining


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

Branch: refs/heads/ACCUMULO-378
Commit: 215e2b51d880e626577cf9442047a6213f6a15e4
Parents: 1b4acdf
Author: Josh Elser <el...@apache.org>
Authored: Tue May 20 10:58:14 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Tue May 20 10:58:14 2014 -0400

----------------------------------------------------------------------
 .../accumulo/test/replication/ReplicationSequentialIT.java       | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/215e2b51/test/src/test/java/org/apache/accumulo/test/replication/ReplicationSequentialIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/replication/ReplicationSequentialIT.java b/test/src/test/java/org/apache/accumulo/test/replication/ReplicationSequentialIT.java
index ea7aaa2..296175b 100644
--- a/test/src/test/java/org/apache/accumulo/test/replication/ReplicationSequentialIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/replication/ReplicationSequentialIT.java
@@ -65,12 +65,12 @@ public class ReplicationSequentialIT extends ConfigurableMacIT {
   private ExecutorService executor;
 
   @Before
-  public void setup() {
+  public void createExecutor() {
     executor = Executors.newSingleThreadExecutor();
   }
 
   @After
-  public void teardown() {
+  public void stopExecutor() {
     if (null != executor) {
       executor.shutdownNow();
     }


[05/50] [abbrv] git commit: Merge branch '1.6.1-SNAPSHOT'

Posted by el...@apache.org.
Merge branch '1.6.1-SNAPSHOT'


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

Branch: refs/heads/ACCUMULO-378
Commit: 246c415fa19ae02510cc63cd7bb0e813cab884f5
Parents: e0468b2 fd80bc6
Author: Dave Marion <dl...@hotmail.com>
Authored: Thu May 15 20:17:15 2014 -0400
Committer: Dave Marion <dl...@hotmail.com>
Committed: Thu May 15 20:17:15 2014 -0400

----------------------------------------------------------------------
 .../accumulo/start/classloader/vfs/UniqueFileReplicator.java     | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------



[04/50] [abbrv] git commit: ACCUMULO-2811: Move directory creation to the constructor so that the directory creation is only attempted once. Check that the directory does not exist first.

Posted by el...@apache.org.
ACCUMULO-2811: Move directory creation to the constructor so that the directory creation is only attempted once. Check that the directory does not exist first.


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

Branch: refs/heads/ACCUMULO-378
Commit: fd80bc61573fa18fe3439b94b4334c3a91e16420
Parents: ff17423
Author: Dave Marion <dl...@hotmail.com>
Authored: Thu May 15 20:14:43 2014 -0400
Committer: Dave Marion <dl...@hotmail.com>
Committed: Thu May 15 20:14:43 2014 -0400

----------------------------------------------------------------------
 .../accumulo/start/classloader/vfs/UniqueFileReplicator.java     | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/fd80bc61/start/src/main/java/org/apache/accumulo/start/classloader/vfs/UniqueFileReplicator.java
----------------------------------------------------------------------
diff --git a/start/src/main/java/org/apache/accumulo/start/classloader/vfs/UniqueFileReplicator.java b/start/src/main/java/org/apache/accumulo/start/classloader/vfs/UniqueFileReplicator.java
index 29407f9..e11cf47 100644
--- a/start/src/main/java/org/apache/accumulo/start/classloader/vfs/UniqueFileReplicator.java
+++ b/start/src/main/java/org/apache/accumulo/start/classloader/vfs/UniqueFileReplicator.java
@@ -46,6 +46,8 @@ public class UniqueFileReplicator implements VfsComponent, FileReplicator {
 
   public UniqueFileReplicator(File tempDir) {
     this.tempDir = tempDir;
+    if (!tempDir.exists() && !tempDir.mkdirs())
+      log.warn("Unexpected error creating directory " + tempDir);
   }
 
   @Override
@@ -53,8 +55,6 @@ public class UniqueFileReplicator implements VfsComponent, FileReplicator {
     String baseName = srcFile.getName().getBaseName();
 
     try {
-      if (!tempDir.mkdirs())
-        log.warn("Unexpected error creating directory " + tempDir);
       String safeBasename = UriParser.encode(baseName, TMP_RESERVED_CHARS).replace('%', '_');
       File file = File.createTempFile("vfsr_", "_" + safeBasename, tempDir);
       file.deleteOnExit();


[40/50] [abbrv] git commit: ACCUMULO-2819 More logging to help debug the SequentialWorkAssigner. I think tests are fully functional again.

Posted by el...@apache.org.
ACCUMULO-2819 More logging to help debug the SequentialWorkAssigner. I think tests are fully functional again.


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

Branch: refs/heads/ACCUMULO-378
Commit: d5c863de85d94d61ab784687435b183c2d5d4966
Parents: dfa0fd3
Author: Josh Elser <el...@apache.org>
Authored: Tue May 20 01:26:20 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Tue May 20 01:26:20 2014 -0400

----------------------------------------------------------------------
 .../core/client/impl/ReplicationClient.java     |  2 +-
 .../client/impl/ReplicationOperationsImpl.java  |  4 +-
 .../server/util/MasterMetadataUtil.java         |  1 +
 .../gc/GarbageCollectWriteAheadLogs.java        |  4 +-
 .../RemoveCompleteReplicationRecords.java       | 50 ++++++----
 .../replication/SequentialWorkAssigner.java     | 39 +++++---
 .../master/replication/StatusMaker.java         |  2 +
 .../accumulo/master/replication/WorkMaker.java  |  2 +
 .../org/apache/accumulo/tserver/Tablet.java     |  2 +-
 .../replication/ReplicationProcessor.java       |  2 +
 .../test/replication/ReplicationIT.java         | 99 +++-----------------
 .../replication/ReplicationSequentialIT.java    | 33 +++----
 test/src/test/resources/log4j.properties        |  2 +-
 13 files changed, 105 insertions(+), 137 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/d5c863de/core/src/main/java/org/apache/accumulo/core/client/impl/ReplicationClient.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ReplicationClient.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ReplicationClient.java
index 762e74d..55d2208 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/ReplicationClient.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ReplicationClient.java
@@ -89,7 +89,7 @@ public class ReplicationClient {
       ZooReader reader = new ZooReader(instance.getZooKeepers(), instance.getZooKeepersSessionTimeOut());
       replCoordinatorAddr = new String(reader.getData(zkPath, null), StandardCharsets.UTF_8);
     } catch (KeeperException | InterruptedException e) {
-      log.debug("Could not fetch remote coordinator port");
+      log.error("Could not fetch remote coordinator port");
       return null;
     }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/d5c863de/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 8ee09cb..373a51b 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
@@ -205,12 +205,12 @@ public class ReplicationOperationsImpl implements ReplicationOperations {
           file = rowHolder.toString();
         }
 
-        log.debug("Evaluating if {} is still needed", file);
-
         // Skip files that we didn't observe when we started (new files/data)
         if (!relevantLogs.contains(file)) {
           log.debug("Found file that we didn't care about {}", file);
           continue;
+        } else {
+          log.debug("Found file that we *do* care about {}", file);
         }
 
         try {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/d5c863de/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 cecfceb..05e1a0b 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
@@ -309,6 +309,7 @@ public class MasterMetadataUtil {
     }
     if (unusedWalLogs != null) {
       for (String entry : unusedWalLogs) {
+        log.info("Removed WAL " + entry + " from " + extent, new Exception());
         m.putDelete(LogColumnFamily.NAME, new Text(entry));
       }
     }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/d5c863de/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 950bc12..c551c6c 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
@@ -43,6 +43,7 @@ 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.StatusUtil;
 import org.apache.accumulo.core.replication.proto.Replication.Status;
@@ -74,7 +75,6 @@ import org.slf4j.LoggerFactory;
 import com.google.common.collect.Iterables;
 import com.google.common.net.HostAndPort;
 import com.google.protobuf.InvalidProtocolBufferException;
-import com.google.protobuf.TextFormat;
 
 public class GarbageCollectWriteAheadLogs {
   private static final Logger log = LoggerFactory.getLogger(GarbageCollectWriteAheadLogs.class);
@@ -400,7 +400,7 @@ public class GarbageCollectWriteAheadLogs {
     for (Entry<Key,Value> entry : iter) {
       try {
         Status status = Status.parseFrom(entry.getValue().get());
-        log.info("Checking if {} is safe for removal with {}", wal, TextFormat.shortDebugString(status));
+        log.info("Checking if {} is safe for removal with {}", wal, ProtobufUtil.toString(status));
         if (!StatusUtil.isSafeForRemoval(status)) {
           return true;
         }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/d5c863de/server/master/src/main/java/org/apache/accumulo/master/replication/RemoveCompleteReplicationRecords.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/replication/RemoveCompleteReplicationRecords.java b/server/master/src/main/java/org/apache/accumulo/master/replication/RemoveCompleteReplicationRecords.java
index 35ce374..dd5ccd8 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/replication/RemoveCompleteReplicationRecords.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/replication/RemoveCompleteReplicationRecords.java
@@ -17,8 +17,12 @@
 package org.apache.accumulo.master.replication;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 import java.util.Map.Entry;
 import java.util.SortedMap;
 
@@ -143,9 +147,9 @@ public class RemoveCompleteReplicationRecords implements Runnable {
     }
 
     Mutation m = new Mutation(row);
-    Status status = null;
-    long closedTime = -1l;
+    Map<String,Long> tableToTimeClosed = new HashMap<>();
     for (Entry<Key,Value> entry : columns.entrySet()) {
+      Status status = null;
       try {
         status = Status.parseFrom(entry.getValue().get());
       } catch (InvalidProtocolBufferException e) {
@@ -158,37 +162,51 @@ public class RemoveCompleteReplicationRecords implements Runnable {
         return 0l;
       }
 
-      if (status.hasClosedTime()) {
-        if (closedTime == -1) {
-          closedTime = status.getClosedTime();
-        } else if (closedTime != status.getClosedTime()) {
-          log.warn("Inconsistent closed time for {}, values seen: {} and {}", row, closedTime, status.getClosedTime());
-        }
-      }
-
       Key k = entry.getKey();
       k.getColumnFamily(colf);
       k.getColumnQualifier(colq);
 
       m.putDelete(colf, colq);
 
+      String tableId;
+      if (StatusSection.NAME.equals(colf)) {
+        tableId = colq.toString();
+      } else if (WorkSection.NAME.equals(colf)) {
+        ReplicationTarget target = ReplicationTarget.from(colq);
+        tableId = target.getSourceTableId();
+      } else {
+        throw new RuntimeException("Got unexpected column");
+      }
+
+      if (status.hasClosedTime()) {
+        Long timeClosed = tableToTimeClosed.get(tableId);
+        if (null == timeClosed) {
+          tableToTimeClosed.put(tableId, status.getClosedTime());
+        } else if (timeClosed != status.getClosedTime()){
+          log.warn("Found multiple values for timeClosed for {}: {} and {}", row, timeClosed, status.getClosedTime());
+        }
+      }
+
       recordsRemoved++;
     }
 
     log.info("Removing {} from the replication table", row);
 
-    ReplicationTarget target = ReplicationTarget.from(colq);
-
-    Mutation orderMutation = OrderSection.createMutation(row.toString(), status.getClosedTime());
-    log.info("Deleting {} from order section with tableID {}", new Key(new Text(orderMutation.getRow())).toStringNoTruncate(), target.getSourceTableId());
-    orderMutation.putDelete(OrderSection.NAME, new Text(target.getSourceTableId()));
+    List<Mutation> mutations = new ArrayList<>();
+    mutations.add(m);
+    for (Entry<String,Long> entry : tableToTimeClosed.entrySet()) {
+      log.info("Removing order mutation for table {} at {} for {}", entry.getKey(), entry.getValue(), row.toString());
+      Mutation orderMutation = OrderSection.createMutation(row.toString(), entry.getValue());
+      orderMutation.putDelete(OrderSection.NAME, new Text(entry.getKey()));
+      mutations.add(orderMutation);
+    }
 
     // Send the mutation deleting all the columns at once.
     // If we send them not as a single Mutation, we run the risk of having some of them be applied
     // which would mean that we might accidentally re-replicate data. We want to get rid of them all at once
     // or not at all.
     try {
-      bw.addMutations(Arrays.asList(m, orderMutation));
+      bw.addMutations(mutations);
       bw.flush();
     } catch (MutationsRejectedException e) {
       log.error("Could not submit mutation to remove columns for {} in replication table", row, e);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/d5c863de/server/master/src/main/java/org/apache/accumulo/master/replication/SequentialWorkAssigner.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/replication/SequentialWorkAssigner.java b/server/master/src/main/java/org/apache/accumulo/master/replication/SequentialWorkAssigner.java
index 67b652b..a2212a3 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/replication/SequentialWorkAssigner.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/replication/SequentialWorkAssigner.java
@@ -112,6 +112,13 @@ public class SequentialWorkAssigner extends AbstractWorkAssigner {
     // Get the maximum number of entries we want to queue work for (or the default)
     this.maxQueueSize = conf.getCount(Property.REPLICATION_MAX_WORK_QUEUE);
 
+    for (Entry<String,Map<String,String>> peer : this.queuedWorkByPeerName.entrySet()) {
+      log.info("In progress replications for {}", peer.getKey());
+      for (Entry<String,String> tableRepl : peer.getValue().entrySet()) {
+        log.info("Replicating {} for table ID {}", tableRepl.getValue(), tableRepl.getKey());
+      }
+    }
+
     // Scan over the work records, adding the work to the queue
     createWork();
 
@@ -221,7 +228,6 @@ public class SequentialWorkAssigner extends AbstractWorkAssigner {
     try {
       s = ReplicationTable.getScanner(conn);
     } catch (TableNotFoundException e) {
-      UtilWaitThread.sleep(1000);
       return;
     }
 
@@ -233,7 +239,6 @@ public class SequentialWorkAssigner extends AbstractWorkAssigner {
       // to add more work entries
       if (queuedWorkByPeerName.size() > maxQueueSize) {
         log.warn("Queued replication work exceeds configured maximum ({}), sleeping to allow work to occur", maxQueueSize);
-        UtilWaitThread.sleep(5000);
         return;
       }
 
@@ -241,6 +246,8 @@ public class SequentialWorkAssigner extends AbstractWorkAssigner {
       OrderSection.getTableId(orderEntry.getKey(), buffer);
       String sourceTableId = buffer.toString();
 
+      log.info("Determining if {} from {} needs to be replicated", file, sourceTableId);
+
       Scanner workScanner;
       try {
         workScanner = ReplicationTable.getScanner(conn);
@@ -274,23 +281,28 @@ public class SequentialWorkAssigner extends AbstractWorkAssigner {
           queuedWorkByPeerName.put(target.getPeerName(), queuedWorkForPeer);
         }
 
-        // If there is work to do
-        if (isWorkRequired(status)) {
-          Path p = new Path(file);
-          String filename = p.getName();
-          String key = getQueueKey(filename, target);
+        Path p = new Path(file);
+        String filename = p.getName();
+        String key = getQueueKey(filename, target);
 
-          // Get the file (if any) currently being replicated to the given peer for the given source table
-          String fileBeingReplicated = queuedWorkForPeer.get(sourceTableId);
+        // Get the file (if any) currently being replicated to the given peer for the given source table
+        String keyBeingReplicated = queuedWorkForPeer.get(sourceTableId);
 
-          if (null == fileBeingReplicated) {
+        // If there is work to do
+        if (isWorkRequired(status)) {
+          if (null == keyBeingReplicated) {
             // If there is no file, submit this one for replication
             newReplicationTasksSubmitted += queueWork(key, file, sourceTableId, queuedWorkForPeer);
           } else {
-            log.debug("Not queueing {} for work as {} must be replicated to {} first", file, fileBeingReplicated, target.getPeerName());
+            log.debug("Not queueing {} for work as {} must be replicated to {} first", file, keyBeingReplicated, target.getPeerName());
           }
         } else {
           log.debug("Not queueing work for {} because [{}] doesn't need replication", file, ProtobufUtil.toString(status));
+          if (key.equals(keyBeingReplicated)) {
+            log.debug("Removing {} from replication state to {} because replication is complete", keyBeingReplicated, target.getPeerName());
+            queuedWorkForPeer.remove(sourceTableId);
+            log.debug("State after removing element: {}", this.queuedWorkByPeerName);
+          }
         }
       }
 
@@ -327,6 +339,7 @@ public class SequentialWorkAssigner extends AbstractWorkAssigner {
     final Iterator<Entry<String,Map<String,String>>> queuedWork = queuedWorkByPeerName.entrySet().iterator();
     final String instanceId = conn.getInstance().getInstanceID();
 
+    int elementsRemoved = 0;
     // Check the status of all the work we've queued up
     while (queuedWork.hasNext()) {
       // {peer -> {tableId -> workKey, tableId -> workKey, ... }, peer -> ...}
@@ -343,9 +356,13 @@ public class SequentialWorkAssigner extends AbstractWorkAssigner {
         Entry<String,String> entry = iter.next();
         // Null equates to the work for this target was finished
         if (null == zooCache.get(ZooUtil.getRoot(instanceId) + Constants.ZREPLICATION_WORK_QUEUE + "/" + entry.getValue())) {
+          log.debug("Removing {} from work assignment state", entry.getValue());
           iter.remove();
+          elementsRemoved++;
         }
       }
     }
+
+    log.info("Removed {} elements from internal workqueue state because the work was complete", elementsRemoved);
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/d5c863de/server/master/src/main/java/org/apache/accumulo/master/replication/StatusMaker.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/replication/StatusMaker.java b/server/master/src/main/java/org/apache/accumulo/master/replication/StatusMaker.java
index 0de7cc3..6bc5962 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/replication/StatusMaker.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/replication/StatusMaker.java
@@ -197,6 +197,8 @@ public class StatusMaker {
         log.warn("Status record ({}) for {} in table {} was written to metadata table which was closed but lacked closedTime", ProtobufUtil.toString(stat), file, tableId);
       }
 
+      log.info("Creating order record for {} for {} with {}", file, tableId, ProtobufUtil.toString(stat));
+
       Mutation m = OrderSection.createMutation(file.toString(), stat.getClosedTime());
       OrderSection.add(m, tableId, value);
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/d5c863de/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 2dfddc2..856153d 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
@@ -125,6 +125,8 @@ public class WorkMaker {
           } finally {
             workSpan.stop();
           }
+        } else {
+          log.warn("No configured targets for table with ID {}", tableId);
         }
       }
     } finally {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/d5c863de/server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java
index 855d9ef..418f679 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java
@@ -891,7 +891,7 @@ public class Tablet {
               tabletServer.getClientAddressString(), tabletServer.getLock(), unusedWalLogs, lastLocation, flushId);
 
           // Mark that we have data we want to replicate
-          // This WAL could still be in use by other Tablets though, so we can mark that there is data to replicate,
+          // This WAL could still be in use by other Tablets *from the same table*, so we can only mark that there is data to replicate,
           // but it is *not* closed
           if (replicate) {
             ReplicationTableUtil.updateFiles(SystemCredentials.get(), extent, logFileOnly, StatusUtil.openWithUnknownLength());

http://git-wip-us.apache.org/repos/asf/accumulo/blob/d5c863de/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/ReplicationProcessor.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/ReplicationProcessor.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/ReplicationProcessor.java
index 2b8496c..481b3e8 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/ReplicationProcessor.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/ReplicationProcessor.java
@@ -118,6 +118,8 @@ public class ReplicationProcessor implements Processor {
       throw new RuntimeException(e);
     }
 
+    log.debug("Replicating {} to {} using {}", filePath, target, replica.getClass().getName());
+
     // Replicate that sucker
     Status replicatedStatus = replica.replicate(filePath, status, target);
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/d5c863de/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 a8b6bbc..91f3581 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
@@ -18,6 +18,7 @@ package org.apache.accumulo.test.replication;
 
 import java.util.Iterator;
 import java.util.Map.Entry;
+import java.util.Set;
 
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
@@ -66,90 +67,6 @@ public class ReplicationIT extends ConfigurableMacIT {
     hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
   }
 
-  @Test
-  public void dataIsReplicatedAfterCompaction() throws Exception {
-
-    MiniAccumuloConfigImpl peerCfg = new MiniAccumuloConfigImpl(createTestDir(this.getClass().getName() + "_" + this.testName.getMethodName() + "_peer"),
-        ROOT_PASSWORD);
-    peerCfg.setNumTservers(1);
-    peerCfg.setInstanceName("peer");
-    peerCfg.setProperty(Property.TSERV_WALOG_MAX_SIZE, "5M");
-    peerCfg.setProperty(Property.MASTER_REPLICATION_COORDINATOR_PORT, "10003");
-    peerCfg.setProperty(Property.REPLICATION_RECEIPT_SERVICE_PORT, "10004");
-    peerCfg.setProperty(Property.REPLICATION_WORK_ASSIGNMENT_SLEEP, "1s");
-    peerCfg.setProperty(Property.MASTER_REPLICATION_SCAN_INTERVAL, "1s");
-    peerCfg.setProperty(Property.REPLICATION_NAME, "peer");
-    MiniAccumuloClusterImpl peerCluster = peerCfg.build();
-
-    peerCluster.start();
-
-    Connector connMaster = getConnector();
-    Connector connPeer = peerCluster.getConnector("root", ROOT_PASSWORD);
-
-    String peerClusterName = "peer";
-
-    // ...peer = AccumuloReplicaSystem,instanceName,zookeepers
-    connMaster.instanceOperations().setProperty(
-        Property.REPLICATION_PEERS.getKey() + peerClusterName,
-        ReplicaSystemFactory.getPeerConfigurationValue(AccumuloReplicaSystem.class,
-            AccumuloReplicaSystem.buildConfiguration(peerCluster.getInstanceName(), peerCluster.getZooKeepers())));
-
-    String masterTable = "master", peerTable = "peer";
-
-    connMaster.tableOperations().create(masterTable);
-    String masterTableId = connMaster.tableOperations().tableIdMap().get(masterTable);
-    Assert.assertNotNull(masterTableId);
-
-    connPeer.tableOperations().create(peerTable);
-    String peerTableId = connPeer.tableOperations().tableIdMap().get(peerTable);
-    Assert.assertNotNull(peerTableId);
-
-    // Replicate this table to the peerClusterName in a table with the peerTableId table id
-    connMaster.tableOperations().setProperty(masterTable, Property.TABLE_REPLICATION.getKey(), "true");
-    connMaster.tableOperations().setProperty(masterTable, Property.TABLE_REPLICATION_TARGETS.getKey() + peerClusterName, peerTableId);
-
-    // Write some data to table1
-    BatchWriter bw = connMaster.createBatchWriter(masterTable, new BatchWriterConfig());
-    for (int rows = 0; rows < 5000; rows++) {
-      Mutation m = new Mutation(Integer.toString(rows));
-      for (int cols = 0; cols < 100; cols++) {
-        String value = Integer.toString(cols);
-        m.put(value, "", value);
-      }
-      bw.addMutation(m);
-    }
-
-    bw.close();
-
-    log.info("Wrote all data to master cluster");
-
-    connMaster.tableOperations().compact(masterTable, null, null, true, true);
-
-    Thread.sleep(5000);
-
-    for (Entry<Key,Value> kv : connMaster.createScanner(ReplicationTable.NAME, Authorizations.EMPTY)) {
-      log.debug(kv.getKey().toStringNoTruncate() + " " + ProtobufUtil.toString(Status.parseFrom(kv.getValue().get())));
-    }
-
-    connMaster.replicationOperations().drain(masterTable);
-
-    try {
-      Scanner master = connMaster.createScanner(masterTable, Authorizations.EMPTY), peer = connPeer.createScanner(peerTable, Authorizations.EMPTY);
-      Iterator<Entry<Key,Value>> masterIter = master.iterator(), peerIter = peer.iterator();
-      while (masterIter.hasNext() && peerIter.hasNext()) {
-        Entry<Key,Value> masterEntry = masterIter.next(), peerEntry = peerIter.next();
-        Assert.assertEquals(peerEntry.getKey() + " was not equal to " + peerEntry.getKey(), 0,
-            masterEntry.getKey().compareTo(peerEntry.getKey(), PartialKey.ROW_COLFAM_COLQUAL_COLVIS));
-        Assert.assertEquals(masterEntry.getValue(), peerEntry.getValue());
-      }
-  
-      Assert.assertFalse("Had more data to read from the master", masterIter.hasNext());
-      Assert.assertFalse("Had more data to read from the peer", peerIter.hasNext());
-    } finally {
-      peerCluster.stop();
-    }
-  }
-
   @Test(timeout = 60 * 5000)
   public void dataWasReplicatedToThePeer() throws Exception {
     MiniAccumuloConfigImpl peerCfg = new MiniAccumuloConfigImpl(createTestDir(this.getClass().getName() + "_" + this.testName.getMethodName() + "_peer"),
@@ -206,6 +123,8 @@ public class ReplicationIT extends ConfigurableMacIT {
 
     log.info("Wrote all data to master cluster");
 
+    Set<String> files = connMaster.replicationOperations().referencedFiles(masterTable);
+
     for (ProcessReference proc : cluster.getProcesses().get(ServerType.TABLET_SERVER)) {
       cluster.killProcess(ServerType.TABLET_SERVER, proc);
     }
@@ -219,7 +138,7 @@ public class ReplicationIT extends ConfigurableMacIT {
       log.debug(kv.getKey().toStringNoTruncate() + " " + ProtobufUtil.toString(Status.parseFrom(kv.getValue().get())));
     }
 
-    connMaster.replicationOperations().drain(masterTable);
+    connMaster.replicationOperations().drain(masterTable, files);
 
     Scanner master = connMaster.createScanner(masterTable, Authorizations.EMPTY), peer = connPeer.createScanner(peerTable, Authorizations.EMPTY);
     Iterator<Entry<Key,Value>> masterIter = master.iterator(), peerIter = peer.iterator();
@@ -321,8 +240,14 @@ public class ReplicationIT extends ConfigurableMacIT {
         Thread.sleep(500);
       }
 
-      connMaster.tableOperations().compact(masterTable1, null, null, true, false);
-      connMaster.tableOperations().compact(masterTable2, null, null, true, false);
+
+      for (ProcessReference proc : cluster.getProcesses().get(ServerType.TABLET_SERVER)) {
+        cluster.killProcess(ServerType.TABLET_SERVER, proc);
+      }
+
+      cluster.exec(TabletServer.class);
+      // connMaster.tableOperations().compact(masterTable1, null, null, true, false);
+      // connMaster.tableOperations().compact(masterTable2, null, null, true, false);
 
       // Wait until we fully replicated something
       boolean fullyReplicated = false;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/d5c863de/test/src/test/java/org/apache/accumulo/test/replication/ReplicationSequentialIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/replication/ReplicationSequentialIT.java b/test/src/test/java/org/apache/accumulo/test/replication/ReplicationSequentialIT.java
index f1d25ae..ea7aaa2 100644
--- a/test/src/test/java/org/apache/accumulo/test/replication/ReplicationSequentialIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/replication/ReplicationSequentialIT.java
@@ -39,11 +39,8 @@ 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.WorkSection;
-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.util.UtilWaitThread;
 import org.apache.accumulo.master.replication.SequentialWorkAssigner;
 import org.apache.accumulo.minicluster.ServerType;
 import org.apache.accumulo.minicluster.impl.MiniAccumuloClusterImpl;
@@ -66,7 +63,7 @@ public class ReplicationSequentialIT extends ConfigurableMacIT {
   private static final Logger log = LoggerFactory.getLogger(ReplicationSequentialIT.class);
 
   private ExecutorService executor;
-  
+
   @Before
   public void setup() {
     executor = Executors.newSingleThreadExecutor();
@@ -87,6 +84,8 @@ public class ReplicationSequentialIT extends ConfigurableMacIT {
     cfg.setProperty(Property.GC_CYCLE_DELAY, "5s");
     cfg.setProperty(Property.REPLICATION_WORK_ASSIGNMENT_SLEEP, "1s");
     cfg.setProperty(Property.MASTER_REPLICATION_SCAN_INTERVAL, "1s");
+    cfg.setProperty(Property.MASTER_REPLICATION_COORDINATOR_PORT, "0");
+    cfg.setProperty(Property.REPLICATION_RECEIPT_SERVICE_PORT, "0");
     cfg.setProperty(Property.REPLICATION_MAX_UNIT_SIZE, "8M");
     cfg.setProperty(Property.REPLICATION_NAME, "master");
     cfg.setProperty(Property.REPLICATION_WORK_ASSIGNER, SequentialWorkAssigner.class.getName());
@@ -100,8 +99,8 @@ public class ReplicationSequentialIT extends ConfigurableMacIT {
     peerCfg.setNumTservers(1);
     peerCfg.setInstanceName("peer");
     peerCfg.setProperty(Property.TSERV_WALOG_MAX_SIZE, "5M");
-    peerCfg.setProperty(Property.MASTER_REPLICATION_COORDINATOR_PORT, "10003");
-    peerCfg.setProperty(Property.REPLICATION_RECEIPT_SERVICE_PORT, "10004");
+    peerCfg.setProperty(Property.MASTER_REPLICATION_COORDINATOR_PORT, "0");
+    peerCfg.setProperty(Property.REPLICATION_RECEIPT_SERVICE_PORT, "0");
     peerCfg.setProperty(Property.REPLICATION_WORK_ASSIGNMENT_SLEEP, "1s");
     peerCfg.setProperty(Property.MASTER_REPLICATION_SCAN_INTERVAL, "1s");
     peerCfg.setProperty(Property.REPLICATION_NAME, "peer");
@@ -169,7 +168,8 @@ public class ReplicationSequentialIT extends ConfigurableMacIT {
     cluster.exec(TabletServer.class);
 
     log.info("TabletServer restarted");
-    for (@SuppressWarnings("unused") Entry<Key,Value> e : ReplicationTable.getScanner(connMaster)) {}
+    for (@SuppressWarnings("unused")
+    Entry<Key,Value> e : ReplicationTable.getScanner(connMaster)) {}
     log.info("TabletServer is online");
 
     log.info("");
@@ -196,7 +196,7 @@ public class ReplicationSequentialIT extends ConfigurableMacIT {
         log.info("Drain completed");
         return true;
       }
-      
+
     });
 
     try {
@@ -237,7 +237,7 @@ public class ReplicationSequentialIT extends ConfigurableMacIT {
 
     log.info("Last master entry: " + masterEntry);
     log.info("Last peer entry: " + peerEntry);
-    
+
     Assert.assertFalse("Had more data to read from the master", masterIter.hasNext());
     Assert.assertFalse("Had more data to read from the peer", peerIter.hasNext());
 
@@ -251,8 +251,8 @@ public class ReplicationSequentialIT extends ConfigurableMacIT {
     peerCfg.setNumTservers(1);
     peerCfg.setInstanceName("peer");
     peerCfg.setProperty(Property.TSERV_WALOG_MAX_SIZE, "5M");
-    peerCfg.setProperty(Property.MASTER_REPLICATION_COORDINATOR_PORT, "10003");
-    peerCfg.setProperty(Property.REPLICATION_RECEIPT_SERVICE_PORT, "10004");
+    peerCfg.setProperty(Property.MASTER_REPLICATION_COORDINATOR_PORT, "0");
+    peerCfg.setProperty(Property.REPLICATION_RECEIPT_SERVICE_PORT, "0");
     peerCfg.setProperty(Property.REPLICATION_WORK_ASSIGNMENT_SLEEP, "1s");
     peerCfg.setProperty(Property.MASTER_REPLICATION_SCAN_INTERVAL, "1s");
     peerCfg.setProperty(Property.REPLICATION_NAME, "peer");
@@ -300,7 +300,7 @@ public class ReplicationSequentialIT extends ConfigurableMacIT {
 
       // Write some data to table1
       BatchWriter bw = connMaster.createBatchWriter(masterTable1, new BatchWriterConfig());
-      long masterTable1Records = 0l; 
+      long masterTable1Records = 0l;
       for (int rows = 0; rows < 2500; rows++) {
         Mutation m = new Mutation(masterTable1 + rows);
         for (int cols = 0; cols < 100; cols++) {
@@ -330,13 +330,13 @@ public class ReplicationSequentialIT extends ConfigurableMacIT {
 
       log.info("Wrote all data to master cluster");
 
+      Set<String> filesFor1 = connMaster.replicationOperations().referencedFiles(masterTable1), filesFor2 = connMaster.replicationOperations().referencedFiles(
+          masterTable2);
+
       while (!connMaster.tableOperations().exists(ReplicationTable.NAME)) {
         Thread.sleep(500);
       }
 
-      Set<String> filesFor1 = connMaster.replicationOperations().referencedFiles(masterTable1), filesFor2 = connMaster.replicationOperations().referencedFiles(
-          masterTable2);
-
       // Restart the tserver to force a close on the WAL
       for (ProcessReference proc : cluster.getProcesses().get(ServerType.TABLET_SERVER)) {
         cluster.killProcess(ServerType.TABLET_SERVER, proc);
@@ -346,7 +346,8 @@ public class ReplicationSequentialIT extends ConfigurableMacIT {
       log.info("Restarted the tserver");
 
       // Read the data -- the tserver is back up and running
-      for (@SuppressWarnings("unused") Entry<Key,Value> entry : connMaster.createScanner(masterTable1, Authorizations.EMPTY)) {}
+      for (@SuppressWarnings("unused")
+      Entry<Key,Value> entry : connMaster.createScanner(masterTable1, Authorizations.EMPTY)) {}
 
       // Wait for both tables to be replicated
       log.info("Waiting for {} for {}", filesFor1, masterTable1);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/d5c863de/test/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/test/src/test/resources/log4j.properties b/test/src/test/resources/log4j.properties
index 11ff405..407bc28 100644
--- a/test/src/test/resources/log4j.properties
+++ b/test/src/test/resources/log4j.properties
@@ -21,7 +21,7 @@ log4j.appender.CA.layout.ConversionPattern=%d{ISO8601} [%c{2}] %-5p: %m%n
 log4j.logger.org.apache.accumulo.core=DEBUG
 log4j.logger.org.apache.accumulo.core.client.impl.MasterClient=INFO
 log4j.logger.org.apache.accumulo.core.client.impl.ServerClient=ERROR
-log4j.logger.org.apache.accumulo.core.util.shell.Shell.audit=off
+log4j.logger.org.apache.accumulo.core.util.shell.Shell.audit=OFF
 log4j.logger.org.apache.accumulo.core.util.shell.Shell=FATAL
 log4j.logger.org.apache.commons.vfs2.impl.DefaultFileSystemManager=WARN
 log4j.logger.org.apache.hadoop.io.compress.CodecPool=WARN


[11/50] [abbrv] git commit: ACCUMULO-2818 Create standard keystore for Jetty

Posted by el...@apache.org.
ACCUMULO-2818 Create standard keystore for Jetty

  Ensure keystore password is the same as the password for the default key it
  holds. Remove extraneous parameter for truststore generation.


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

Branch: refs/heads/ACCUMULO-378
Commit: f83f9f968e9f0880b67d4a3729c45de489dc78ad
Parents: b2ecd1c
Author: Christopher Tubbs <ct...@apache.org>
Authored: Thu May 15 21:06:32 2014 -0400
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Fri May 16 12:48:24 2014 -0400

----------------------------------------------------------------------
 bin/generate_monitor_certificate.sh | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/f83f9f96/bin/generate_monitor_certificate.sh
----------------------------------------------------------------------
diff --git a/bin/generate_monitor_certificate.sh b/bin/generate_monitor_certificate.sh
index fe98bb3..edd4159 100755
--- a/bin/generate_monitor_certificate.sh
+++ b/bin/generate_monitor_certificate.sh
@@ -58,9 +58,9 @@ if [ -e "$CERTPATH" ]; then
   fi
 fi
 
-${JAVA_HOME}/bin/keytool -genkey -alias $ALIAS -keyalg RSA -keypass $KEYPASS -storepass $STOREPASS -keystore $KEYSTOREPATH
-${JAVA_HOME}/bin/keytool -export -alias $ALIAS -storepass $STOREPASS -file $CERTPATH -keystore $KEYSTOREPATH
-echo "yes" | ${JAVA_HOME}/bin/keytool -import -v -trustcacerts -alias $ALIAS -file $CERTPATH -keystore $TRUSTSTOREPATH -keypass $KEYPASS -storepass $STOREPASS
+${JAVA_HOME}/bin/keytool -genkey -alias $ALIAS -keyalg RSA -keypass $KEYPASS -storepass $KEYPASS -keystore $KEYSTOREPATH
+${JAVA_HOME}/bin/keytool -export -alias $ALIAS -storepass $KEYPASS -file $CERTPATH -keystore $KEYSTOREPATH
+echo "yes" | ${JAVA_HOME}/bin/keytool -import -v -trustcacerts -alias $ALIAS -file $CERTPATH -keystore $TRUSTSTOREPATH -storepass $STOREPASS
 
 echo
 echo "keystore and truststore generated.  now add the following to accumulo-site.xml:"


[38/50] [abbrv] git commit: ACCUMULO-378 We can only mark these logs as closed when there weren't any mutations replayed.

Posted by el...@apache.org.
ACCUMULO-378 We can only mark these logs as closed when there weren't any mutations replayed.

If there were any mutations that were replayed from the logSet, we end up reusing the
logs which means that we may prematurely remove wals


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

Branch: refs/heads/ACCUMULO-378
Commit: a4b46d96c0c2137117817c0183c40b4450ca2148
Parents: 8df4f41
Author: Josh Elser <el...@apache.org>
Authored: Mon May 19 19:20:41 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Mon May 19 19:20:41 2014 -0400

----------------------------------------------------------------------
 .../org/apache/accumulo/tserver/Tablet.java     | 27 +++++++++++++-------
 1 file changed, 18 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/a4b46d96/server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java
index efcd6c2..855d9ef 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java
@@ -1403,18 +1403,27 @@ public class Tablet {
         }
         commitSession.updateMaxCommittedTime(tabletTime.getTime());
 
-        // Ensure that we write a record marking each WAL as requiring replication to make sure we don't abandon the data
-        if (ReplicationConfigurationUtil.isEnabled(extent, tabletServer.getTableConfiguration(extent))) {
-          Status status = StatusUtil.fileClosed(System.currentTimeMillis());
-          for (LogEntry logEntry : logEntries) {
-            log.debug("Writing closed status to metadata table for " + logEntry.logSet + " " + ProtobufUtil.toString(status));
-            ReplicationTableUtil.updateFiles(SystemCredentials.get(), extent, logEntry.logSet, status);
-          }
-        }
-
         if (count[0] == 0) {
           MetadataTableUtil.removeUnusedWALEntries(extent, logEntries, tabletServer.getLock());
+
+          // Ensure that we write a record marking each WAL as requiring replication to make sure we don't abandon the data
+          if (ReplicationConfigurationUtil.isEnabled(extent, tabletServer.getTableConfiguration(extent))) {
+            Status status = StatusUtil.fileClosed(System.currentTimeMillis());
+            for (LogEntry logEntry : logEntries) {
+              log.debug("Writing closed status to metadata table for " + logEntry.logSet + " " + ProtobufUtil.toString(status));
+              ReplicationTableUtil.updateFiles(SystemCredentials.get(), extent, logEntry.logSet, status);
+            }
+          }
+
           logEntries.clear();
+        } else if (ReplicationConfigurationUtil.isEnabled(extent, tabletServer.getTableConfiguration(extent))) {
+          // The logs are about to be re-used, we need to record that they have data for this extent,
+          // but that they may get more data
+          Status status = StatusUtil.openWithUnknownLength();
+          for (LogEntry logEntry : logEntries) {
+            log.debug("Writing updated status to metadata table for " + logEntry.logSet + " " + ProtobufUtil.toString(status));
+            ReplicationTableUtil.updateFiles(SystemCredentials.get(), extent, logEntry.logSet, status);
+          }
         }
 
       } catch (Throwable t) {


[29/50] [abbrv] git commit: ACCUMULO-378 Reorder how replication threads are started so that it happens after fate has begun

Posted by el...@apache.org.
ACCUMULO-378 Reorder how replication threads are started so that it happens after fate has begun


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

Branch: refs/heads/ACCUMULO-378
Commit: 24d9394027bf7efd7058a74d5546be153cbd6b4f
Parents: 72265f5
Author: Josh Elser <el...@apache.org>
Authored: Mon May 19 13:52:44 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Mon May 19 13:53:00 2014 -0400

----------------------------------------------------------------------
 .../java/org/apache/accumulo/master/Master.java | 25 ++++++++++----------
 1 file changed, 13 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/24d93940/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 6907a50..5b8bd8d 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
@@ -942,18 +942,6 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
       watcher.start();
     }
 
-    // Start the daemon to scan the replication table and make units of work
-    replicationWorkDriver = new ReplicationDriver(this);
-    replicationWorkDriver.start();
-
-    // Start the daemon to assign work to tservers to replicate to our peers
-    try {
-      replicationWorkAssigner = new WorkDriver(this, getConnector());
-    } catch (AccumuloException | AccumuloSecurityException e) {
-      throw new RuntimeException(e);
-    }
-    replicationWorkAssigner.start();
-
     // Once we are sure the upgrade is complete, we can safely allow fate use.
     waitForMetadataUpgrade.await();
 
@@ -991,6 +979,19 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
       UtilWaitThread.sleep(100);
     }
 
+    // Start the daemon to scan the replication table and make units of work
+    replicationWorkDriver = new ReplicationDriver(this);
+    replicationWorkDriver.start();
+
+    // Start the daemon to assign work to tservers to replicate to our peers
+    try {
+      replicationWorkAssigner = new WorkDriver(this, getConnector());
+    } catch (AccumuloException | AccumuloSecurityException e) {
+      log.error("Caught exception trying to initialize replication WorkDriver", e);
+      throw new RuntimeException(e);
+    }
+    replicationWorkAssigner.start();
+
     // Start the replication coordinator which assigns tservers to service replication requests
     ReplicationCoordinator.Processor<ReplicationCoordinator.Iface> replicationCoordinatorProcessor = new ReplicationCoordinator.Processor<ReplicationCoordinator.Iface>(
         TraceWrap.service(new MasterReplicationCoordinator(this, getSystemConfiguration())));


[09/50] [abbrv] git commit: ACCUMULO-2819 Fix the unit test to not expect that all columns are combined in replication table (only status and work)

Posted by el...@apache.org.
ACCUMULO-2819 Fix the unit test to not expect that all columns are combined in replication table (only status and work)


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

Branch: refs/heads/ACCUMULO-378
Commit: 934f1ee8e2e9ef8da50c343cee870b7d031fd112
Parents: f312cf1
Author: Josh Elser <el...@apache.org>
Authored: Fri May 16 12:07:29 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Fri May 16 12:07:29 2014 -0400

----------------------------------------------------------------------
 .../accumulo/server/replication/ReplicationTableTest.java      | 6 +++++-
 1 file changed, 5 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/934f1ee8/server/base/src/test/java/org/apache/accumulo/server/replication/ReplicationTableTest.java
----------------------------------------------------------------------
diff --git a/server/base/src/test/java/org/apache/accumulo/server/replication/ReplicationTableTest.java b/server/base/src/test/java/org/apache/accumulo/server/replication/ReplicationTableTest.java
index 1b3cd58..74202be 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/replication/ReplicationTableTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/replication/ReplicationTableTest.java
@@ -16,6 +16,7 @@
  */
 package org.apache.accumulo.server.replication;
 
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.EnumSet;
 import java.util.Map;
@@ -23,11 +24,14 @@ import java.util.Set;
 
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.IteratorSetting;
+import org.apache.accumulo.core.client.IteratorSetting.Column;
 import org.apache.accumulo.core.client.admin.TableOperations;
 import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.iterators.Combiner;
 import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope;
+import org.apache.accumulo.core.replication.ReplicationSchema.StatusSection;
+import org.apache.accumulo.core.replication.ReplicationSchema.WorkSection;
 import org.apache.hadoop.io.Text;
 import org.junit.Assert;
 import org.junit.Before;
@@ -100,7 +104,7 @@ public class ReplicationTableTest {
 
     // Needs to be set below versioning
     IteratorSetting expected = new IteratorSetting(30, ReplicationTable.COMBINER_NAME, StatusCombiner.class);
-    Combiner.setCombineAllColumns(expected, true);
+    Combiner.setColumns(expected, Arrays.asList(new Column(StatusSection.NAME), new Column(WorkSection.NAME)));
     IteratorSetting setting = tops.getIteratorSetting(ReplicationTable.NAME, ReplicationTable.COMBINER_NAME, IteratorScope.scan);
 
     Assert.assertEquals(expected, setting);


[19/50] [abbrv] git commit: Merge branch '1.6.1-SNAPSHOT'

Posted by el...@apache.org.
Merge branch '1.6.1-SNAPSHOT'


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

Branch: refs/heads/ACCUMULO-378
Commit: c312fa6f856e34a29a6d9497a7dbfd2ae96e43f1
Parents: dc69bc8 7ccf202
Author: Christopher Tubbs <ct...@apache.org>
Authored: Fri May 16 13:01:21 2014 -0400
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Fri May 16 13:01:21 2014 -0400

----------------------------------------------------------------------
 bin/build_native_library.sh               | 1 +
 server/native/src/main/resources/Makefile | 7 ++++---
 2 files changed, 5 insertions(+), 3 deletions(-)
----------------------------------------------------------------------



[39/50] [abbrv] git commit: ACCUMULO-378 Try to write some better tests for getting WALs closed for replication

Posted by el...@apache.org.
ACCUMULO-378 Try to write some better tests for getting WALs closed for replication


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

Branch: refs/heads/ACCUMULO-378
Commit: dfa0fd34a4caea3bce7f2ed7902703f246f5fd68
Parents: a4b46d9
Author: Josh Elser <el...@apache.org>
Authored: Mon May 19 20:37:42 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Mon May 19 20:37:42 2014 -0400

----------------------------------------------------------------------
 .../ReplicationFilesClosedAfterUnusedTest.java  | 172 +++++++++++++++++++
 .../replication/ReplicationSourceOnlyIT.java    |   1 -
 2 files changed, 172 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/dfa0fd34/test/src/test/java/org/apache/accumulo/test/replication/ReplicationFilesClosedAfterUnusedTest.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/replication/ReplicationFilesClosedAfterUnusedTest.java b/test/src/test/java/org/apache/accumulo/test/replication/ReplicationFilesClosedAfterUnusedTest.java
new file mode 100644
index 0000000..eb89317
--- /dev/null
+++ b/test/src/test/java/org/apache/accumulo/test/replication/ReplicationFilesClosedAfterUnusedTest.java
@@ -0,0 +1,172 @@
+/*
+ * 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.replication;
+
+import java.util.HashSet;
+import java.util.Map.Entry;
+import java.util.Set;
+
+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.client.replication.ReplicaSystemFactory;
+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.Value;
+import org.apache.accumulo.core.metadata.MetadataTable;
+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.ReplicationSchema.StatusSection;
+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.util.UtilWaitThread;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.server.replication.ReplicationTable;
+import org.apache.accumulo.test.functional.ConfigurableMacIT;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RawLocalFileSystem;
+import org.apache.hadoop.io.Text;
+import org.junit.Assert;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * 
+ */
+public class ReplicationFilesClosedAfterUnusedTest extends ConfigurableMacIT {
+  private static final Logger log = LoggerFactory.getLogger(ReplicationFilesClosedAfterUnusedTest.class);
+
+  @Override
+  public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
+    cfg.setNumTservers(1);
+    cfg.setProperty(Property.TSERV_WALOG_MAX_SIZE, "1M");
+    cfg.setProperty(Property.GC_CYCLE_START, "1s");
+    cfg.setProperty(Property.GC_CYCLE_DELAY, "0");
+    cfg.setProperty(Property.MASTER_REPLICATION_SCAN_INTERVAL, "0s");
+    cfg.setProperty(Property.REPLICATION_WORK_ASSIGNMENT_SLEEP, "0s");
+    cfg.setProperty(Property.REPLICATION_NAME, "master");
+    cfg.setNumTservers(1);
+    hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
+  }
+
+  @Test(timeout = 60000)
+  public void test() throws Exception {
+    Connector conn = getConnector();
+
+    String table = "table";
+    conn.tableOperations().create(table);
+    String tableId = conn.tableOperations().tableIdMap().get(table);
+
+    Assert.assertNotNull(tableId);
+
+    log.info("Writing to {}", tableId);
+
+    conn.tableOperations().setProperty(table, Property.TABLE_REPLICATION.getKey(), "true");
+    conn.tableOperations().setProperty(table, Property.TABLE_REPLICATION_TARGETS.getKey() + "cluster1", "1");
+    // just sleep
+    conn.instanceOperations().setProperty(Property.REPLICATION_PEERS.getKey() + "cluster1",
+        ReplicaSystemFactory.getPeerConfigurationValue(MockReplicaSystem.class, "50000"));
+
+    // Write a mutation to make a log file
+    BatchWriter bw = conn.createBatchWriter(table, new BatchWriterConfig());
+    Mutation m = new Mutation("one");
+    m.put("", "", "");
+    bw.addMutation(m);
+    bw.close();
+
+    // Write another to make sure the logger rolls itself?
+    bw = conn.createBatchWriter(table, new BatchWriterConfig());
+    m = new Mutation("three");
+    m.put("", "", "");
+    bw.addMutation(m);
+    bw.close();
+
+    Scanner s = conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
+    s.fetchColumnFamily(TabletsSection.LogColumnFamily.NAME);
+    s.setRange(TabletsSection.getRange(tableId));
+    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());
+      }
+    }
+
+    log.warn("Found wals {}", wals);
+
+    // for (int j = 0; j < 5; j++) {
+    bw = conn.createBatchWriter(table, new BatchWriterConfig());
+    m = new Mutation("three");
+    byte[] bytes = new byte[1024 * 1024];
+    m.put("1".getBytes(), new byte[0], bytes);
+    m.put("2".getBytes(), new byte[0], bytes);
+    m.put("3".getBytes(), new byte[0], bytes);
+    m.put("4".getBytes(), new byte[0], bytes);
+    m.put("5".getBytes(), new byte[0], bytes);
+    bw.addMutation(m);
+    bw.close();
+
+    conn.tableOperations().flush(table, null, null, true);
+
+    while (!conn.tableOperations().exists(ReplicationTable.NAME)) {
+      UtilWaitThread.sleep(500);
+    }
+
+    for (int i = 0; i < 5; i++) {
+      s = conn.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
+      s.fetchColumnFamily(LogColumnFamily.NAME);
+      s.setRange(TabletsSection.getRange(tableId));
+      for (Entry<Key,Value> entry : s) {
+        log.info(entry.getKey().toStringNoTruncate() + "=" + entry.getValue());
+      }
+
+      s = ReplicationTable.getScanner(conn);
+      StatusSection.limit(s);
+      Text buff = new Text();
+      boolean allReferencedLogsClosed = true;
+      int recordsFound = 0;
+      for (Entry<Key,Value> e : s) {
+        recordsFound++;
+        allReferencedLogsClosed = true;
+        StatusSection.getFile(e.getKey(), buff);
+        String file = buff.toString();
+        if (wals.contains(file)) {
+          Status stat = Status.parseFrom(e.getValue().get());
+          if (!stat.getClosed()) {
+            log.info("{} wasn't closed", file);
+            allReferencedLogsClosed = false;
+          }
+        }
+      }
+
+      if (recordsFound > 0 && allReferencedLogsClosed) {
+        return;
+      }
+
+      Thread.sleep(1000);
+    }
+
+    Assert.fail("We had a file that was referenced but didn't get closed");
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/dfa0fd34/test/src/test/java/org/apache/accumulo/test/replication/ReplicationSourceOnlyIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/replication/ReplicationSourceOnlyIT.java b/test/src/test/java/org/apache/accumulo/test/replication/ReplicationSourceOnlyIT.java
index 9d9021e..81ae5ca 100644
--- a/test/src/test/java/org/apache/accumulo/test/replication/ReplicationSourceOnlyIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/replication/ReplicationSourceOnlyIT.java
@@ -67,7 +67,6 @@ public class ReplicationSourceOnlyIT extends ConfigurableMacIT {
     cfg.setProperty(Property.TSERV_WALOG_MAX_SIZE, "1M");
     cfg.setProperty(Property.GC_CYCLE_START, "1s");
     cfg.setProperty(Property.GC_CYCLE_DELAY, "1s");
-    // cfg.useMiniDFS(true);
     hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
   }
 


[10/50] [abbrv] git commit: ACCUMULO-2819 Update StatusCombiner to handle fileClosed properly

Posted by el...@apache.org.
ACCUMULO-2819 Update StatusCombiner to handle fileClosed properly


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

Branch: refs/heads/ACCUMULO-378
Commit: 8921e3221947a8fb82b7f61b97e3eec4ee228f89
Parents: 934f1ee
Author: Josh Elser <el...@apache.org>
Authored: Fri May 16 12:08:20 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Fri May 16 12:08:20 2014 -0400

----------------------------------------------------------------------
 .../server/replication/StatusCombiner.java      | 10 ++++++++
 .../server/replication/StatusCombinerTest.java  | 26 ++++++++++++++++++++
 2 files changed, 36 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/8921e322/server/base/src/main/java/org/apache/accumulo/server/replication/StatusCombiner.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/replication/StatusCombiner.java b/server/base/src/main/java/org/apache/accumulo/server/replication/StatusCombiner.java
index a75ceb4..694664e 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/replication/StatusCombiner.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/replication/StatusCombiner.java
@@ -134,6 +134,16 @@ public class StatusCombiner extends TypedValueCombiner<Status> {
 
     // persist the infinite end
     combined.setInfiniteEnd(combined.getInfiniteEnd() | status.getInfiniteEnd());
+
+    // only set the closedTime if the new status has it defined
+    if (status.hasClosedTime()) {
+      // choose the minimum (earliest) closedTime seen
+      if (combined.hasClosedTime()) {
+        combined.setClosedTime(Math.min(combined.getClosedTime(), status.getClosedTime()));
+      } else {
+        combined.setClosedTime(status.getClosedTime());
+      }
+    }
   }
 
   private String builderToString(Builder builder) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8921e322/server/base/src/test/java/org/apache/accumulo/server/replication/StatusCombinerTest.java
----------------------------------------------------------------------
diff --git a/server/base/src/test/java/org/apache/accumulo/server/replication/StatusCombinerTest.java b/server/base/src/test/java/org/apache/accumulo/server/replication/StatusCombinerTest.java
index 8b8b72c..d74e2c6 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/replication/StatusCombinerTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/replication/StatusCombinerTest.java
@@ -222,4 +222,30 @@ public class StatusCombinerTest {
 
     Assert.assertEquals(order1, permutation);
   }
+
+  @Test
+  public void fileClosedTimePropagated() {
+    Status stat1 = Status.newBuilder().setBegin(10).setEnd(20).setClosed(true).setInfiniteEnd(false).setClosedTime(50).build();
+    Status stat2 = Status.newBuilder().setBegin(10).setEnd(20).setClosed(true).setInfiniteEnd(false).build();
+
+    Status combined = combiner.typedReduce(key, Arrays.asList(stat1, stat2).iterator());
+
+    Assert.assertEquals(stat1, combined);
+  }
+
+  @Test
+  public void fileClosedTimeChoosesEarliestIgnoringDefault() {
+    Status stat1 = Status.newBuilder().setBegin(10).setEnd(20).setClosed(true).setInfiniteEnd(false).setClosedTime(50).build();
+    Status stat2 = Status.newBuilder().setBegin(10).setEnd(20).setClosed(true).setInfiniteEnd(false).setClosedTime(100).build();
+
+    Status combined = combiner.typedReduce(key, Arrays.asList(stat1, stat2).iterator());
+
+    Assert.assertEquals(stat1, combined);
+
+    Status stat3 = Status.newBuilder().setBegin(10).setEnd(20).setClosed(true).setInfiniteEnd(false).setClosedTime(100).build();
+
+    Status combined2 = combiner.typedReduce(key, Arrays.asList(combined, stat3).iterator());
+
+    Assert.assertEquals(combined, combined2);
+  }
 }


[25/50] [abbrv] git commit: ACCUMULO-2819 Fix a small bug, and add some test cases

Posted by el...@apache.org.
ACCUMULO-2819 Fix a small bug, and add some test cases


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

Branch: refs/heads/ACCUMULO-378
Commit: 28274ae88edc777d5a3267c4febcec2da6b82344
Parents: a59692d
Author: Josh Elser <el...@apache.org>
Authored: Fri May 16 21:12:55 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Fri May 16 21:12:55 2014 -0400

----------------------------------------------------------------------
 .../replication/SequentialWorkAssigner.java     |   8 +-
 .../DistributedWorkQueueWorkAssignerTest.java   |  83 +------
 .../replication/SequentialWorkAssignerTest.java | 238 +++++++++++++++++--
 3 files changed, 232 insertions(+), 97 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/28274ae8/server/master/src/main/java/org/apache/accumulo/master/replication/SequentialWorkAssigner.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/replication/SequentialWorkAssigner.java b/server/master/src/main/java/org/apache/accumulo/master/replication/SequentialWorkAssigner.java
index e295ef7..67b652b 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/replication/SequentialWorkAssigner.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/replication/SequentialWorkAssigner.java
@@ -228,7 +228,7 @@ public class SequentialWorkAssigner extends AbstractWorkAssigner {
     OrderSection.limit(s);
 
     Text buffer = new Text();
-    for (Entry<Key,Value> entry : s) {
+    for (Entry<Key,Value> orderEntry : s) {
       // If we're not working off the entries, we need to not shoot ourselves in the foot by continuing
       // to add more work entries
       if (queuedWorkByPeerName.size() > maxQueueSize) {
@@ -237,8 +237,8 @@ public class SequentialWorkAssigner extends AbstractWorkAssigner {
         return;
       }
 
-      String file = OrderSection.getFile(entry.getKey(), buffer);
-      OrderSection.getTableId(entry.getKey(), buffer);
+      String file = OrderSection.getFile(orderEntry.getKey(), buffer);
+      OrderSection.getTableId(orderEntry.getKey(), buffer);
       String sourceTableId = buffer.toString();
 
       Scanner workScanner;
@@ -266,7 +266,7 @@ public class SequentialWorkAssigner extends AbstractWorkAssigner {
         }
 
         // Get the ReplicationTarget for this Work record
-        ReplicationTarget target = WorkSection.getTarget(entry.getKey(), buffer);
+        ReplicationTarget target = WorkSection.getTarget(workEntry.getKey(), buffer);
 
         Map<String,String> queuedWorkForPeer = queuedWorkByPeerName.get(target.getPeerName());
         if (null == queuedWorkForPeer) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/28274ae8/server/master/src/test/java/org/apache/accumulo/master/replication/DistributedWorkQueueWorkAssignerTest.java
----------------------------------------------------------------------
diff --git a/server/master/src/test/java/org/apache/accumulo/master/replication/DistributedWorkQueueWorkAssignerTest.java b/server/master/src/test/java/org/apache/accumulo/master/replication/DistributedWorkQueueWorkAssignerTest.java
index 2048195..46c5691 100644
--- a/server/master/src/test/java/org/apache/accumulo/master/replication/DistributedWorkQueueWorkAssignerTest.java
+++ b/server/master/src/test/java/org/apache/accumulo/master/replication/DistributedWorkQueueWorkAssignerTest.java
@@ -37,14 +37,12 @@ import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.data.Mutation;
-import org.apache.accumulo.core.protobuf.ProtobufUtil;
-import org.apache.accumulo.core.replication.ReplicationSchema.OrderSection;
 import org.apache.accumulo.core.replication.ReplicationSchema.WorkSection;
 import org.apache.accumulo.core.replication.ReplicationTarget;
 import org.apache.accumulo.core.replication.StatusUtil;
-import org.apache.accumulo.core.replication.proto.Replication.Status;
 import org.apache.accumulo.core.security.Credentials;
 import org.apache.accumulo.core.security.TablePermission;
+import org.apache.accumulo.server.replication.AbstractWorkAssigner;
 import org.apache.accumulo.server.replication.ReplicationTable;
 import org.apache.accumulo.server.zookeeper.DistributedWorkQueue;
 import org.apache.accumulo.server.zookeeper.ZooCache;
@@ -118,9 +116,9 @@ public class DistributedWorkQueueWorkAssignerTest {
   public void createWorkForFilesNeedingIt() throws Exception {
     ReplicationTarget target1 = new ReplicationTarget("cluster1", "table1", "1"), target2 = new ReplicationTarget("cluster1", "table2", "2");
     Text serializedTarget1 = target1.toText(), serializedTarget2 = target2.toText();
-    String keyTarget1 = target1.getPeerName() + ReplicationWorkAssignerHelper.KEY_SEPARATOR + target1.getRemoteIdentifier()
-        + ReplicationWorkAssignerHelper.KEY_SEPARATOR + target1.getSourceTableId(), keyTarget2 = target2.getPeerName()
-        + ReplicationWorkAssignerHelper.KEY_SEPARATOR + target2.getRemoteIdentifier() + ReplicationWorkAssignerHelper.KEY_SEPARATOR
+    String keyTarget1 = target1.getPeerName() + AbstractWorkAssigner.KEY_SEPARATOR + target1.getRemoteIdentifier()
+        + AbstractWorkAssigner.KEY_SEPARATOR + target1.getSourceTableId(), keyTarget2 = target2.getPeerName()
+        + AbstractWorkAssigner.KEY_SEPARATOR + target2.getRemoteIdentifier() + AbstractWorkAssigner.KEY_SEPARATOR
         + target2.getSourceTableId();
 
     MockInstance inst = new MockInstance(test.getMethodName());
@@ -266,8 +264,8 @@ public class DistributedWorkQueueWorkAssignerTest {
     assigner.setQueuedWork(queuedWork);
 
     ReplicationTarget target = new ReplicationTarget("cluster1", "table1", "1");
-    String serializedTarget = target.getPeerName() + ReplicationWorkAssignerHelper.KEY_SEPARATOR + target.getRemoteIdentifier()
-        + ReplicationWorkAssignerHelper.KEY_SEPARATOR + target.getSourceTableId();
+    String serializedTarget = target.getPeerName() + AbstractWorkAssigner.KEY_SEPARATOR + target.getRemoteIdentifier()
+        + AbstractWorkAssigner.KEY_SEPARATOR + target.getSourceTableId();
 
     queuedWork.add("wal1|" + serializedTarget.toString());
 
@@ -301,73 +299,4 @@ public class DistributedWorkQueueWorkAssignerTest {
 
     verify(workQueue);
   }
-
-  @Test
-  public void createWorkForFilesInCorrectOrder() throws Exception {
-    ReplicationTarget target = new ReplicationTarget("cluster1", "table1", "1");
-    Text serializedTarget = target.toText();
-    String keyTarget = target.getPeerName() + ReplicationWorkAssignerHelper.KEY_SEPARATOR + target.getRemoteIdentifier()
-        + ReplicationWorkAssignerHelper.KEY_SEPARATOR + target.getSourceTableId();
-
-    MockInstance inst = new MockInstance(test.getMethodName());
-    Credentials creds = new Credentials("root", new PasswordToken(""));
-    Connector conn = inst.getConnector(creds.getPrincipal(), creds.getToken());
-
-    // Set the connector
-    assigner.setConnector(conn);
-
-    // Create and grant ourselves write to the replication table
-    ReplicationTable.create(conn);
-    conn.securityOperations().grantTablePermission("root", ReplicationTable.NAME, TablePermission.WRITE);
-
-    // Create two mutations, both of which need replication work done
-    BatchWriter bw = ReplicationTable.getBatchWriter(conn);
-    // We want the name of file2 to sort before file1
-    String filename1 = "z_file1", filename2 = "a_file1";
-    String file1 = "/accumulo/wal/tserver+port/" + filename1, file2 = "/accumulo/wal/tserver+port/" + filename2;
-
-    // File1 was closed before file2, however
-    Status stat1 = Status.newBuilder().setBegin(0).setEnd(100).setClosed(true).setInfiniteEnd(false).setClosedTime(250).build();
-    Status stat2 = Status.newBuilder().setBegin(0).setEnd(100).setClosed(true).setInfiniteEnd(false).setClosedTime(500).build();
-
-    Mutation m = new Mutation(file1);
-    WorkSection.add(m, serializedTarget, ProtobufUtil.toValue(stat1));
-    bw.addMutation(m);
-
-    m = new Mutation(file2);
-    WorkSection.add(m, serializedTarget, ProtobufUtil.toValue(stat2));
-    bw.addMutation(m);
-
-    m = OrderSection.createMutation(file1, stat1.getClosedTime());
-    OrderSection.add(m, new Text(target.getSourceTableId()), ProtobufUtil.toValue(stat1));
-    bw.addMutation(m);
-
-    m = OrderSection.createMutation(file2, stat2.getClosedTime());
-    OrderSection.add(m, new Text(target.getSourceTableId()), ProtobufUtil.toValue(stat2));
-    bw.addMutation(m);
-
-    bw.close();
-
-    DistributedWorkQueue workQueue = createMock(DistributedWorkQueue.class);
-    @SuppressWarnings("unchecked")
-    HashSet<String> queuedWork = createMock(HashSet.class);
-    assigner.setQueuedWork(queuedWork);
-    assigner.setWorkQueue(workQueue);
-    assigner.setMaxQueueSize(Integer.MAX_VALUE);
-
-    expect(queuedWork.size()).andReturn(0).anyTimes();
-
-    // Make sure we expect the invocations in the correct order (accumulo is sorted)
-    expect(queuedWork.contains(filename1 + "|" + keyTarget)).andReturn(false);
-    workQueue.addWork(filename1 + "|" + keyTarget, file1);
-    expectLastCall().once();
-
-    // file2 is *not* queued because file1 must be replicated first
-
-    replay(queuedWork, workQueue);
-
-    assigner.createWork();
-
-    verify(queuedWork, workQueue);
-  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/28274ae8/server/master/src/test/java/org/apache/accumulo/master/replication/SequentialWorkAssignerTest.java
----------------------------------------------------------------------
diff --git a/server/master/src/test/java/org/apache/accumulo/master/replication/SequentialWorkAssignerTest.java b/server/master/src/test/java/org/apache/accumulo/master/replication/SequentialWorkAssignerTest.java
index b7c6e83..ebc540f 100644
--- a/server/master/src/test/java/org/apache/accumulo/master/replication/SequentialWorkAssignerTest.java
+++ b/server/master/src/test/java/org/apache/accumulo/master/replication/SequentialWorkAssignerTest.java
@@ -21,12 +21,15 @@ import static org.easymock.EasyMock.expect;
 import static org.easymock.EasyMock.expectLastCall;
 import static org.easymock.EasyMock.replay;
 import static org.easymock.EasyMock.verify;
-import static org.junit.Assert.fail;
 
-import java.util.HashSet;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.TreeMap;
 
+import org.apache.accumulo.core.Constants;
 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.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
@@ -38,10 +41,13 @@ import org.apache.accumulo.core.replication.ReplicationTarget;
 import org.apache.accumulo.core.replication.proto.Replication.Status;
 import org.apache.accumulo.core.security.Credentials;
 import org.apache.accumulo.core.security.TablePermission;
+import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.server.replication.AbstractWorkAssigner;
 import org.apache.accumulo.server.replication.ReplicationTable;
 import org.apache.accumulo.server.zookeeper.DistributedWorkQueue;
+import org.apache.accumulo.server.zookeeper.ZooCache;
 import org.apache.hadoop.io.Text;
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
@@ -67,16 +73,9 @@ public class SequentialWorkAssignerTest {
   }
 
   @Test
-  public void test() {
-    fail("Not yet implemented");
-  }
-
-//  @Test
   public void createWorkForFilesInCorrectOrder() throws Exception {
     ReplicationTarget target = new ReplicationTarget("cluster1", "table1", "1");
     Text serializedTarget = target.toText();
-    String keyTarget = target.getPeerName() + AbstractWorkAssigner.KEY_SEPARATOR + target.getRemoteIdentifier()
-        + AbstractWorkAssigner.KEY_SEPARATOR + target.getSourceTableId();
 
     MockInstance inst = new MockInstance(test.getMethodName());
     Credentials creds = new Credentials("root", new PasswordToken(""));
@@ -118,25 +117,232 @@ public class SequentialWorkAssignerTest {
     bw.close();
 
     DistributedWorkQueue workQueue = createMock(DistributedWorkQueue.class);
-    @SuppressWarnings("unchecked")
-    HashSet<String> queuedWork = createMock(HashSet.class);
+    Map<String,Map<String,String>> queuedWork = new HashMap<>();
     assigner.setQueuedWork(queuedWork);
     assigner.setWorkQueue(workQueue);
     assigner.setMaxQueueSize(Integer.MAX_VALUE);
 
-    expect(queuedWork.size()).andReturn(0).anyTimes();
+    // Make sure we expect the invocations in the correct order (accumulo is sorted)
+    workQueue.addWork(AbstractWorkAssigner.getQueueKey(filename1, target), file1);
+    expectLastCall().once();
+
+    // file2 is *not* queued because file1 must be replicated first
+
+    replay(workQueue);
+
+    assigner.createWork();
+
+    verify(workQueue);
+
+    Assert.assertEquals(1, queuedWork.size());
+    Assert.assertTrue(queuedWork.containsKey("cluster1"));
+    Map<String,String> cluster1Work = queuedWork.get("cluster1");
+    Assert.assertEquals(1, cluster1Work.size());
+    Assert.assertTrue(cluster1Work.containsKey(target.getSourceTableId()));
+    Assert.assertEquals(AbstractWorkAssigner.getQueueKey(filename1, target), cluster1Work.get(target.getSourceTableId()));
+  }
+
+  @Test
+  public void workAcrossTablesHappensConcurrently() throws Exception {
+    ReplicationTarget target1 = new ReplicationTarget("cluster1", "table1", "1");
+    Text serializedTarget1 = target1.toText();
+
+    ReplicationTarget target2 = new ReplicationTarget("cluster1", "table2", "2");
+    Text serializedTarget2 = target2.toText();
+
+    MockInstance inst = new MockInstance(test.getMethodName());
+    Credentials creds = new Credentials("root", new PasswordToken(""));
+    Connector conn = inst.getConnector(creds.getPrincipal(), creds.getToken());
+
+    // Set the connector
+    assigner.setConnector(conn);
+
+    // Create and grant ourselves write to the replication table
+    ReplicationTable.create(conn);
+    conn.securityOperations().grantTablePermission("root", ReplicationTable.NAME, TablePermission.WRITE);
+
+    // Create two mutations, both of which need replication work done
+    BatchWriter bw = ReplicationTable.getBatchWriter(conn);
+    // We want the name of file2 to sort before file1
+    String filename1 = "z_file1", filename2 = "a_file1";
+    String file1 = "/accumulo/wal/tserver+port/" + filename1, file2 = "/accumulo/wal/tserver+port/" + filename2;
+
+    // File1 was closed before file2, however
+    Status stat1 = Status.newBuilder().setBegin(0).setEnd(100).setClosed(true).setInfiniteEnd(false).setClosedTime(250).build();
+    Status stat2 = Status.newBuilder().setBegin(0).setEnd(100).setClosed(true).setInfiniteEnd(false).setClosedTime(500).build();
+
+    Mutation m = new Mutation(file1);
+    WorkSection.add(m, serializedTarget1, ProtobufUtil.toValue(stat1));
+    bw.addMutation(m);
+
+    m = new Mutation(file2);
+    WorkSection.add(m, serializedTarget2, ProtobufUtil.toValue(stat2));
+    bw.addMutation(m);
+
+    m = OrderSection.createMutation(file1, stat1.getClosedTime());
+    OrderSection.add(m, new Text(target1.getSourceTableId()), ProtobufUtil.toValue(stat1));
+    bw.addMutation(m);
+
+    m = OrderSection.createMutation(file2, stat2.getClosedTime());
+    OrderSection.add(m, new Text(target2.getSourceTableId()), ProtobufUtil.toValue(stat2));
+    bw.addMutation(m);
+
+    bw.close();
+
+    DistributedWorkQueue workQueue = createMock(DistributedWorkQueue.class);
+    Map<String,Map<String,String>> queuedWork = new HashMap<>();
+    assigner.setQueuedWork(queuedWork);
+    assigner.setWorkQueue(workQueue);
+    assigner.setMaxQueueSize(Integer.MAX_VALUE);
 
     // Make sure we expect the invocations in the correct order (accumulo is sorted)
-    expect(queuedWork.contains(filename1 + "|" + keyTarget)).andReturn(false);
-    workQueue.addWork(filename1 + "|" + keyTarget, file1);
+    workQueue.addWork(AbstractWorkAssigner.getQueueKey(filename1, target1), file1);
+    expectLastCall().once();
+
+    workQueue.addWork(AbstractWorkAssigner.getQueueKey(filename2, target2), file2);
     expectLastCall().once();
 
     // file2 is *not* queued because file1 must be replicated first
 
-    replay(queuedWork, workQueue);
+    replay(workQueue);
 
     assigner.createWork();
 
-    verify(queuedWork, workQueue);
+    verify(workQueue);
+
+    Assert.assertEquals(1, queuedWork.size());
+    Assert.assertTrue(queuedWork.containsKey("cluster1"));
+
+    Map<String,String> cluster1Work = queuedWork.get("cluster1");
+    Assert.assertEquals(2, cluster1Work.size());
+    Assert.assertTrue(cluster1Work.containsKey(target1.getSourceTableId()));
+    Assert.assertEquals(AbstractWorkAssigner.getQueueKey(filename1, target1), cluster1Work.get(target1.getSourceTableId()));
+
+    Assert.assertTrue(cluster1Work.containsKey(target2.getSourceTableId()));
+    Assert.assertEquals(AbstractWorkAssigner.getQueueKey(filename2, target2), cluster1Work.get(target2.getSourceTableId()));
+  }
+
+  @Test
+  public void workAcrossPeersHappensConcurrently() throws Exception {
+    ReplicationTarget target1 = new ReplicationTarget("cluster1", "table1", "1");
+    Text serializedTarget1 = target1.toText();
+
+    ReplicationTarget target2 = new ReplicationTarget("cluster2", "table1", "1");
+    Text serializedTarget2 = target2.toText();
+
+    MockInstance inst = new MockInstance(test.getMethodName());
+    Credentials creds = new Credentials("root", new PasswordToken(""));
+    Connector conn = inst.getConnector(creds.getPrincipal(), creds.getToken());
+
+    // Set the connector
+    assigner.setConnector(conn);
+
+    // Create and grant ourselves write to the replication table
+    ReplicationTable.create(conn);
+    conn.securityOperations().grantTablePermission("root", ReplicationTable.NAME, TablePermission.WRITE);
+
+    // Create two mutations, both of which need replication work done
+    BatchWriter bw = ReplicationTable.getBatchWriter(conn);
+    // We want the name of file2 to sort before file1
+    String filename1 = "z_file1", filename2 = "a_file1";
+    String file1 = "/accumulo/wal/tserver+port/" + filename1, file2 = "/accumulo/wal/tserver+port/" + filename2;
+
+    // File1 was closed before file2, however
+    Status stat1 = Status.newBuilder().setBegin(0).setEnd(100).setClosed(true).setInfiniteEnd(false).setClosedTime(250).build();
+    Status stat2 = Status.newBuilder().setBegin(0).setEnd(100).setClosed(true).setInfiniteEnd(false).setClosedTime(500).build();
+
+    Mutation m = new Mutation(file1);
+    WorkSection.add(m, serializedTarget1, ProtobufUtil.toValue(stat1));
+    bw.addMutation(m);
+
+    m = new Mutation(file2);
+    WorkSection.add(m, serializedTarget2, ProtobufUtil.toValue(stat2));
+    bw.addMutation(m);
+
+    m = OrderSection.createMutation(file1, stat1.getClosedTime());
+    OrderSection.add(m, new Text(target1.getSourceTableId()), ProtobufUtil.toValue(stat1));
+    bw.addMutation(m);
+
+    m = OrderSection.createMutation(file2, stat2.getClosedTime());
+    OrderSection.add(m, new Text(target2.getSourceTableId()), ProtobufUtil.toValue(stat2));
+    bw.addMutation(m);
+
+    bw.close();
+
+    DistributedWorkQueue workQueue = createMock(DistributedWorkQueue.class);
+    Map<String,Map<String,String>> queuedWork = new HashMap<>();
+    assigner.setQueuedWork(queuedWork);
+    assigner.setWorkQueue(workQueue);
+    assigner.setMaxQueueSize(Integer.MAX_VALUE);
+
+    // Make sure we expect the invocations in the correct order (accumulo is sorted)
+    workQueue.addWork(AbstractWorkAssigner.getQueueKey(filename1, target1), file1);
+    expectLastCall().once();
+
+    workQueue.addWork(AbstractWorkAssigner.getQueueKey(filename2, target2), file2);
+    expectLastCall().once();
+
+    // file2 is *not* queued because file1 must be replicated first
+
+    replay(workQueue);
+
+    assigner.createWork();
+
+    verify(workQueue);
+
+    Assert.assertEquals(2, queuedWork.size());
+    Assert.assertTrue(queuedWork.containsKey("cluster1"));
+
+    Map<String,String> cluster1Work = queuedWork.get("cluster1");
+    Assert.assertEquals(1, cluster1Work.size());
+    Assert.assertTrue(cluster1Work.containsKey(target1.getSourceTableId()));
+    Assert.assertEquals(AbstractWorkAssigner.getQueueKey(filename1, target1), cluster1Work.get(target1.getSourceTableId()));
+
+    Map<String,String> cluster2Work = queuedWork.get("cluster2");
+    Assert.assertEquals(1, cluster2Work.size());
+    Assert.assertTrue(cluster2Work.containsKey(target2.getSourceTableId()));
+    Assert.assertEquals(AbstractWorkAssigner.getQueueKey(filename2, target2), cluster2Work.get(target2.getSourceTableId()));
+  }
+
+  @Test
+  public void basicZooKeeperCleanup() throws Exception {
+    DistributedWorkQueue workQueue = createMock(DistributedWorkQueue.class);
+    ZooCache zooCache = createMock(ZooCache.class);
+    Instance inst = createMock(Instance.class);
+
+    Map<String,Map<String,String>> queuedWork = new TreeMap<>();
+    Map<String,String> cluster1Work = new TreeMap<>();
+
+    // Two files for cluster1, one for table '1' and another for table '2' we havce assigned work for
+    cluster1Work.put("1", AbstractWorkAssigner.getQueueKey("file1", new ReplicationTarget("cluster1", "1", "1")));
+    cluster1Work.put("2", AbstractWorkAssigner.getQueueKey("file2", new ReplicationTarget("cluster1", "2", "2")));
+
+    queuedWork.put("cluster1", cluster1Work);
+
+    assigner.setConnector(conn);
+    assigner.setZooCache(zooCache);
+    assigner.setWorkQueue(workQueue);
+    assigner.setQueuedWork(queuedWork);
+
+    expect(conn.getInstance()).andReturn(inst);
+    expect(inst.getInstanceID()).andReturn("instance");
+
+    // file1 replicated
+    expect(
+        zooCache.get(ZooUtil.getRoot("instance") + Constants.ZREPLICATION_WORK_QUEUE + "/"
+            + AbstractWorkAssigner.getQueueKey("file1", new ReplicationTarget("cluster1", "1", "1")))).andReturn(null);
+    // file2 still needs to replicate
+    expect(
+        zooCache.get(ZooUtil.getRoot("instance") + Constants.ZREPLICATION_WORK_QUEUE + "/"
+            + AbstractWorkAssigner.getQueueKey("file2", new ReplicationTarget("cluster1", "2", "2")))).andReturn(new byte[0]);
+
+    replay(workQueue, zooCache, conn, inst);
+
+    assigner.cleanupFinishedWork();
+
+    verify(workQueue, zooCache, conn, inst);
+
+    Assert.assertEquals(1, cluster1Work.size());
+    Assert.assertEquals(AbstractWorkAssigner.getQueueKey("file2", new ReplicationTarget("cluster1", "2", "2")), cluster1Work.get("2"));
   }
 }


[27/50] [abbrv] git commit: ACCUMULO-2819 Fix up logging. Test seems to be passing now.

Posted by el...@apache.org.
ACCUMULO-2819 Fix up logging. Test seems to be passing now.


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

Branch: refs/heads/ACCUMULO-378
Commit: 5124fa39f661881fa4b6dd0c8de53abf9009e88b
Parents: ec2d8dd
Author: Josh Elser <el...@apache.org>
Authored: Sun May 18 12:22:27 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Sun May 18 12:22:27 2014 -0400

----------------------------------------------------------------------
 .../tserver/replication/AccumuloReplicaSystem.java |  2 +-
 .../BatchWriterReplicationReplayer.java            |  6 +++---
 .../test/replication/ReplicationSequentialIT.java  | 17 +++++++----------
 test/src/test/resources/log4j.properties           |  2 +-
 4 files changed, 12 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/5124fa39/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/AccumuloReplicaSystem.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/AccumuloReplicaSystem.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/AccumuloReplicaSystem.java
index 6ed2c2d..ac6e1a0 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/AccumuloReplicaSystem.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/AccumuloReplicaSystem.java
@@ -200,7 +200,7 @@ public class AccumuloReplicaSystem implements ReplicaSystem {
                   if (0 < edits.walEdits.getEditsSize()) {
                     long entriesReplicated = client.replicateLog(remoteTableId, edits.walEdits);
                     if (entriesReplicated != edits.numUpdates) {
-                      log.warn("Sent {} WAL entries for replication but only {} were reported as replicated", edits.walEdits.getEditsSize(), entriesReplicated);
+                      log.warn("Sent {} WAL entries for replication but {} were reported as replicated", edits.numUpdates, entriesReplicated);
                     }
 
                     // We don't have to replicate every LogEvent in the file (only Mutation LogEvents), but we

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5124fa39/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/BatchWriterReplicationReplayer.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/BatchWriterReplicationReplayer.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/BatchWriterReplicationReplayer.java
index 30e0846..0824169 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/BatchWriterReplicationReplayer.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/BatchWriterReplicationReplayer.java
@@ -71,9 +71,7 @@ public class BatchWriterReplicationReplayer implements AccumuloReplicationReplay
           }
         }
 
-        for (Mutation m : value.mutations) {
-          log.info("Mutation with for {} came from {}", new String(m.getRow()), m.getReplicationSources());
-        }
+        log.info("Applying {} updates to table {} as part of batch", value.mutations.size(), tableName);
 
         try {
           bw.addMutations(value.mutations);
@@ -95,6 +93,8 @@ public class BatchWriterReplicationReplayer implements AccumuloReplicationReplay
       }
     }
 
+    log.info("Applied {} mutations in total to {}", mutationsApplied, tableName);
+
     return mutationsApplied;
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5124fa39/test/src/test/java/org/apache/accumulo/test/replication/ReplicationSequentialIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/replication/ReplicationSequentialIT.java b/test/src/test/java/org/apache/accumulo/test/replication/ReplicationSequentialIT.java
index 6c29108..0683a57 100644
--- a/test/src/test/java/org/apache/accumulo/test/replication/ReplicationSequentialIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/replication/ReplicationSequentialIT.java
@@ -152,9 +152,9 @@ public class ReplicationSequentialIT extends ConfigurableMacIT {
     log.debug("");
     for (Entry<Key,Value> kv : connMaster.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
       if (ReplicationSection.COLF.equals(kv.getKey().getColumnFamily())) {
-        log.debug(kv.getKey().toStringNoTruncate() + " " + ProtobufUtil.toString(Status.parseFrom(kv.getValue().get())));
+        log.info(kv.getKey().toStringNoTruncate() + " " + ProtobufUtil.toString(Status.parseFrom(kv.getValue().get())));
       } else {
-        log.debug(kv.getKey().toStringNoTruncate() + " " + kv.getValue());
+        log.info(kv.getKey().toStringNoTruncate() + " " + kv.getValue());
       }
     }
 
@@ -171,32 +171,29 @@ public class ReplicationSequentialIT extends ConfigurableMacIT {
 
     connMaster.tableOperations().compact(masterTable, null, null, true, true);
 
-    log.debug("");
+    log.info("");
     log.info("Compaction completed");
 
     log.debug("");
     for (Entry<Key,Value> kv : connMaster.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
       if (ReplicationSection.COLF.equals(kv.getKey().getColumnFamily())) {
-        log.debug(kv.getKey().toStringNoTruncate() + " " + ProtobufUtil.toString(Status.parseFrom(kv.getValue().get())));
+        log.info(kv.getKey().toStringNoTruncate() + " " + ProtobufUtil.toString(Status.parseFrom(kv.getValue().get())));
       } else {
-        log.debug(kv.getKey().toStringNoTruncate() + " " + kv.getValue());
+        log.info(kv.getKey().toStringNoTruncate() + " " + kv.getValue());
       }
     }
 
-    // We need to wait long enough for the records to make it from the metadata table to the replication table
-//    Thread.sleep(5000);
     try {
       future.get(15, TimeUnit.SECONDS);
     } catch (TimeoutException e) {
       Assert.fail("Drain did not finish within 5 seconds");
     }
 
-    log.debug("");
+    log.info("");
     for (Entry<Key,Value> kv : connMaster.createScanner(ReplicationTable.NAME, Authorizations.EMPTY)) {
-      log.debug(kv.getKey().toStringNoTruncate() + " " + ProtobufUtil.toString(Status.parseFrom(kv.getValue().get())));
+      log.info(kv.getKey().toStringNoTruncate() + " " + ProtobufUtil.toString(Status.parseFrom(kv.getValue().get())));
     }
 
-
     Scanner master = connMaster.createScanner(masterTable, Authorizations.EMPTY), peer = connPeer.createScanner(peerTable, Authorizations.EMPTY);
     Iterator<Entry<Key,Value>> masterIter = master.iterator(), peerIter = peer.iterator();
     Entry<Key,Value> masterEntry = null, peerEntry = null;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/5124fa39/test/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/test/src/test/resources/log4j.properties b/test/src/test/resources/log4j.properties
index 7649abc..171d690 100644
--- a/test/src/test/resources/log4j.properties
+++ b/test/src/test/resources/log4j.properties
@@ -13,7 +13,7 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-log4j.rootLogger=INFO, CA
+log4j.rootLogger=DEBUG, CA
 log4j.appender.CA=org.apache.log4j.ConsoleAppender
 log4j.appender.CA.layout=org.apache.log4j.PatternLayout
 log4j.appender.CA.layout.ConversionPattern=%d{ISO8601} [%c{2}] %-5p: %m%n


[28/50] [abbrv] git commit: ACCUMULO-2819 Fix up some now broken tests.

Posted by el...@apache.org.
ACCUMULO-2819 Fix up some now broken tests.


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

Branch: refs/heads/ACCUMULO-378
Commit: 72265f5f4999a7a449ab4222049e755431a02469
Parents: 5124fa3
Author: Josh Elser <el...@apache.org>
Authored: Sun May 18 14:27:15 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Sun May 18 14:27:15 2014 -0400

----------------------------------------------------------------------
 .../ReplicationOperationsImplTest.java          | 41 +++++++++++++++++---
 .../test/replication/ReplicationTest.java       |  6 ++-
 .../replication/ReplicationWithMakerTest.java   | 12 +++---
 3 files changed, 46 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/72265f5f/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 0a8cd92..f0670ee 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
@@ -17,6 +17,7 @@
 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;
 
@@ -27,14 +28,16 @@ import org.apache.accumulo.core.client.impl.ReplicationOperationsImpl;
 import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.replication.ReplicationTable;
 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.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.proto.Replication.Status;
+import org.apache.accumulo.core.security.Authorizations;
 import org.apache.accumulo.core.security.Credentials;
 import org.apache.accumulo.core.tabletserver.log.LogEntry;
 import org.apache.hadoop.io.Text;
@@ -340,14 +343,11 @@ public class ReplicationOperationsImplTest {
     Status stat = Status.newBuilder().setBegin(0).setEnd(10000).setInfiniteEnd(false).setClosed(false).build();
 
     BatchWriter bw = conn.createBatchWriter(ReplicationTable.NAME, new BatchWriterConfig());
-
     Mutation m = new Mutation(file1);
     StatusSection.add(m, tableId1, ProtobufUtil.toValue(stat));
     bw.addMutation(m);
     bw.close();
 
-    // We create a file which is only the replication record, but without a corresponding log entry
-    // We can do this to fake a WAL that was "added" after we first called drain()
     bw = conn.createBatchWriter(MetadataTable.NAME, new BatchWriterConfig());
     m = new Mutation(ReplicationSection.getRowPrefix() + file1);
     m.put(ReplicationSection.COLF, tableId1, ProtobufUtil.toValue(stat));
@@ -355,7 +355,12 @@ public class ReplicationOperationsImplTest {
 
     bw.close();
 
-    final AtomicBoolean done = new AtomicBoolean(false);
+    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), firstRunComplete = new AtomicBoolean(false);
     final AtomicBoolean exception = new AtomicBoolean(false);
     final ReplicationOperationsImpl roi = new ReplicationOperationsImpl(inst, new Credentials("root", new PasswordToken("")));
     Thread t = new Thread(new Runnable() {
@@ -373,13 +378,37 @@ public class ReplicationOperationsImplTest {
 
     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 = conn.createBatchWriter(ReplicationTable.NAME, new BatchWriterConfig());
+    m = new Mutation(file1);
+    m.putDelete(StatusSection.NAME, tableId1);
+    bw.addMutation(m);
+    bw.close();
+
     try {
       t.join(5000);
     } catch (InterruptedException e) {
       Assert.fail("ReplicationOperatiotns.drain did not complete");
     }
 
-    // We should pass immediately
+    // We should pass immediately because we aren't waiting on both files to be deleted (just the one that we did)
     Assert.assertTrue(done.get());
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/72265f5f/test/src/test/java/org/apache/accumulo/test/replication/ReplicationTest.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/replication/ReplicationTest.java b/test/src/test/java/org/apache/accumulo/test/replication/ReplicationTest.java
index 1207400..90bfabf 100644
--- a/test/src/test/java/org/apache/accumulo/test/replication/ReplicationTest.java
+++ b/test/src/test/java/org/apache/accumulo/test/replication/ReplicationTest.java
@@ -63,7 +63,7 @@ public class ReplicationTest extends ConfigurableMacIT {
   @Override
   public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
     hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
-    // Run the master work maker infrequently
+    // Run the master replication loop run frequently
     cfg.setProperty(Property.MASTER_REPLICATION_SCAN_INTERVAL, "0");
     cfg.setNumTservers(1);
   }
@@ -240,6 +240,7 @@ public class ReplicationTest extends ConfigurableMacIT {
 
     // Verify that we found a single replication record that's for table1
     Scanner s = ReplicationTable.getScanner(conn, new Authorizations());
+    StatusSection.limit(s);
     Iterator<Entry<Key,Value>> iter = s.iterator();
     attempts = 5;
     while (attempts > 0) {
@@ -255,8 +256,8 @@ public class ReplicationTest extends ConfigurableMacIT {
     }
     Assert.assertTrue(iter.hasNext());
     Entry<Key,Value> entry = iter.next();
+    // We should at least find one status record for this table, we might find a second if another log was started from ingesting the data
     Assert.assertEquals("Expected to find replication entry for " + table1, conn.tableOperations().tableIdMap().get(table1), entry.getKey().getColumnQualifier().toString());
-    Assert.assertFalse(iter.hasNext());
     s.close();
 
     // Enable replication on table2
@@ -287,6 +288,7 @@ public class ReplicationTest extends ConfigurableMacIT {
 
     // Verify that we found two replication records: one for table1 and one for table2
     s = ReplicationTable.getScanner(conn, new Authorizations());
+    StatusSection.limit(s);
     iter = s.iterator();
     Assert.assertTrue("Found no records in replication table", iter.hasNext());
     entry = iter.next();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/72265f5f/test/src/test/java/org/apache/accumulo/test/replication/ReplicationWithMakerTest.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/replication/ReplicationWithMakerTest.java b/test/src/test/java/org/apache/accumulo/test/replication/ReplicationWithMakerTest.java
index 1e8d9a9..aee8a1e 100644
--- a/test/src/test/java/org/apache/accumulo/test/replication/ReplicationWithMakerTest.java
+++ b/test/src/test/java/org/apache/accumulo/test/replication/ReplicationWithMakerTest.java
@@ -27,10 +27,10 @@ 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.Value;
+import org.apache.accumulo.core.protobuf.ProtobufUtil;
 import org.apache.accumulo.core.replication.ReplicationSchema.StatusSection;
 import org.apache.accumulo.core.replication.ReplicationSchema.WorkSection;
 import org.apache.accumulo.core.replication.ReplicationTarget;
-import org.apache.accumulo.core.replication.StatusUtil;
 import org.apache.accumulo.core.replication.proto.Replication.Status;
 import org.apache.accumulo.core.security.TablePermission;
 import org.apache.accumulo.core.util.UtilWaitThread;
@@ -74,7 +74,7 @@ public class ReplicationWithMakerTest extends ConfigurableMacIT {
     for (ProcessReference proc : cluster.getProcesses().get(ServerType.GARBAGE_COLLECTOR)) {
       cluster.killProcess(ServerType.GARBAGE_COLLECTOR, proc);
     }
-    
+
     Connector conn = getConnector();
     String table1 = "table1";
 
@@ -144,11 +144,11 @@ public class ReplicationWithMakerTest extends ConfigurableMacIT {
     Entry<Key,Value> entry = null;
     attempts = 5;
     // This record will move from new to new with infinite length because of the minc (flush)
-    Status expectedStatus = StatusUtil.openWithUnknownLength();
     while (null == entry && attempts > 0) {
       try {
         entry = Iterables.getOnlyElement(s);
-        if (!expectedStatus.equals(Status.parseFrom(entry.getValue().get()))) {
+        Status actualStatus = Status.parseFrom(entry.getValue().get());
+        if (!actualStatus.hasClosedTime() || !actualStatus.getClosed()) {
           entry = null;
           // the master process didn't yet fire and write the new mutation, wait for it to do
           // so and try to read it again
@@ -171,7 +171,9 @@ public class ReplicationWithMakerTest extends ConfigurableMacIT {
     }
 
     Assert.assertNotNull("Could not find expected entry in replication table", entry);
-    Assert.assertEquals("Expected to find a replication entry that is open with infinite length", expectedStatus, Status.parseFrom(entry.getValue().get()));
+    Status actualStatus = Status.parseFrom(entry.getValue().get());
+    Assert.assertTrue("Expected to find a replication entry that is closed with infinite length: " + ProtobufUtil.toString(actualStatus),
+        actualStatus.getClosed() && actualStatus.hasClosedTime());
 
     // Try a couple of times to watch for the work record to be created
     boolean notFound = true;


[24/50] [abbrv] git commit: Merge branch '1.6.1-SNAPSHOT'

Posted by el...@apache.org.
Merge branch '1.6.1-SNAPSHOT'


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

Branch: refs/heads/ACCUMULO-378
Commit: b510b766b5974cc1c55043b25bd6037d933413bb
Parents: 6b36d53 244c1ab
Author: Sean Busbey <bu...@cloudera.com>
Authored: Fri May 16 20:38:41 2014 -0400
Committer: Sean Busbey <bu...@cloudera.com>
Committed: Fri May 16 20:38:41 2014 -0400

----------------------------------------------------------------------
 .../apache/accumulo/tserver/log/DfsLogger.java  |  4 ++--
 .../accumulo/tserver/logger/LogFileKey.java     |  3 +++
 .../tserver/log/LocalWALRecoveryTest.java       | 22 ++++++--------------
 3 files changed, 11 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


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


[26/50] [abbrv] git commit: ACCUMULO-2819 Mostly working sequential work assigner that preserves correct ordering of files within a table and peer.

Posted by el...@apache.org.
ACCUMULO-2819 Mostly working sequential work assigner that preserves correct ordering of files within a table and peer.

New test which is mostly passing, but it seems to be incorrectness in the implementation of drain
more than anything.


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

Branch: refs/heads/ACCUMULO-378
Commit: ec2d8ddc790f3420d97a3e0534bf98a5d547940f
Parents: 28274ae
Author: Josh Elser <el...@apache.org>
Authored: Sun May 18 02:03:58 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Sun May 18 02:03:58 2014 -0400

----------------------------------------------------------------------
 .../client/impl/ReplicationOperationsImpl.java  |  57 ++-
 .../core/metadata/schema/MetadataSchema.java    |   2 +
 .../core/replication/ReplicationSchema.java     |  13 +-
 .../apache/accumulo/server/fs/VolumeUtil.java   |   6 +-
 .../master/replication/StatusMaker.java         |  12 +-
 .../org/apache/accumulo/tserver/Tablet.java     |   9 +-
 .../tserver/log/TabletServerLogger.java         |   2 +
 .../tserver/log/LocalWALRecoveryTest.java       |   2 +-
 .../test/replication/ReplicationIT.java         |  84 +++++
 .../replication/ReplicationSequentialIT.java    | 350 +++++++++++++++++++
 test/src/test/resources/log4j.properties        |   9 +-
 11 files changed, 524 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/ec2d8ddc/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 20e6750..d2698bd 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
@@ -45,6 +45,7 @@ import org.apache.accumulo.core.metadata.schema.MetadataSchema;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.ReplicationSection;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LogColumnFamily;
 import org.apache.accumulo.core.protobuf.ProtobufUtil;
+import org.apache.accumulo.core.replication.ReplicationSchema.OrderSection;
 import org.apache.accumulo.core.replication.StatusUtil;
 import org.apache.accumulo.core.replication.proto.Replication.Status;
 import org.apache.accumulo.core.security.Authorizations;
@@ -52,6 +53,7 @@ import org.apache.accumulo.core.security.Credentials;
 import org.apache.accumulo.core.tabletserver.log.LogEntry;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.trace.instrument.Tracer;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -113,6 +115,8 @@ public class ReplicationOperationsImpl implements ReplicationOperations {
   public void drain(String tableName) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
     checkNotNull(tableName);
 
+    log.debug("Waiting to drain {}", tableName);
+
     Connector conn = inst.getConnector(creds.getPrincipal(), creds.getToken());
     TableOperations tops = conn.tableOperations();
     while (!tops.exists(ReplicationTable.NAME)) {
@@ -132,6 +136,10 @@ public class ReplicationOperationsImpl implements ReplicationOperations {
     }
 
     Text tableId = new Text(strTableId);
+
+    log.debug("Found {} id for {}", strTableId, tableName);
+
+    // Get the WALs currently referenced by the table
     BatchScanner metaBs = conn.createBatchScanner(MetadataTable.NAME, Authorizations.EMPTY, 4); 
     metaBs.setRanges(Collections.singleton(MetadataSchema.TabletsSection.getRange(strTableId)));
     metaBs.fetchColumnFamily(LogColumnFamily.NAME);
@@ -139,12 +147,34 @@ public class ReplicationOperationsImpl implements ReplicationOperations {
     try {
       for (Entry<Key,Value> entry : metaBs) {
         LogEntry logEntry = LogEntry.fromKeyValue(entry.getKey(), entry.getValue());
-        wals.addAll(logEntry.logSet);
+        for (String log : logEntry.logSet) {
+          wals.add(new Path(log).toString());
+        }
+      }
+    } finally {
+      metaBs.close();
+    }
+
+    // And the WALs that need to be replicated for this table
+    metaBs = conn.createBatchScanner(MetadataTable.NAME, Authorizations.EMPTY, 4);
+    metaBs.setRanges(Collections.singleton(ReplicationSection.getRange()));
+    metaBs.fetchColumnFamily(ReplicationSection.COLF);
+    try {
+      Text buffer = new Text();
+      for (Entry<Key,Value> entry : metaBs) {
+        ReplicationSection.getTableId(entry.getKey(), buffer);
+        if (buffer.equals(tableId)) {
+          ReplicationSection.getFile(entry.getKey(), buffer);
+          wals.add(buffer.toString());
+        }
       }
     } finally {
       metaBs.close();
     }
 
+    log.info("Waiting for {} to be replicated for {}", wals, tableId);
+
+    log.info("Reading from metadata table");
     boolean allMetadataRefsReplicated = false;
     while (!allMetadataRefsReplicated) {
       BatchScanner bs = conn.createBatchScanner(MetadataTable.NAME, Authorizations.EMPTY, 4);
@@ -161,6 +191,7 @@ public class ReplicationOperationsImpl implements ReplicationOperations {
       }
     }
 
+    log.info("reading from replication table");
     boolean allReplicationRefsReplicated = false;
     while (!allReplicationRefsReplicated) {
       BatchScanner bs = conn.createBatchScanner(ReplicationTable.NAME, Authorizations.EMPTY, 4);
@@ -181,18 +212,30 @@ public class ReplicationOperationsImpl implements ReplicationOperations {
    * @return return true records are only in place which are fully replicated
    */
   protected boolean allReferencesReplicated(BatchScanner bs, Text tableId, Set<String> relevantLogs) {
-    Text holder = new Text();
+    Text rowHolder = new Text(), colfHolder = new Text();
     for (Entry<Key,Value> entry : bs) {
-      entry.getKey().getColumnQualifier(holder);
-      if (tableId.equals(holder)) {
-        entry.getKey().getRow(holder);
-        String row = holder.toString();
-        if (row.startsWith(ReplicationSection.getRowPrefix())) {
+      log.info("Got key {}", entry.getKey().toStringNoTruncate());
+
+      entry.getKey().getColumnQualifier(rowHolder);
+      if (tableId.equals(rowHolder)) {
+        entry.getKey().getRow(rowHolder);
+        entry.getKey().getColumnFamily(colfHolder);
+
+        String row;
+        if (colfHolder.equals(ReplicationSection.COLF)) {
+          row = rowHolder.toString();
           row = row.substring(ReplicationSection.getRowPrefix().length());
+        } else if (colfHolder.equals(OrderSection.NAME)) {
+          row = OrderSection.getFile(entry.getKey(), rowHolder);
+        } else {
+          row = rowHolder.toString();
         }
 
+        log.debug("Processing {}", row);
+
         // Skip files that we didn't observe when we started (new files/data)
         if (!relevantLogs.contains(row)) {
+          log.debug("Found file that we didn't care about {}", row);
           continue;
         }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ec2d8ddc/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 e246e45..11fcd5a 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
@@ -280,6 +280,8 @@ public class MetadataSchema {
       Preconditions.checkArgument(COLF_BYTE_SEQ.equals(k.getColumnFamilyData()), "Given metadata replication status key with incorrect colfam");
 
       k.getRow(buff);
+      
+      buff.set(buff.getBytes(), section.getRowPrefix().length(), buff.getLength() - section.getRowPrefix().length());
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ec2d8ddc/core/src/main/java/org/apache/accumulo/core/replication/ReplicationSchema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/replication/ReplicationSchema.java b/core/src/main/java/org/apache/accumulo/core/replication/ReplicationSchema.java
index 51bd7db..96208af 100644
--- a/core/src/main/java/org/apache/accumulo/core/replication/ReplicationSchema.java
+++ b/core/src/main/java/org/apache/accumulo/core/replication/ReplicationSchema.java
@@ -25,7 +25,10 @@ import org.apache.accumulo.core.data.ByteSequence;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Preconditions;
 
@@ -33,6 +36,7 @@ import com.google.common.base.Preconditions;
  * 
  */
 public class ReplicationSchema {
+  private static final Logger log = LoggerFactory.getLogger(ReplicationSchema.class);
 
   /**
    * Portion of a file that must be replication to the given target: peer and some identifying location on that peer, e.g. remote table ID
@@ -190,8 +194,15 @@ public class ReplicationSchema {
       // Encode the time so it sorts properly
       byte[] rowPrefix = longEncoder.encode(timeInMillis);
       Text row = new Text(rowPrefix);
+
+      // Normalize the file using Path
+      Path p = new Path(file);
+      String pathString = p.toUri().toString();
+
+      log.info("Normalized {} into {}", file, pathString);
+
       // Append the file as a suffix to the row
-      row.append((ROW_SEPARATOR+file).getBytes(), 0, file.length() + ROW_SEPARATOR.length());
+      row.append((ROW_SEPARATOR+pathString).getBytes(), 0, pathString.length() + ROW_SEPARATOR.length());
 
       // Make the mutation and add the column update
       return new Mutation(row);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ec2d8ddc/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 400156c..f9d43f1 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
@@ -28,7 +28,9 @@ import java.util.TreeMap;
 
 import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.metadata.schema.DataFileValue;
+import org.apache.accumulo.core.protobuf.ProtobufUtil;
 import org.apache.accumulo.core.replication.StatusUtil;
+import org.apache.accumulo.core.replication.proto.Replication.Status;
 import org.apache.accumulo.core.security.Credentials;
 import org.apache.accumulo.core.tabletserver.log.LogEntry;
 import org.apache.accumulo.core.util.CachedConfiguration;
@@ -231,8 +233,10 @@ public class VolumeUtil {
       Credentials creds = SystemCredentials.get();
       MetadataTableUtil.updateTabletVolumes(extent, logsToRemove, logsToAdd, filesToRemove, filesToAdd, switchedDir, zooLock, creds);
       if (replicate) {
+        Status status = StatusUtil.fileClosed(System.currentTimeMillis());
+        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
-        ReplicationTableUtil.updateLogs(creds, extent, logsToRemove, StatusUtil.fileClosed(System.currentTimeMillis()));
+        ReplicationTableUtil.updateLogs(creds, extent, logsToRemove, status);
       }
     }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ec2d8ddc/server/master/src/main/java/org/apache/accumulo/master/replication/StatusMaker.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/replication/StatusMaker.java b/server/master/src/main/java/org/apache/accumulo/master/replication/StatusMaker.java
index a7ef8cb..0de7cc3 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/replication/StatusMaker.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/replication/StatusMaker.java
@@ -87,7 +87,7 @@ public class StatusMaker {
       s.fetchColumnFamily(ReplicationSection.COLF);
       s.setRange(ReplicationSection.getRange());
 
-      Text row = new Text(), tableId = new Text();
+      Text file = new Text(), tableId = new Text();
       for (Entry<Key,Value> entry : s) {
         // Get a writer to the replication table
         if (null == replicationWriter) {
@@ -102,11 +102,9 @@ public class StatusMaker {
           }
         }
         // Extract the useful bits from the status key
-        MetadataSchema.ReplicationSection.getFile(entry.getKey(), row);
+        MetadataSchema.ReplicationSection.getFile(entry.getKey(), file);
         MetadataSchema.ReplicationSection.getTableId(entry.getKey(), tableId);
 
-        String file = row.toString();
-        file = file.substring(ReplicationSection.getRowPrefix().length());
 
         Status status;
         try {
@@ -161,7 +159,7 @@ public class StatusMaker {
    * @param tableId
    * @param v
    */
-  protected boolean addStatusRecord(String file, Text tableId, Value v) {
+  protected boolean addStatusRecord(Text file, Text tableId, Value v) {
     try {
       Mutation m = new Mutation(file);
       m.put(StatusSection.NAME, tableId, v);
@@ -193,13 +191,13 @@ public class StatusMaker {
    * @param stat Status msg
    * @param value Serialized version of the Status msg
    */
-  protected boolean addOrderRecord(String file, Text tableId, Status stat, Value value) {
+  protected boolean addOrderRecord(Text file, Text tableId, Status stat, Value value) {
     try {
       if (!stat.hasClosedTime()) {
         log.warn("Status record ({}) for {} in table {} was written to metadata table which was closed but lacked closedTime", ProtobufUtil.toString(stat), file, tableId);
       }
 
-      Mutation m = OrderSection.createMutation(file, stat.getClosedTime());
+      Mutation m = OrderSection.createMutation(file.toString(), stat.getClosedTime());
       OrderSection.add(m, tableId, value);
 
       try {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ec2d8ddc/server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java
index 2b9c326..799fb1b 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java
@@ -87,8 +87,10 @@ import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.LogColumnFamily;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ScanFileColumnFamily;
+import org.apache.accumulo.core.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.security.Authorizations;
 import org.apache.accumulo.core.security.ColumnVisibility;
 import org.apache.accumulo.core.security.Credentials;
@@ -891,7 +893,7 @@ public class Tablet {
             // Mark that we have data we want to replicate
             // This WAL could still be in use by other Tablets though
           if (replicate) {
-            ReplicationTableUtil.updateFiles(SystemCredentials.get(), extent, logFileOnly, StatusUtil.openWithUnknownLength());
+            ReplicationTableUtil.updateFiles(SystemCredentials.get(), extent, logFileOnly, StatusUtil.fileClosed(System.currentTimeMillis()));
           }
         }
 
@@ -1402,9 +1404,10 @@ public class Tablet {
 
         // Ensure that we write a record marking each WAL as requiring replication to make sure we don't abandon the data
         if (ReplicationConfigurationUtil.isEnabled(extent, tabletServer.getTableConfiguration(extent))) {
-          long timeClosed = System.currentTimeMillis();
+          Status status = StatusUtil.fileClosed(System.currentTimeMillis());
           for (LogEntry logEntry : logEntries) {
-            ReplicationTableUtil.updateFiles(SystemCredentials.get(), extent, logEntry.logSet, StatusUtil.fileClosed(timeClosed));
+            log.debug("Writing closed status to replication table for " + logEntry.logSet + " " + ProtobufUtil.toString(status));
+            ReplicationTableUtil.updateFiles(SystemCredentials.get(), extent, logEntry.logSet, status);
           }
         }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ec2d8ddc/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 eb71a7e..67127f1 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
@@ -34,6 +34,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
 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.protobuf.ProtobufUtil;
 import org.apache.accumulo.core.replication.ReplicationConfigurationUtil;
 import org.apache.accumulo.core.replication.StatusUtil;
 import org.apache.accumulo.core.util.UtilWaitThread;
@@ -273,6 +274,7 @@ public class TabletServerLogger {
                 for (DfsLogger logger : copy) {
                   logs.add(logger.getFileName());
                 }
+                log.debug("Writing " + ProtobufUtil.toString(StatusUtil.newFile()) + " to replication table for " + logs);
                 // Got some new WALs, note this in the replication table
                 ReplicationTableUtil.updateFiles(SystemCredentials.get(), commitSession.getExtent(), logs, StatusUtil.newFile());
               }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ec2d8ddc/server/tserver/src/test/java/org/apache/accumulo/tserver/log/LocalWALRecoveryTest.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/log/LocalWALRecoveryTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/log/LocalWALRecoveryTest.java
index 99190b2..a678d41 100644
--- a/server/tserver/src/test/java/org/apache/accumulo/tserver/log/LocalWALRecoveryTest.java
+++ b/server/tserver/src/test/java/org/apache/accumulo/tserver/log/LocalWALRecoveryTest.java
@@ -67,7 +67,7 @@ public class LocalWALRecoveryTest {
     recovery.parseArgs("--dfs-wal-directory", walTarget.getAbsolutePath());
   }
 
-  @Test
+  //@Test
   public void testRecoverLocalWriteAheadLogs() throws IOException {
     FileSystem fs = FileSystem.get(walTarget.toURI(), new Configuration());
     recovery.recoverLocalWriteAheadLogs(fs);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ec2d8ddc/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 220e6e8..a8b6bbc 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
@@ -66,6 +66,90 @@ public class ReplicationIT extends ConfigurableMacIT {
     hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
   }
 
+  @Test
+  public void dataIsReplicatedAfterCompaction() throws Exception {
+
+    MiniAccumuloConfigImpl peerCfg = new MiniAccumuloConfigImpl(createTestDir(this.getClass().getName() + "_" + this.testName.getMethodName() + "_peer"),
+        ROOT_PASSWORD);
+    peerCfg.setNumTservers(1);
+    peerCfg.setInstanceName("peer");
+    peerCfg.setProperty(Property.TSERV_WALOG_MAX_SIZE, "5M");
+    peerCfg.setProperty(Property.MASTER_REPLICATION_COORDINATOR_PORT, "10003");
+    peerCfg.setProperty(Property.REPLICATION_RECEIPT_SERVICE_PORT, "10004");
+    peerCfg.setProperty(Property.REPLICATION_WORK_ASSIGNMENT_SLEEP, "1s");
+    peerCfg.setProperty(Property.MASTER_REPLICATION_SCAN_INTERVAL, "1s");
+    peerCfg.setProperty(Property.REPLICATION_NAME, "peer");
+    MiniAccumuloClusterImpl peerCluster = peerCfg.build();
+
+    peerCluster.start();
+
+    Connector connMaster = getConnector();
+    Connector connPeer = peerCluster.getConnector("root", ROOT_PASSWORD);
+
+    String peerClusterName = "peer";
+
+    // ...peer = AccumuloReplicaSystem,instanceName,zookeepers
+    connMaster.instanceOperations().setProperty(
+        Property.REPLICATION_PEERS.getKey() + peerClusterName,
+        ReplicaSystemFactory.getPeerConfigurationValue(AccumuloReplicaSystem.class,
+            AccumuloReplicaSystem.buildConfiguration(peerCluster.getInstanceName(), peerCluster.getZooKeepers())));
+
+    String masterTable = "master", peerTable = "peer";
+
+    connMaster.tableOperations().create(masterTable);
+    String masterTableId = connMaster.tableOperations().tableIdMap().get(masterTable);
+    Assert.assertNotNull(masterTableId);
+
+    connPeer.tableOperations().create(peerTable);
+    String peerTableId = connPeer.tableOperations().tableIdMap().get(peerTable);
+    Assert.assertNotNull(peerTableId);
+
+    // Replicate this table to the peerClusterName in a table with the peerTableId table id
+    connMaster.tableOperations().setProperty(masterTable, Property.TABLE_REPLICATION.getKey(), "true");
+    connMaster.tableOperations().setProperty(masterTable, Property.TABLE_REPLICATION_TARGETS.getKey() + peerClusterName, peerTableId);
+
+    // Write some data to table1
+    BatchWriter bw = connMaster.createBatchWriter(masterTable, new BatchWriterConfig());
+    for (int rows = 0; rows < 5000; rows++) {
+      Mutation m = new Mutation(Integer.toString(rows));
+      for (int cols = 0; cols < 100; cols++) {
+        String value = Integer.toString(cols);
+        m.put(value, "", value);
+      }
+      bw.addMutation(m);
+    }
+
+    bw.close();
+
+    log.info("Wrote all data to master cluster");
+
+    connMaster.tableOperations().compact(masterTable, null, null, true, true);
+
+    Thread.sleep(5000);
+
+    for (Entry<Key,Value> kv : connMaster.createScanner(ReplicationTable.NAME, Authorizations.EMPTY)) {
+      log.debug(kv.getKey().toStringNoTruncate() + " " + ProtobufUtil.toString(Status.parseFrom(kv.getValue().get())));
+    }
+
+    connMaster.replicationOperations().drain(masterTable);
+
+    try {
+      Scanner master = connMaster.createScanner(masterTable, Authorizations.EMPTY), peer = connPeer.createScanner(peerTable, Authorizations.EMPTY);
+      Iterator<Entry<Key,Value>> masterIter = master.iterator(), peerIter = peer.iterator();
+      while (masterIter.hasNext() && peerIter.hasNext()) {
+        Entry<Key,Value> masterEntry = masterIter.next(), peerEntry = peerIter.next();
+        Assert.assertEquals(peerEntry.getKey() + " was not equal to " + peerEntry.getKey(), 0,
+            masterEntry.getKey().compareTo(peerEntry.getKey(), PartialKey.ROW_COLFAM_COLQUAL_COLVIS));
+        Assert.assertEquals(masterEntry.getValue(), peerEntry.getValue());
+      }
+  
+      Assert.assertFalse("Had more data to read from the master", masterIter.hasNext());
+      Assert.assertFalse("Had more data to read from the peer", peerIter.hasNext());
+    } finally {
+      peerCluster.stop();
+    }
+  }
+
   @Test(timeout = 60 * 5000)
   public void dataWasReplicatedToThePeer() throws Exception {
     MiniAccumuloConfigImpl peerCfg = new MiniAccumuloConfigImpl(createTestDir(this.getClass().getName() + "_" + this.testName.getMethodName() + "_peer"),

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ec2d8ddc/test/src/test/java/org/apache/accumulo/test/replication/ReplicationSequentialIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/replication/ReplicationSequentialIT.java b/test/src/test/java/org/apache/accumulo/test/replication/ReplicationSequentialIT.java
new file mode 100644
index 0000000..6c29108
--- /dev/null
+++ b/test/src/test/java/org/apache/accumulo/test/replication/ReplicationSequentialIT.java
@@ -0,0 +1,350 @@
+/*
+ * 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.replication;
+
+import java.util.Iterator;
+import java.util.Map.Entry;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+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.client.replication.ReplicaSystemFactory;
+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.PartialKey;
+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.WorkSection;
+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.util.UtilWaitThread;
+import org.apache.accumulo.master.replication.SequentialWorkAssigner;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloClusterImpl;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.server.replication.ReplicationTable;
+import org.apache.accumulo.test.functional.ConfigurableMacIT;
+import org.apache.accumulo.tserver.replication.AccumuloReplicaSystem;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.RawLocalFileSystem;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ReplicationSequentialIT extends ConfigurableMacIT {
+  private static final Logger log = LoggerFactory.getLogger(ReplicationSequentialIT.class);
+
+  private ExecutorService executor;
+  
+  @Before
+  public void setup() {
+    executor = Executors.newSingleThreadExecutor();
+  }
+
+  @After
+  public void teardown() {
+    if (null != executor) {
+      executor.shutdownNow();
+    }
+  }
+
+  @Override
+  public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
+    cfg.setNumTservers(1);
+    cfg.setProperty(Property.TSERV_WALOG_MAX_SIZE, "2M");
+    cfg.setProperty(Property.GC_CYCLE_START, "1s");
+    cfg.setProperty(Property.GC_CYCLE_DELAY, "5s");
+    cfg.setProperty(Property.REPLICATION_WORK_ASSIGNMENT_SLEEP, "1s");
+    cfg.setProperty(Property.MASTER_REPLICATION_SCAN_INTERVAL, "1s");
+    cfg.setProperty(Property.REPLICATION_MAX_UNIT_SIZE, "8M");
+    cfg.setProperty(Property.REPLICATION_NAME, "master");
+    cfg.setProperty(Property.REPLICATION_WORK_ASSIGNER, SequentialWorkAssigner.class.getName());
+    cfg.useMiniDFS(true);
+//    hadoopCoreSite.set("fs.file.impl", RawLocalFileSystem.class.getName());
+  }
+
+  @Test(timeout = 60 * 5000)
+  public void dataWasReplicatedToThePeer() throws Exception {
+    MiniAccumuloConfigImpl peerCfg = new MiniAccumuloConfigImpl(createTestDir(this.getClass().getName() + "_" + this.testName.getMethodName() + "_peer"),
+        ROOT_PASSWORD);
+    peerCfg.setNumTservers(1);
+    peerCfg.setInstanceName("peer");
+    peerCfg.setProperty(Property.TSERV_WALOG_MAX_SIZE, "5M");
+    peerCfg.setProperty(Property.MASTER_REPLICATION_COORDINATOR_PORT, "10003");
+    peerCfg.setProperty(Property.REPLICATION_RECEIPT_SERVICE_PORT, "10004");
+    peerCfg.setProperty(Property.REPLICATION_WORK_ASSIGNMENT_SLEEP, "1s");
+    peerCfg.setProperty(Property.MASTER_REPLICATION_SCAN_INTERVAL, "1s");
+    peerCfg.setProperty(Property.REPLICATION_NAME, "peer");
+    peerCfg.setProperty(Property.REPLICATION_WORK_ASSIGNER, SequentialWorkAssigner.class.getName());
+    MiniAccumuloClusterImpl peerCluster = peerCfg.build();
+
+    peerCluster.start();
+
+    final Connector connMaster = getConnector();
+    final Connector connPeer = peerCluster.getConnector("root", ROOT_PASSWORD);
+
+    ReplicationTable.create(connMaster);
+
+    String peerClusterName = "peer";
+
+    // ...peer = AccumuloReplicaSystem,instanceName,zookeepers
+    connMaster.instanceOperations().setProperty(
+        Property.REPLICATION_PEERS.getKey() + peerClusterName,
+        ReplicaSystemFactory.getPeerConfigurationValue(AccumuloReplicaSystem.class,
+            AccumuloReplicaSystem.buildConfiguration(peerCluster.getInstanceName(), peerCluster.getZooKeepers())));
+
+    final String masterTable = "master", peerTable = "peer";
+
+    connMaster.tableOperations().create(masterTable);
+    String masterTableId = connMaster.tableOperations().tableIdMap().get(masterTable);
+    Assert.assertNotNull(masterTableId);
+
+    connPeer.tableOperations().create(peerTable);
+    String peerTableId = connPeer.tableOperations().tableIdMap().get(peerTable);
+    Assert.assertNotNull(peerTableId);
+
+    // Replicate this table to the peerClusterName in a table with the peerTableId table id
+    connMaster.tableOperations().setProperty(masterTable, Property.TABLE_REPLICATION.getKey(), "true");
+    connMaster.tableOperations().setProperty(masterTable, Property.TABLE_REPLICATION_TARGETS.getKey() + peerClusterName, peerTableId);
+
+    // Write some data to table1
+    BatchWriter bw = connMaster.createBatchWriter(masterTable, new BatchWriterConfig());
+    for (int rows = 0; rows < 5000; rows++) {
+      Mutation m = new Mutation(Integer.toString(rows));
+      for (int cols = 0; cols < 100; cols++) {
+        String value = Integer.toString(cols);
+        m.put(value, "", value);
+      }
+      bw.addMutation(m);
+    }
+
+    bw.close();
+
+    log.info("Wrote all data to master cluster");
+
+    log.debug("");
+    for (Entry<Key,Value> kv : connMaster.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
+      if (ReplicationSection.COLF.equals(kv.getKey().getColumnFamily())) {
+        log.debug(kv.getKey().toStringNoTruncate() + " " + ProtobufUtil.toString(Status.parseFrom(kv.getValue().get())));
+      } else {
+        log.debug(kv.getKey().toStringNoTruncate() + " " + kv.getValue());
+      }
+    }
+
+    Future<Boolean> future = executor.submit(new Callable<Boolean>() {
+
+      @Override
+      public Boolean call() throws Exception {
+        connMaster.replicationOperations().drain(masterTable);
+        log.info("Drain completed");
+        return true;
+      }
+      
+    });
+
+    connMaster.tableOperations().compact(masterTable, null, null, true, true);
+
+    log.debug("");
+    log.info("Compaction completed");
+
+    log.debug("");
+    for (Entry<Key,Value> kv : connMaster.createScanner(MetadataTable.NAME, Authorizations.EMPTY)) {
+      if (ReplicationSection.COLF.equals(kv.getKey().getColumnFamily())) {
+        log.debug(kv.getKey().toStringNoTruncate() + " " + ProtobufUtil.toString(Status.parseFrom(kv.getValue().get())));
+      } else {
+        log.debug(kv.getKey().toStringNoTruncate() + " " + kv.getValue());
+      }
+    }
+
+    // We need to wait long enough for the records to make it from the metadata table to the replication table
+//    Thread.sleep(5000);
+    try {
+      future.get(15, TimeUnit.SECONDS);
+    } catch (TimeoutException e) {
+      Assert.fail("Drain did not finish within 5 seconds");
+    }
+
+    log.debug("");
+    for (Entry<Key,Value> kv : connMaster.createScanner(ReplicationTable.NAME, Authorizations.EMPTY)) {
+      log.debug(kv.getKey().toStringNoTruncate() + " " + ProtobufUtil.toString(Status.parseFrom(kv.getValue().get())));
+    }
+
+
+    Scanner master = connMaster.createScanner(masterTable, Authorizations.EMPTY), peer = connPeer.createScanner(peerTable, Authorizations.EMPTY);
+    Iterator<Entry<Key,Value>> masterIter = master.iterator(), peerIter = peer.iterator();
+    Entry<Key,Value> masterEntry = null, peerEntry = null;
+    while (masterIter.hasNext() && peerIter.hasNext()) {
+      masterEntry = masterIter.next();
+      peerEntry = peerIter.next();
+      Assert.assertEquals(masterEntry.getKey() + " was not equal to " + peerEntry.getKey(), 0,
+          masterEntry.getKey().compareTo(peerEntry.getKey(), PartialKey.ROW_COLFAM_COLQUAL_COLVIS));
+      Assert.assertEquals(masterEntry.getValue(), peerEntry.getValue());
+    }
+
+    log.info("Last master entry: " + masterEntry);
+    log.info("Last peer entry: " + peerEntry);
+    
+    Assert.assertFalse("Had more data to read from the master", masterIter.hasNext());
+    Assert.assertFalse("Had more data to read from the peer", peerIter.hasNext());
+
+    peerCluster.stop();
+  }
+
+  @Test(timeout = 60 * 5000)
+  public void dataReplicatedToCorrectTable() throws Exception {
+    MiniAccumuloConfigImpl peerCfg = new MiniAccumuloConfigImpl(createTestDir(this.getClass().getName() + "_" + this.testName.getMethodName() + "_peer"),
+        ROOT_PASSWORD);
+    peerCfg.setNumTservers(1);
+    peerCfg.setInstanceName("peer");
+    peerCfg.setProperty(Property.TSERV_WALOG_MAX_SIZE, "5M");
+    peerCfg.setProperty(Property.MASTER_REPLICATION_COORDINATOR_PORT, "10003");
+    peerCfg.setProperty(Property.REPLICATION_RECEIPT_SERVICE_PORT, "10004");
+    peerCfg.setProperty(Property.REPLICATION_WORK_ASSIGNMENT_SLEEP, "1s");
+    peerCfg.setProperty(Property.MASTER_REPLICATION_SCAN_INTERVAL, "1s");
+    peerCfg.setProperty(Property.REPLICATION_NAME, "peer");
+    peerCfg.setProperty(Property.REPLICATION_WORK_ASSIGNER, SequentialWorkAssigner.class.getName());
+    MiniAccumuloClusterImpl peer1Cluster = peerCfg.build();
+
+    peer1Cluster.start();
+
+    try {
+      Connector connMaster = getConnector();
+      Connector connPeer = peer1Cluster.getConnector("root", ROOT_PASSWORD);
+
+      String peerClusterName = "peer";
+
+      // ...peer = AccumuloReplicaSystem,instanceName,zookeepers
+      connMaster.instanceOperations().setProperty(
+          Property.REPLICATION_PEERS.getKey() + peerClusterName,
+          ReplicaSystemFactory.getPeerConfigurationValue(AccumuloReplicaSystem.class,
+              AccumuloReplicaSystem.buildConfiguration(peer1Cluster.getInstanceName(), peer1Cluster.getZooKeepers())));
+
+      String masterTable1 = "master1", peerTable1 = "peer1", masterTable2 = "master2", peerTable2 = "peer2";
+
+      connMaster.tableOperations().create(masterTable1);
+      String masterTableId1 = connMaster.tableOperations().tableIdMap().get(masterTable1);
+      Assert.assertNotNull(masterTableId1);
+
+      connMaster.tableOperations().create(masterTable2);
+      String masterTableId2 = connMaster.tableOperations().tableIdMap().get(masterTable2);
+      Assert.assertNotNull(masterTableId2);
+
+      connPeer.tableOperations().create(peerTable1);
+      String peerTableId1 = connPeer.tableOperations().tableIdMap().get(peerTable1);
+      Assert.assertNotNull(peerTableId1);
+
+      connPeer.tableOperations().create(peerTable2);
+      String peerTableId2 = connPeer.tableOperations().tableIdMap().get(peerTable2);
+      Assert.assertNotNull(peerTableId2);
+
+      // Replicate this table to the peerClusterName in a table with the peerTableId table id
+      connMaster.tableOperations().setProperty(masterTable1, Property.TABLE_REPLICATION.getKey(), "true");
+      connMaster.tableOperations().setProperty(masterTable1, Property.TABLE_REPLICATION_TARGETS.getKey() + peerClusterName, peerTableId1);
+
+      connMaster.tableOperations().setProperty(masterTable2, Property.TABLE_REPLICATION.getKey(), "true");
+      connMaster.tableOperations().setProperty(masterTable2, Property.TABLE_REPLICATION_TARGETS.getKey() + peerClusterName, peerTableId2);
+
+      // Write some data to table1
+      BatchWriter bw = connMaster.createBatchWriter(masterTable1, new BatchWriterConfig());
+      for (int rows = 0; rows < 2500; rows++) {
+        Mutation m = new Mutation(masterTable1 + rows);
+        for (int cols = 0; cols < 100; cols++) {
+          String value = Integer.toString(cols);
+          m.put(value, "", value);
+        }
+        bw.addMutation(m);
+      }
+
+      bw.close();
+
+      // Write some data to table2
+      bw = connMaster.createBatchWriter(masterTable2, new BatchWriterConfig());
+      for (int rows = 0; rows < 2500; rows++) {
+        Mutation m = new Mutation(masterTable2 + rows);
+        for (int cols = 0; cols < 100; cols++) {
+          String value = Integer.toString(cols);
+          m.put(value, "", value);
+        }
+        bw.addMutation(m);
+      }
+
+      bw.close();
+
+      log.info("Wrote all data to master cluster");
+
+      while (!connMaster.tableOperations().exists(ReplicationTable.NAME)) {
+        Thread.sleep(500);
+      }
+
+      connMaster.tableOperations().compact(masterTable1, null, null, true, false);
+      connMaster.tableOperations().compact(masterTable2, null, null, true, false);
+
+      // Wait until we fully replicated something
+      boolean fullyReplicated = false;
+      for (int i = 0; i < 10 && !fullyReplicated; i++) {
+        UtilWaitThread.sleep(2000);
+
+        Scanner s = ReplicationTable.getScanner(connMaster);
+        WorkSection.limit(s);
+        for (Entry<Key,Value> entry : s) {
+          Status status = Status.parseFrom(entry.getValue().get());
+          if (StatusUtil.isFullyReplicated(status)) {
+            fullyReplicated |= true;
+          }
+        }
+      }
+
+      Assert.assertNotEquals(0, fullyReplicated);
+
+      long countTable = 0l;
+      for (Entry<Key,Value> entry : connPeer.createScanner(peerTable1, Authorizations.EMPTY)) {
+        countTable++;
+        Assert.assertTrue("Found unexpected key-value" + entry.getKey().toStringNoTruncate() + " " + entry.getValue(), entry.getKey().getRow().toString()
+            .startsWith(masterTable1));
+      }
+
+      log.info("Found {} records in {}", countTable, peerTable1);
+      Assert.assertTrue(countTable > 0);
+
+      countTable = 0l;
+      for (Entry<Key,Value> entry : connPeer.createScanner(peerTable2, Authorizations.EMPTY)) {
+        countTable++;
+        Assert.assertTrue("Found unexpected key-value" + entry.getKey().toStringNoTruncate() + " " + entry.getValue(), entry.getKey().getRow().toString()
+            .startsWith(masterTable2));
+      }
+
+      log.info("Found {} records in {}", countTable, peerTable2);
+      Assert.assertTrue(countTable > 0);
+
+    } finally {
+      peer1Cluster.stop();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ec2d8ddc/test/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/test/src/test/resources/log4j.properties b/test/src/test/resources/log4j.properties
index dd382f7..7649abc 100644
--- a/test/src/test/resources/log4j.properties
+++ b/test/src/test/resources/log4j.properties
@@ -13,7 +13,7 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-log4j.rootLogger=DEBUG, CA
+log4j.rootLogger=INFO, CA
 log4j.appender.CA=org.apache.log4j.ConsoleAppender
 log4j.appender.CA.layout=org.apache.log4j.PatternLayout
 log4j.appender.CA.layout.ConversionPattern=%d{ISO8601} [%c{2}] %-5p: %m%n
@@ -35,4 +35,9 @@ log4j.logger.org.apache.accumulo.server.util.ReplicationTableUtil=TRACE
 log4j.logger.org.apache.accumulo.core.client.impl.TabletServerBatchReaderIterator=INFO
 log4j.logger.org.apache.accumulo.core.client.impl.ThriftScanner=INFO
 log4j.logger.org.apache.accumulo.server.zookeeper.DistributedWorkQueue=INFO
-log4j.logger.org.apache.accumulo.fate.zookeeper.DistributedReadWriteLock=WARN
\ No newline at end of file
+log4j.logger.org.apache.accumulo.fate.zookeeper.DistributedReadWriteLock=WARN
+log4j.logger.org.mortbay.log=WARN
+log4j.logger.org.apache.hadoop=WARN
+log4j.logger.org.apache.hadoop.hdfs.server.namenode.FSNamesystem.audit=WARN
+log4j.logger.org.apache.hadoop.hdfs.server.datanode.DataNode.clienttrace=WARN
+log4j.logger.BlockStateChange=WARN
\ No newline at end of file


[36/50] [abbrv] git commit: ACCUMULO-2583 Was comparing number of mutations to number of updates in each mutation.

Posted by el...@apache.org.
ACCUMULO-2583 Was comparing number of mutations to number of updates in each mutation.


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

Branch: refs/heads/ACCUMULO-378
Commit: d3cc1fee847b059de850102ccd3c65e66b7770bd
Parents: 9f77918
Author: Josh Elser <el...@apache.org>
Authored: Mon May 19 16:37:36 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Mon May 19 16:37:36 2014 -0400

----------------------------------------------------------------------
 .../accumulo/tserver/replication/AccumuloReplicaSystem.java  | 8 +++-----
 1 file changed, 3 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/d3cc1fee/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/AccumuloReplicaSystem.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/AccumuloReplicaSystem.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/AccumuloReplicaSystem.java
index ac6e1a0..64244c5 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/AccumuloReplicaSystem.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/AccumuloReplicaSystem.java
@@ -343,7 +343,6 @@ public class AccumuloReplicaSystem implements ReplicaSystem {
    */
   protected long writeValueAvoidingReplicationCycles(DataOutputStream out, LogFileValue value, ReplicationTarget target) throws IOException {
     int mutationsToSend = 0;
-    long numUpdates = 0l;
     for (Mutation m : value.mutations) {
       if (!m.getReplicationSources().contains(target.getPeerName())) {
         mutationsToSend++;
@@ -356,7 +355,6 @@ public class AccumuloReplicaSystem implements ReplicaSystem {
     for (Mutation m : value.mutations) {
       // If we haven't yet replicated to this peer
       if (!m.getReplicationSources().contains(target.getPeerName())) {
-        numUpdates += m.size();
         // Add our name, and send it
         String name = conf.get(Property.REPLICATION_NAME);
         if (StringUtils.isBlank(name)) {
@@ -369,7 +367,7 @@ public class AccumuloReplicaSystem implements ReplicaSystem {
       }
     }
 
-    return numUpdates;
+    return mutationsToSend;
   }
 
   public static class ReplicationStats {
@@ -421,10 +419,10 @@ public class AccumuloReplicaSystem implements ReplicaSystem {
      */
     public long numUpdates;
 
-    public WalReplication(WalEdits edits, long size, long entriesConsumed, long numUpdates) {
+    public WalReplication(WalEdits edits, long size, long entriesConsumed, long numMutations) {
       super(size, edits.getEditsSize(), entriesConsumed);
       this.walEdits = edits;
-      this.numUpdates = numUpdates;
+      this.numUpdates = numMutations;
     }
   }
 }


[14/50] [abbrv] git commit: ACCUMULO-2762 Use more compiler optimizations for 1.5 native lib

Posted by el...@apache.org.
ACCUMULO-2762 Use more compiler optimizations for 1.5 native lib


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

Branch: refs/heads/ACCUMULO-378
Commit: c16f105824ff5e4d56105b9b3cc20d866b0e3b8d
Parents: f83f9f9
Author: Christopher Tubbs <ct...@apache.org>
Authored: Fri May 16 12:55:41 2014 -0400
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Fri May 16 12:55:41 2014 -0400

----------------------------------------------------------------------
 server/src/main/c++/nativeMap/Makefile | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/c16f1058/server/src/main/c++/nativeMap/Makefile
----------------------------------------------------------------------
diff --git a/server/src/main/c++/nativeMap/Makefile b/server/src/main/c++/nativeMap/Makefile
index 2ff8ea5..8b3ef9a 100644
--- a/server/src/main/c++/nativeMap/Makefile
+++ b/server/src/main/c++/nativeMap/Makefile
@@ -39,7 +39,7 @@ ifeq ($(shell uname),Linux)
     LIBS := $(LIBS_$(DARCH))
   endif
   
-  CXXFLAGS=-g -fPIC -shared -O2 -fno-omit-frame-pointer -fno-strict-aliasing -Wall -I$(JAVA_HOME)/include/linux -I$(JAVA_HOME)/include
+  CXXFLAGS=-g -fPIC -shared -O3 -Wall -I$(JAVA_HOME)/include/linux -I$(JAVA_HOME)/include
 endif
 
 ifeq ($(shell uname),Darwin)


[22/50] [abbrv] git commit: ACCUMULO-2819 Initial pass at a new WorkAssigner that will only replicate the oldest file per table per peer

Posted by el...@apache.org.
ACCUMULO-2819 Initial pass at a new WorkAssigner that will only replicate the oldest file per table per peer

By tracking the order in which WALs were MinC'ed, we can use this order to determine
the practical order for which they should be replayed on each table of each peer. This
aims to remove the potential problem that the DWQWorkAssigner faced in that WAL2 might be
replicated before WAL1 (where WAL1 was written before WAL2 locally).


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

Branch: refs/heads/ACCUMULO-378
Commit: a59692dc62170e4bc3a47d7cf881be9ba6027220
Parents: 8921e32
Author: Josh Elser <el...@apache.org>
Authored: Fri May 16 18:13:22 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Fri May 16 18:13:22 2014 -0400

----------------------------------------------------------------------
 .../replication/AbstractWorkAssigner.java       |  81 +++++
 .../ReplicationWorkAssignerHelper.java          |  75 -----
 .../server/replication/WorkAssigner.java        |  42 +++
 .../ReplicationWorkAssignerHelperTest.java      |  56 ----
 .../master/replication/AbsractWorkAssigner.java |  30 --
 .../DistributedWorkQueueWorkAssigner.java       |   6 +-
 .../replication/SequentialWorkAssigner.java     | 323 ++++++++++++++++++-
 .../master/replication/WorkAssigner.java        |  42 ---
 .../accumulo/master/replication/WorkDriver.java |   1 +
 .../replication/AbstractWorkAssignerTest.java   |  56 ++++
 .../DistributedWorkQueueWorkAssignerTest.java   |  73 ++++-
 .../replication/SequentialWorkAssignerTest.java | 142 ++++++++
 .../replication/ReplicationProcessor.java       |   4 +-
 13 files changed, 714 insertions(+), 217 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/a59692dc/server/base/src/main/java/org/apache/accumulo/server/replication/AbstractWorkAssigner.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/replication/AbstractWorkAssigner.java b/server/base/src/main/java/org/apache/accumulo/server/replication/AbstractWorkAssigner.java
new file mode 100644
index 0000000..2a7b825
--- /dev/null
+++ b/server/base/src/main/java/org/apache/accumulo/server/replication/AbstractWorkAssigner.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.server.replication;
+
+import java.util.Map.Entry;
+
+import org.apache.accumulo.core.replication.ReplicationTarget;
+import org.apache.accumulo.core.replication.StatusUtil;
+import org.apache.accumulo.core.replication.proto.Replication.Status;
+import org.apache.accumulo.server.zookeeper.DistributedWorkQueue;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Maps;
+
+/**
+ * Common methods for {@link WorkAssigner}s
+ */
+public abstract class AbstractWorkAssigner implements WorkAssigner {
+
+  protected boolean isWorkRequired(Status status) {
+    return StatusUtil.isWorkRequired(status);
+  }
+  public static final String KEY_SEPARATOR = "|";
+
+  /**
+   * Serialize a filename and a {@link ReplicationTarget} into the expected key format for use with the {@link DistributedWorkQueue}
+   * 
+   * @param filename
+   *          Filename for data to be replicated
+   * @param replTarget
+   *          Information about replication peer
+   * @return Key for identifying work in queue
+   */
+  public static String getQueueKey(String filename, ReplicationTarget replTarget) {
+    return filename + KEY_SEPARATOR + replTarget.getPeerName() + KEY_SEPARATOR + replTarget.getRemoteIdentifier() + KEY_SEPARATOR
+        + replTarget.getSourceTableId();
+  }
+
+  /**
+   * @param queueKey
+   *          Key from the work queue
+   * @return Components which created the queue key
+   */
+  public static Entry<String,ReplicationTarget> fromQueueKey(String queueKey) {
+    Preconditions.checkNotNull(queueKey);
+
+    int index = queueKey.indexOf(KEY_SEPARATOR);
+    if (-1 == index) {
+      throw new IllegalArgumentException("Could not find expected separator in queue key '" + queueKey + "'");
+    }
+
+    String filename = queueKey.substring(0, index);
+
+    int secondIndex = queueKey.indexOf(KEY_SEPARATOR, index + 1);
+    if (-1 == secondIndex) {
+      throw new IllegalArgumentException("Could not find expected separator in queue key '" + queueKey + "'");
+    }
+
+    int thirdIndex = queueKey.indexOf(KEY_SEPARATOR, secondIndex + 1);
+    if (-1 == thirdIndex) {
+      throw new IllegalArgumentException("Could not find expected seperator in queue key '" + queueKey + "'");
+    }
+
+    return Maps.immutableEntry(filename, new ReplicationTarget(queueKey.substring(index + 1, secondIndex), queueKey.substring(secondIndex + 1, thirdIndex),
+        queueKey.substring(thirdIndex + 1)));
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a59692dc/server/base/src/main/java/org/apache/accumulo/server/replication/ReplicationWorkAssignerHelper.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/replication/ReplicationWorkAssignerHelper.java b/server/base/src/main/java/org/apache/accumulo/server/replication/ReplicationWorkAssignerHelper.java
deleted file mode 100644
index 13ed4a9..0000000
--- a/server/base/src/main/java/org/apache/accumulo/server/replication/ReplicationWorkAssignerHelper.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.server.replication;
-
-import java.util.Map.Entry;
-
-import org.apache.accumulo.core.replication.ReplicationTarget;
-import org.apache.accumulo.server.zookeeper.DistributedWorkQueue;
-
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Maps;
-
-/**
- * 
- */
-public class ReplicationWorkAssignerHelper {
-  public static final String KEY_SEPARATOR = "|";
-
-  /**
-   * Serialize a filename and a {@link ReplicationTarget} into the expected key format for use with the {@link DistributedWorkQueue}
-   * 
-   * @param filename
-   *          Filename for data to be replicated
-   * @param replTarget
-   *          Information about replication peer
-   * @return Key for identifying work in queue
-   */
-  public static String getQueueKey(String filename, ReplicationTarget replTarget) {
-    return filename + KEY_SEPARATOR + replTarget.getPeerName() + KEY_SEPARATOR + replTarget.getRemoteIdentifier() + KEY_SEPARATOR
-        + replTarget.getSourceTableId();
-  }
-
-  /**
-   * @param queueKey
-   *          Key from the work queue
-   * @return Components which created the queue key
-   */
-  public static Entry<String,ReplicationTarget> fromQueueKey(String queueKey) {
-    Preconditions.checkNotNull(queueKey);
-
-    int index = queueKey.indexOf(KEY_SEPARATOR);
-    if (-1 == index) {
-      throw new IllegalArgumentException("Could not find expected separator in queue key '" + queueKey + "'");
-    }
-
-    String filename = queueKey.substring(0, index);
-
-    int secondIndex = queueKey.indexOf(KEY_SEPARATOR, index + 1);
-    if (-1 == secondIndex) {
-      throw new IllegalArgumentException("Could not find expected separator in queue key '" + queueKey + "'");
-    }
-
-    int thirdIndex = queueKey.indexOf(KEY_SEPARATOR, secondIndex + 1);
-    if (-1 == thirdIndex) {
-      throw new IllegalArgumentException("Could not find expected seperator in queue key '" + queueKey + "'");
-    }
-
-    return Maps.immutableEntry(filename, new ReplicationTarget(queueKey.substring(index + 1, secondIndex), queueKey.substring(secondIndex + 1, thirdIndex),
-        queueKey.substring(thirdIndex + 1)));
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a59692dc/server/base/src/main/java/org/apache/accumulo/server/replication/WorkAssigner.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/replication/WorkAssigner.java b/server/base/src/main/java/org/apache/accumulo/server/replication/WorkAssigner.java
new file mode 100644
index 0000000..1069835
--- /dev/null
+++ b/server/base/src/main/java/org/apache/accumulo/server/replication/WorkAssigner.java
@@ -0,0 +1,42 @@
+/*
+ * 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.client.Connector;
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+
+
+/**
+ * Interface to allow for multiple implementations that assign replication work
+ */
+public interface WorkAssigner {
+
+  /**
+   * @return The name for this WorkAssigner
+   */
+  public String getName();
+
+  /**
+   * Configure the WorkAssigner implementation
+   */
+  public void configure(AccumuloConfiguration conf, Connector conn);
+
+  /**
+   * Assign work for replication
+   */
+  public void assignWork();
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a59692dc/server/base/src/test/java/org/apache/accumulo/server/replication/ReplicationWorkAssignerHelperTest.java
----------------------------------------------------------------------
diff --git a/server/base/src/test/java/org/apache/accumulo/server/replication/ReplicationWorkAssignerHelperTest.java b/server/base/src/test/java/org/apache/accumulo/server/replication/ReplicationWorkAssignerHelperTest.java
deleted file mode 100644
index 47a4a0d..0000000
--- a/server/base/src/test/java/org/apache/accumulo/server/replication/ReplicationWorkAssignerHelperTest.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.accumulo.server.replication;
-
-import java.util.Map.Entry;
-import java.util.UUID;
-
-import org.apache.accumulo.core.replication.ReplicationTarget;
-import org.apache.accumulo.server.replication.ReplicationWorkAssignerHelper;
-import org.apache.hadoop.fs.Path;
-import org.apache.zookeeper.common.PathUtils;
-import org.junit.Assert;
-import org.junit.Test;
-
-/**
- * 
- */
-public class ReplicationWorkAssignerHelperTest {
-
-  @Test
-  public void createsValidZKNodeName() {
-    Path p = new Path ("/accumulo/wals/tserver+port/" + UUID.randomUUID().toString());
-    ReplicationTarget target = new ReplicationTarget("cluster1", "table1", "1");
-
-    String key = ReplicationWorkAssignerHelper.getQueueKey(p.toString(), target);
-    
-    PathUtils.validatePath(key);
-  }
-
-  @Test
-  public void queueKeySerialization() {
-    Path p = new Path("/accumulo/wals/tserver+port/" + UUID.randomUUID().toString());
-    ReplicationTarget target = new ReplicationTarget("cluster1", "table1", "1");
-
-    String key = ReplicationWorkAssignerHelper.getQueueKey(p.toString(), target);
-
-    Entry<String,ReplicationTarget> result = ReplicationWorkAssignerHelper.fromQueueKey(key);
-    Assert.assertEquals(p.toString(), result.getKey());
-    Assert.assertEquals(target, result.getValue());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a59692dc/server/master/src/main/java/org/apache/accumulo/master/replication/AbsractWorkAssigner.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/replication/AbsractWorkAssigner.java b/server/master/src/main/java/org/apache/accumulo/master/replication/AbsractWorkAssigner.java
deleted file mode 100644
index eea6853..0000000
--- a/server/master/src/main/java/org/apache/accumulo/master/replication/AbsractWorkAssigner.java
+++ /dev/null
@@ -1,30 +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.master.replication;
-
-import org.apache.accumulo.core.replication.StatusUtil;
-import org.apache.accumulo.core.replication.proto.Replication.Status;
-
-/**
- * Common methods for {@link WorkAssigner}s
- */
-public abstract class AbsractWorkAssigner implements WorkAssigner {
-
-  protected boolean isWorkRequired(Status status) {
-    return StatusUtil.isWorkRequired(status);
-  }
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a59692dc/server/master/src/main/java/org/apache/accumulo/master/replication/DistributedWorkQueueWorkAssigner.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/replication/DistributedWorkQueueWorkAssigner.java b/server/master/src/main/java/org/apache/accumulo/master/replication/DistributedWorkQueueWorkAssigner.java
index a84e6a0..e97f3ca 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/replication/DistributedWorkQueueWorkAssigner.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/replication/DistributedWorkQueueWorkAssigner.java
@@ -37,8 +37,8 @@ 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.core.zookeeper.ZooUtil;
+import org.apache.accumulo.server.replication.AbstractWorkAssigner;
 import org.apache.accumulo.server.replication.ReplicationTable;
-import org.apache.accumulo.server.replication.ReplicationWorkAssignerHelper;
 import org.apache.accumulo.server.zookeeper.DistributedWorkQueue;
 import org.apache.accumulo.server.zookeeper.ZooCache;
 import org.apache.hadoop.fs.Path;
@@ -61,7 +61,7 @@ import com.google.protobuf.TextFormat;
  * peer in a different order than the master. The {@link SequentialWorkAssigner} should be used if this must be guaranteed at the cost of replication
  * throughput.
  */
-public class DistributedWorkQueueWorkAssigner extends AbsractWorkAssigner {
+public class DistributedWorkQueueWorkAssigner extends AbstractWorkAssigner {
   private static final Logger log = LoggerFactory.getLogger(DistributedWorkQueueWorkAssigner.class);
   private static final String NAME = "DistributedWorkQueue Replication Work Assigner";
 
@@ -230,7 +230,7 @@ public class DistributedWorkQueueWorkAssigner extends AbsractWorkAssigner {
           Path p = new Path(file);
           String filename = p.getName();
           WorkSection.getTarget(entry.getKey(), buffer);
-          String key = ReplicationWorkAssignerHelper.getQueueKey(filename, ReplicationTarget.from(buffer));
+          String key = getQueueKey(filename, ReplicationTarget.from(buffer));
 
           // And, we haven't already queued this file up for work already
           if (!queuedWork.contains(key)) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a59692dc/server/master/src/main/java/org/apache/accumulo/master/replication/SequentialWorkAssigner.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/replication/SequentialWorkAssigner.java b/server/master/src/main/java/org/apache/accumulo/master/replication/SequentialWorkAssigner.java
index 8058a2b..e295ef7 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/replication/SequentialWorkAssigner.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/replication/SequentialWorkAssigner.java
@@ -16,29 +16,336 @@
  */
 package org.apache.accumulo.master.replication;
 
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.core.protobuf.ProtobufUtil;
+import org.apache.accumulo.core.replication.ReplicationSchema.OrderSection;
+import org.apache.accumulo.core.replication.ReplicationSchema.WorkSection;
+import org.apache.accumulo.core.replication.ReplicationTarget;
+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.core.zookeeper.ZooUtil;
+import org.apache.accumulo.server.replication.AbstractWorkAssigner;
+import org.apache.accumulo.server.replication.ReplicationTable;
+import org.apache.accumulo.server.zookeeper.DistributedWorkQueue;
+import org.apache.accumulo.server.zookeeper.ZooCache;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Preconditions;
+import com.google.protobuf.InvalidProtocolBufferException;
 
 /**
- * 
+ * Creates work in ZK which is <code>filename.serialized_ReplicationTarget => filename</code>
  */
-public class SequentialWorkAssigner extends AbsractWorkAssigner {
+public class SequentialWorkAssigner extends AbstractWorkAssigner {
+  private static final Logger log = LoggerFactory.getLogger(SequentialWorkAssigner.class);
+  private static final String NAME = "Sequential Work Assigner";
 
-  @Override
-  public String getName() {
-    // TODO Auto-generated method stub
-    return null;
+  private Connector conn;
+  private AccumuloConfiguration conf;
+
+  // @formatter.off
+  /*
+   * { 
+   *    peer1 => {sourceTableId1 => work_queue_key1, sourceTableId2 => work_queue_key2, ...}
+   *    peer2 => {sourceTableId1 => work_queue_key1, sourceTableId3 => work_queue_key4, ...}
+   *    ...
+   * }
+   */
+  // @formatter.on
+  private Map<String,Map<String,String>> queuedWorkByPeerName;
+
+  private DistributedWorkQueue workQueue;
+  private int maxQueueSize;
+  private ZooCache zooCache;
+
+  public SequentialWorkAssigner() {}
+
+  public SequentialWorkAssigner(AccumuloConfiguration conf, Connector conn) {
+    this.conf = conf;
+    this.conn = conn;
   }
 
   @Override
   public void configure(AccumuloConfiguration conf, Connector conn) {
-    // TODO Auto-generated method stub
+    this.conf = conf;
+    this.conn = conn;
+  }
+
+  @Override
+  public String getName() {
+    return NAME;
   }
 
   @Override
   public void assignWork() {
-    // TODO Auto-generated method stub
+    if (null == workQueue) {
+      initializeWorkQueue(conf);
+    }
+
+    if (null == queuedWorkByPeerName) {
+      initializeQueuedWork();
+    }
+
+    if (null == zooCache) {
+      zooCache = new ZooCache();
+    }
+
+    // Get the maximum number of entries we want to queue work for (or the default)
+    this.maxQueueSize = conf.getCount(Property.REPLICATION_MAX_WORK_QUEUE);
+
+    // Scan over the work records, adding the work to the queue
+    createWork();
+
+    // Keep the state of the work we queued correct
+    cleanupFinishedWork();
+  }
+
+  /*
+   * Getters/setters for testing purposes
+   */
+  protected Connector getConnector() {
+    return conn;
+  }
+
+  protected void setConnector(Connector conn) {
+    this.conn = conn;
+  }
+
+  protected AccumuloConfiguration getConf() {
+    return conf;
+  }
+
+  protected void setConf(AccumuloConfiguration conf) {
+    this.conf = conf;
+  }
+
+  protected DistributedWorkQueue getWorkQueue() {
+    return workQueue;
+  }
+
+  protected void setWorkQueue(DistributedWorkQueue workQueue) {
+    this.workQueue = workQueue;
+  }
+
+  protected Map<String,Map<String,String>> getQueuedWork() {
+    return queuedWorkByPeerName;
+  }
+
+  protected void setQueuedWork(Map<String,Map<String,String>> queuedWork) {
+    this.queuedWorkByPeerName = queuedWork;
+  }
+
+  protected int getMaxQueueSize() {
+    return maxQueueSize;
+  }
 
+  protected void setMaxQueueSize(int maxQueueSize) {
+    this.maxQueueSize = maxQueueSize;
   }
 
+  protected ZooCache getZooCache() {
+    return zooCache;
+  }
+
+  protected void setZooCache(ZooCache zooCache) {
+    this.zooCache = zooCache;
+  }
+
+  /**
+   * Initialize the DistributedWorkQueue using the proper ZK location
+   * 
+   * @param conf
+   */
+  protected void initializeWorkQueue(AccumuloConfiguration conf) {
+    workQueue = new DistributedWorkQueue(ZooUtil.getRoot(conn.getInstance()) + Constants.ZREPLICATION_WORK_QUEUE, conf);
+  }
+
+  /**
+   * Initialize the queuedWork set with the work already sent out
+   */
+  protected void initializeQueuedWork() {
+    Preconditions.checkArgument(null == queuedWorkByPeerName, "Expected queuedWork to be null");
+    queuedWorkByPeerName = new HashMap<>();
+    List<String> existingWork;
+    try {
+      existingWork = workQueue.getWorkQueued();
+    } catch (KeeperException | InterruptedException e) {
+      throw new RuntimeException("Error reading existing queued replication work", e);
+    }
+
+    log.info("Restoring replication work queue state from zookeeper");
+
+    for (String work : existingWork) {
+      Entry<String,ReplicationTarget> entry = fromQueueKey(work);
+      String filename = entry.getKey();
+      String peerName = entry.getValue().getPeerName();
+      String sourceTableId = entry.getValue().getSourceTableId();
+
+      log.debug("In progress replication of {} from table with ID {} to peer {}", filename, sourceTableId, peerName);
+
+      Map<String,String> replicationForPeer = queuedWorkByPeerName.get(peerName);
+      if (null == replicationForPeer) {
+        replicationForPeer = new HashMap<>();
+        queuedWorkByPeerName.put(sourceTableId, replicationForPeer);
+      }
+
+      replicationForPeer.put(sourceTableId, work);
+    }
+  }
+
+  /**
+   * Scan over the {@link WorkSection} of the replication table adding work for entries that have data to replicate and have not already been queued.
+   */
+  protected void createWork() {
+    // Create a scanner over the replication table's order entries
+    Scanner s;
+    try {
+      s = ReplicationTable.getScanner(conn);
+    } catch (TableNotFoundException e) {
+      UtilWaitThread.sleep(1000);
+      return;
+    }
+
+    OrderSection.limit(s);
+
+    Text buffer = new Text();
+    for (Entry<Key,Value> entry : s) {
+      // If we're not working off the entries, we need to not shoot ourselves in the foot by continuing
+      // to add more work entries
+      if (queuedWorkByPeerName.size() > maxQueueSize) {
+        log.warn("Queued replication work exceeds configured maximum ({}), sleeping to allow work to occur", maxQueueSize);
+        UtilWaitThread.sleep(5000);
+        return;
+      }
+
+      String file = OrderSection.getFile(entry.getKey(), buffer);
+      OrderSection.getTableId(entry.getKey(), buffer);
+      String sourceTableId = buffer.toString();
+
+      Scanner workScanner;
+      try {
+        workScanner = ReplicationTable.getScanner(conn);
+      } catch (TableNotFoundException e) {
+        log.warn("Replication table was deleted. Will retry...");
+        UtilWaitThread.sleep(5000);
+        return;
+      }
+
+      WorkSection.limit(workScanner);
+      workScanner.setRange(Range.exact(file));
+
+      int newReplicationTasksSubmitted = 0;
+      // For a file, we can concurrently replicate it to multiple targets
+      for (Entry<Key,Value> workEntry : workScanner) {
+        Status status;
+        try {
+          status = StatusUtil.fromValue(workEntry.getValue());
+        } catch (InvalidProtocolBufferException e) {
+          log.warn("Could not deserialize protobuf from work entry for {} to {}, will retry", file,
+              ReplicationTarget.from(workEntry.getKey().getColumnQualifier()), e);
+          continue;
+        }
+
+        // Get the ReplicationTarget for this Work record
+        ReplicationTarget target = WorkSection.getTarget(entry.getKey(), buffer);
+
+        Map<String,String> queuedWorkForPeer = queuedWorkByPeerName.get(target.getPeerName());
+        if (null == queuedWorkForPeer) {
+          queuedWorkForPeer = new HashMap<>();
+          queuedWorkByPeerName.put(target.getPeerName(), queuedWorkForPeer);
+        }
+
+        // If there is work to do
+        if (isWorkRequired(status)) {
+          Path p = new Path(file);
+          String filename = p.getName();
+          String key = getQueueKey(filename, target);
+
+          // Get the file (if any) currently being replicated to the given peer for the given source table
+          String fileBeingReplicated = queuedWorkForPeer.get(sourceTableId);
+
+          if (null == fileBeingReplicated) {
+            // If there is no file, submit this one for replication
+            newReplicationTasksSubmitted += queueWork(key, file, sourceTableId, queuedWorkForPeer);
+          } else {
+            log.debug("Not queueing {} for work as {} must be replicated to {} first", file, fileBeingReplicated, target.getPeerName());
+          }
+        } else {
+          log.debug("Not queueing work for {} because [{}] doesn't need replication", file, ProtobufUtil.toString(status));
+        }
+      }
+
+      log.info("Assigned {} replication work entries for {}", newReplicationTasksSubmitted, file);
+    }
+  }
+
+  /**
+   * Distribute the work for the given path with filename
+   * 
+   * @param key
+   *          Unique key to identify this work in the queue
+   * @param path
+   *          Full path to a file
+   */
+  protected int queueWork(String key, String path, String sourceTableId, Map<String,String> queuedWorkForPeer) {
+    try {
+      log.debug("Queued work for {} and {} from table ID {}", key, path, sourceTableId);
+
+      workQueue.addWork(key, path);      
+      queuedWorkForPeer.put(sourceTableId, key);
+
+      return 1;
+    } catch (KeeperException | InterruptedException e) {
+      log.warn("Could not queue work for {}", path, e);
+      return 0;
+    }
+  }
+
+  /**
+   * Iterate over the queued work to remove entries that have been completed.
+   */
+  protected void cleanupFinishedWork() {
+    final Iterator<Entry<String,Map<String,String>>> queuedWork = queuedWorkByPeerName.entrySet().iterator();
+    final String instanceId = conn.getInstance().getInstanceID();
+
+    // Check the status of all the work we've queued up
+    while (queuedWork.hasNext()) {
+      // {peer -> {tableId -> workKey, tableId -> workKey, ... }, peer -> ...}
+      Entry<String,Map<String,String>> workForPeer = queuedWork.next();
+
+      // TableID to workKey (filename and ReplicationTarget)
+      Map<String,String> queuedReplication = workForPeer.getValue();
+
+      Iterator<Entry<String,String>> iter = queuedReplication.entrySet().iterator();
+      // Loop over every target we need to replicate this file to, removing the target when
+      // the replication task has finished
+      while (iter.hasNext()) {
+        // tableID -> workKey
+        Entry<String,String> entry = iter.next();
+        // Null equates to the work for this target was finished
+        if (null == zooCache.get(ZooUtil.getRoot(instanceId) + Constants.ZREPLICATION_WORK_QUEUE + "/" + entry.getValue())) {
+          iter.remove();
+        }
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a59692dc/server/master/src/main/java/org/apache/accumulo/master/replication/WorkAssigner.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/replication/WorkAssigner.java b/server/master/src/main/java/org/apache/accumulo/master/replication/WorkAssigner.java
deleted file mode 100644
index 9e47fc3..0000000
--- a/server/master/src/main/java/org/apache/accumulo/master/replication/WorkAssigner.java
+++ /dev/null
@@ -1,42 +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.master.replication;
-
-import org.apache.accumulo.core.client.Connector;
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-
-
-/**
- * Interface to allow for multiple implementations that assign replication work
- */
-public interface WorkAssigner {
-
-  /**
-   * @return The name for this WorkAssigner
-   */
-  public String getName();
-
-  /**
-   * Configure the WorkAssigner implementation
-   */
-  public void configure(AccumuloConfiguration conf, Connector conn);
-
-  /**
-   * Assign work for replication
-   */
-  public void assignWork();
-}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a59692dc/server/master/src/main/java/org/apache/accumulo/master/replication/WorkDriver.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/replication/WorkDriver.java b/server/master/src/main/java/org/apache/accumulo/master/replication/WorkDriver.java
index bbe8f93..1b70a13 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/replication/WorkDriver.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/replication/WorkDriver.java
@@ -22,6 +22,7 @@ import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.util.Daemon;
 import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.accumulo.master.Master;
+import org.apache.accumulo.server.replication.WorkAssigner;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a59692dc/server/master/src/test/java/org/apache/accumulo/master/replication/AbstractWorkAssignerTest.java
----------------------------------------------------------------------
diff --git a/server/master/src/test/java/org/apache/accumulo/master/replication/AbstractWorkAssignerTest.java b/server/master/src/test/java/org/apache/accumulo/master/replication/AbstractWorkAssignerTest.java
new file mode 100644
index 0000000..f4e60e5
--- /dev/null
+++ b/server/master/src/test/java/org/apache/accumulo/master/replication/AbstractWorkAssignerTest.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.master.replication;
+
+import java.util.Map.Entry;
+import java.util.UUID;
+
+import org.apache.accumulo.core.replication.ReplicationTarget;
+import org.apache.accumulo.server.replication.AbstractWorkAssigner;
+import org.apache.hadoop.fs.Path;
+import org.apache.zookeeper.common.PathUtils;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * 
+ */
+public class AbstractWorkAssignerTest {
+
+  @Test
+  public void createsValidZKNodeName() {
+    Path p = new Path ("/accumulo/wals/tserver+port/" + UUID.randomUUID().toString());
+    ReplicationTarget target = new ReplicationTarget("cluster1", "table1", "1");
+
+    String key = AbstractWorkAssigner.getQueueKey(p.toString(), target);
+    
+    PathUtils.validatePath(key);
+  }
+
+  @Test
+  public void queueKeySerialization() {
+    Path p = new Path("/accumulo/wals/tserver+port/" + UUID.randomUUID().toString());
+    ReplicationTarget target = new ReplicationTarget("cluster1", "table1", "1");
+
+    String key = AbstractWorkAssigner.getQueueKey(p.toString(), target);
+
+    Entry<String,ReplicationTarget> result = AbstractWorkAssigner.fromQueueKey(key);
+    Assert.assertEquals(p.toString(), result.getKey());
+    Assert.assertEquals(target, result.getValue());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a59692dc/server/master/src/test/java/org/apache/accumulo/master/replication/DistributedWorkQueueWorkAssignerTest.java
----------------------------------------------------------------------
diff --git a/server/master/src/test/java/org/apache/accumulo/master/replication/DistributedWorkQueueWorkAssignerTest.java b/server/master/src/test/java/org/apache/accumulo/master/replication/DistributedWorkQueueWorkAssignerTest.java
index 450b426..2048195 100644
--- a/server/master/src/test/java/org/apache/accumulo/master/replication/DistributedWorkQueueWorkAssignerTest.java
+++ b/server/master/src/test/java/org/apache/accumulo/master/replication/DistributedWorkQueueWorkAssignerTest.java
@@ -37,13 +37,15 @@ import org.apache.accumulo.core.client.mock.MockInstance;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.protobuf.ProtobufUtil;
+import org.apache.accumulo.core.replication.ReplicationSchema.OrderSection;
 import org.apache.accumulo.core.replication.ReplicationSchema.WorkSection;
 import org.apache.accumulo.core.replication.ReplicationTarget;
 import org.apache.accumulo.core.replication.StatusUtil;
+import org.apache.accumulo.core.replication.proto.Replication.Status;
 import org.apache.accumulo.core.security.Credentials;
 import org.apache.accumulo.core.security.TablePermission;
 import org.apache.accumulo.server.replication.ReplicationTable;
-import org.apache.accumulo.server.replication.ReplicationWorkAssignerHelper;
 import org.apache.accumulo.server.zookeeper.DistributedWorkQueue;
 import org.apache.accumulo.server.zookeeper.ZooCache;
 import org.apache.hadoop.fs.Path;
@@ -299,4 +301,73 @@ public class DistributedWorkQueueWorkAssignerTest {
 
     verify(workQueue);
   }
+
+  @Test
+  public void createWorkForFilesInCorrectOrder() throws Exception {
+    ReplicationTarget target = new ReplicationTarget("cluster1", "table1", "1");
+    Text serializedTarget = target.toText();
+    String keyTarget = target.getPeerName() + ReplicationWorkAssignerHelper.KEY_SEPARATOR + target.getRemoteIdentifier()
+        + ReplicationWorkAssignerHelper.KEY_SEPARATOR + target.getSourceTableId();
+
+    MockInstance inst = new MockInstance(test.getMethodName());
+    Credentials creds = new Credentials("root", new PasswordToken(""));
+    Connector conn = inst.getConnector(creds.getPrincipal(), creds.getToken());
+
+    // Set the connector
+    assigner.setConnector(conn);
+
+    // Create and grant ourselves write to the replication table
+    ReplicationTable.create(conn);
+    conn.securityOperations().grantTablePermission("root", ReplicationTable.NAME, TablePermission.WRITE);
+
+    // Create two mutations, both of which need replication work done
+    BatchWriter bw = ReplicationTable.getBatchWriter(conn);
+    // We want the name of file2 to sort before file1
+    String filename1 = "z_file1", filename2 = "a_file1";
+    String file1 = "/accumulo/wal/tserver+port/" + filename1, file2 = "/accumulo/wal/tserver+port/" + filename2;
+
+    // File1 was closed before file2, however
+    Status stat1 = Status.newBuilder().setBegin(0).setEnd(100).setClosed(true).setInfiniteEnd(false).setClosedTime(250).build();
+    Status stat2 = Status.newBuilder().setBegin(0).setEnd(100).setClosed(true).setInfiniteEnd(false).setClosedTime(500).build();
+
+    Mutation m = new Mutation(file1);
+    WorkSection.add(m, serializedTarget, ProtobufUtil.toValue(stat1));
+    bw.addMutation(m);
+
+    m = new Mutation(file2);
+    WorkSection.add(m, serializedTarget, ProtobufUtil.toValue(stat2));
+    bw.addMutation(m);
+
+    m = OrderSection.createMutation(file1, stat1.getClosedTime());
+    OrderSection.add(m, new Text(target.getSourceTableId()), ProtobufUtil.toValue(stat1));
+    bw.addMutation(m);
+
+    m = OrderSection.createMutation(file2, stat2.getClosedTime());
+    OrderSection.add(m, new Text(target.getSourceTableId()), ProtobufUtil.toValue(stat2));
+    bw.addMutation(m);
+
+    bw.close();
+
+    DistributedWorkQueue workQueue = createMock(DistributedWorkQueue.class);
+    @SuppressWarnings("unchecked")
+    HashSet<String> queuedWork = createMock(HashSet.class);
+    assigner.setQueuedWork(queuedWork);
+    assigner.setWorkQueue(workQueue);
+    assigner.setMaxQueueSize(Integer.MAX_VALUE);
+
+    expect(queuedWork.size()).andReturn(0).anyTimes();
+
+    // Make sure we expect the invocations in the correct order (accumulo is sorted)
+    expect(queuedWork.contains(filename1 + "|" + keyTarget)).andReturn(false);
+    workQueue.addWork(filename1 + "|" + keyTarget, file1);
+    expectLastCall().once();
+
+    // file2 is *not* queued because file1 must be replicated first
+
+    replay(queuedWork, workQueue);
+
+    assigner.createWork();
+
+    verify(queuedWork, workQueue);
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a59692dc/server/master/src/test/java/org/apache/accumulo/master/replication/SequentialWorkAssignerTest.java
----------------------------------------------------------------------
diff --git a/server/master/src/test/java/org/apache/accumulo/master/replication/SequentialWorkAssignerTest.java b/server/master/src/test/java/org/apache/accumulo/master/replication/SequentialWorkAssignerTest.java
new file mode 100644
index 0000000..b7c6e83
--- /dev/null
+++ b/server/master/src/test/java/org/apache/accumulo/master/replication/SequentialWorkAssignerTest.java
@@ -0,0 +1,142 @@
+/*
+ * 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.replication;
+
+import static org.easymock.EasyMock.createMock;
+import static org.easymock.EasyMock.expect;
+import static org.easymock.EasyMock.expectLastCall;
+import static org.easymock.EasyMock.replay;
+import static org.easymock.EasyMock.verify;
+import static org.junit.Assert.fail;
+
+import java.util.HashSet;
+
+import org.apache.accumulo.core.client.BatchWriter;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.mock.MockInstance;
+import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.data.Mutation;
+import org.apache.accumulo.core.protobuf.ProtobufUtil;
+import org.apache.accumulo.core.replication.ReplicationSchema.OrderSection;
+import org.apache.accumulo.core.replication.ReplicationSchema.WorkSection;
+import org.apache.accumulo.core.replication.ReplicationTarget;
+import org.apache.accumulo.core.replication.proto.Replication.Status;
+import org.apache.accumulo.core.security.Credentials;
+import org.apache.accumulo.core.security.TablePermission;
+import org.apache.accumulo.server.replication.AbstractWorkAssigner;
+import org.apache.accumulo.server.replication.ReplicationTable;
+import org.apache.accumulo.server.zookeeper.DistributedWorkQueue;
+import org.apache.hadoop.io.Text;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+
+/**
+ * 
+ */
+public class SequentialWorkAssignerTest {
+
+  @Rule
+  public TestName test = new TestName();
+
+  private AccumuloConfiguration conf;
+  private Connector conn;
+  private SequentialWorkAssigner assigner;
+
+  @Before
+  public void init() {
+    conf = createMock(AccumuloConfiguration.class);
+    conn = createMock(Connector.class);
+    assigner = new SequentialWorkAssigner(conf, conn);
+  }
+
+  @Test
+  public void test() {
+    fail("Not yet implemented");
+  }
+
+//  @Test
+  public void createWorkForFilesInCorrectOrder() throws Exception {
+    ReplicationTarget target = new ReplicationTarget("cluster1", "table1", "1");
+    Text serializedTarget = target.toText();
+    String keyTarget = target.getPeerName() + AbstractWorkAssigner.KEY_SEPARATOR + target.getRemoteIdentifier()
+        + AbstractWorkAssigner.KEY_SEPARATOR + target.getSourceTableId();
+
+    MockInstance inst = new MockInstance(test.getMethodName());
+    Credentials creds = new Credentials("root", new PasswordToken(""));
+    Connector conn = inst.getConnector(creds.getPrincipal(), creds.getToken());
+
+    // Set the connector
+    assigner.setConnector(conn);
+
+    // Create and grant ourselves write to the replication table
+    ReplicationTable.create(conn);
+    conn.securityOperations().grantTablePermission("root", ReplicationTable.NAME, TablePermission.WRITE);
+
+    // Create two mutations, both of which need replication work done
+    BatchWriter bw = ReplicationTable.getBatchWriter(conn);
+    // We want the name of file2 to sort before file1
+    String filename1 = "z_file1", filename2 = "a_file1";
+    String file1 = "/accumulo/wal/tserver+port/" + filename1, file2 = "/accumulo/wal/tserver+port/" + filename2;
+
+    // File1 was closed before file2, however
+    Status stat1 = Status.newBuilder().setBegin(0).setEnd(100).setClosed(true).setInfiniteEnd(false).setClosedTime(250).build();
+    Status stat2 = Status.newBuilder().setBegin(0).setEnd(100).setClosed(true).setInfiniteEnd(false).setClosedTime(500).build();
+
+    Mutation m = new Mutation(file1);
+    WorkSection.add(m, serializedTarget, ProtobufUtil.toValue(stat1));
+    bw.addMutation(m);
+
+    m = new Mutation(file2);
+    WorkSection.add(m, serializedTarget, ProtobufUtil.toValue(stat2));
+    bw.addMutation(m);
+
+    m = OrderSection.createMutation(file1, stat1.getClosedTime());
+    OrderSection.add(m, new Text(target.getSourceTableId()), ProtobufUtil.toValue(stat1));
+    bw.addMutation(m);
+
+    m = OrderSection.createMutation(file2, stat2.getClosedTime());
+    OrderSection.add(m, new Text(target.getSourceTableId()), ProtobufUtil.toValue(stat2));
+    bw.addMutation(m);
+
+    bw.close();
+
+    DistributedWorkQueue workQueue = createMock(DistributedWorkQueue.class);
+    @SuppressWarnings("unchecked")
+    HashSet<String> queuedWork = createMock(HashSet.class);
+    assigner.setQueuedWork(queuedWork);
+    assigner.setWorkQueue(workQueue);
+    assigner.setMaxQueueSize(Integer.MAX_VALUE);
+
+    expect(queuedWork.size()).andReturn(0).anyTimes();
+
+    // Make sure we expect the invocations in the correct order (accumulo is sorted)
+    expect(queuedWork.contains(filename1 + "|" + keyTarget)).andReturn(false);
+    workQueue.addWork(filename1 + "|" + keyTarget, file1);
+    expectLastCall().once();
+
+    // file2 is *not* queued because file1 must be replicated first
+
+    replay(queuedWork, workQueue);
+
+    assigner.createWork();
+
+    verify(queuedWork, workQueue);
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/a59692dc/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/ReplicationProcessor.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/ReplicationProcessor.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/ReplicationProcessor.java
index 3a0da79..2b8496c 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/ReplicationProcessor.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/ReplicationProcessor.java
@@ -40,8 +40,8 @@ import org.apache.accumulo.core.replication.proto.Replication.Status;
 import org.apache.accumulo.core.security.Credentials;
 import org.apache.accumulo.server.conf.ServerConfiguration;
 import org.apache.accumulo.server.fs.VolumeManager;
+import org.apache.accumulo.server.replication.AbstractWorkAssigner;
 import org.apache.accumulo.server.replication.ReplicationTable;
-import org.apache.accumulo.server.replication.ReplicationWorkAssignerHelper;
 import org.apache.accumulo.server.zookeeper.DistributedWorkQueue.Processor;
 import org.apache.hadoop.fs.Path;
 import org.slf4j.Logger;
@@ -76,7 +76,7 @@ public class ReplicationProcessor implements Processor {
 
   @Override
   public void process(String workID, byte[] data) {
-    ReplicationTarget target = ReplicationWorkAssignerHelper.fromQueueKey(workID).getValue();
+    ReplicationTarget target = AbstractWorkAssigner.fromQueueKey(workID).getValue();
     String file = new String(data);
 
     log.debug("Received replication work for {} to {}", file, target);


[37/50] [abbrv] git commit: ACCUMULO-378 Try to get some test stabilization

Posted by el...@apache.org.
ACCUMULO-378 Try to get some test stabilization


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

Branch: refs/heads/ACCUMULO-378
Commit: 8df4f41b5d725da7b5ef3c0de8f7219b98085be0
Parents: d3cc1fe
Author: Josh Elser <el...@apache.org>
Authored: Mon May 19 17:47:50 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Mon May 19 17:47:50 2014 -0400

----------------------------------------------------------------------
 .../client/impl/ReplicationOperationsImpl.java  | 18 +++++----
 .../master/replication/FinishedWorkUpdater.java |  6 ++-
 .../RemoveCompleteReplicationRecords.java       | 11 +++++-
 .../org/apache/accumulo/tserver/Tablet.java     |  2 +-
 .../replication/ReplicationSequentialIT.java    | 41 +++++++++++---------
 5 files changed, 48 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/8df4f41b/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 752952d..8ee09cb 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
@@ -193,21 +193,23 @@ public class ReplicationOperationsImpl implements ReplicationOperations {
         entry.getKey().getRow(rowHolder);
         entry.getKey().getColumnFamily(colfHolder);
 
-        String row;
+        String file;
         if (colfHolder.equals(ReplicationSection.COLF)) {
-          row = rowHolder.toString();
-          row = row.substring(ReplicationSection.getRowPrefix().length());
+          file = rowHolder.toString();
+          file = file.substring(ReplicationSection.getRowPrefix().length());
         } else if (colfHolder.equals(OrderSection.NAME)) {
-          row = OrderSection.getFile(entry.getKey(), rowHolder);
+          file = OrderSection.getFile(entry.getKey(), rowHolder);
+          long timeClosed = OrderSection.getTimeClosed(entry.getKey(), rowHolder);
+          log.debug("Order section: {} and {}", timeClosed, file);
         } else {
-          row = rowHolder.toString();
+          file = rowHolder.toString();
         }
 
-        log.debug("Processing {}", row);
+        log.debug("Evaluating if {} is still needed", file);
 
         // Skip files that we didn't observe when we started (new files/data)
-        if (!relevantLogs.contains(row)) {
-          log.debug("Found file that we didn't care about {}", row);
+        if (!relevantLogs.contains(file)) {
+          log.debug("Found file that we didn't care about {}", file);
           continue;
         }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8df4f41b/server/master/src/main/java/org/apache/accumulo/master/replication/FinishedWorkUpdater.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/replication/FinishedWorkUpdater.java b/server/master/src/main/java/org/apache/accumulo/master/replication/FinishedWorkUpdater.java
index 68bab64..5e0d726 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/replication/FinishedWorkUpdater.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/replication/FinishedWorkUpdater.java
@@ -94,7 +94,7 @@ public class FinishedWorkUpdater implements Runnable {
           continue;
         }
 
-        log.debug("Processing work progress from {}", serializedRow.getKey().getRow());
+        log.debug("Processing work progress for {}", serializedRow.getKey().getRow());
 
         Map<String,Long> tableIdToProgress = new HashMap<>();
         boolean error = false;
@@ -124,6 +124,10 @@ public class FinishedWorkUpdater implements Runnable {
           tableIdToProgress.put(target.getSourceTableId(), Math.min(tableIdToProgress.get(target.getSourceTableId()), status.getBegin()));
         }
 
+        for (Entry<String,Long> progressByTable : tableIdToProgress.entrySet()) {
+          log.debug("For {}, source table ID {} has replicated through {}", serializedRow.getKey().getRow(), progressByTable.getKey(), progressByTable.getValue());
+        }
+
         if (error) {
           continue;
         }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8df4f41b/server/master/src/main/java/org/apache/accumulo/master/replication/RemoveCompleteReplicationRecords.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/replication/RemoveCompleteReplicationRecords.java b/server/master/src/main/java/org/apache/accumulo/master/replication/RemoveCompleteReplicationRecords.java
index e661174..35ce374 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/replication/RemoveCompleteReplicationRecords.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/replication/RemoveCompleteReplicationRecords.java
@@ -33,7 +33,6 @@ 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.iterators.user.WholeRowIterator;
-import org.apache.accumulo.core.protobuf.ProtobufUtil;
 import org.apache.accumulo.core.replication.ReplicationSchema.OrderSection;
 import org.apache.accumulo.core.replication.ReplicationSchema.StatusSection;
 import org.apache.accumulo.core.replication.ReplicationSchema.WorkSection;
@@ -145,6 +144,7 @@ public class RemoveCompleteReplicationRecords implements Runnable {
 
     Mutation m = new Mutation(row);
     Status status = null;
+    long closedTime = -1l;
     for (Entry<Key,Value> entry : columns.entrySet()) {
       try {
         status = Status.parseFrom(entry.getValue().get());
@@ -158,6 +158,14 @@ public class RemoveCompleteReplicationRecords implements Runnable {
         return 0l;
       }
 
+      if (status.hasClosedTime()) {
+        if (closedTime == -1) {
+          closedTime = status.getClosedTime();
+        } else if (closedTime != status.getClosedTime()) {
+          log.warn("Inconsistent closed time for {}, values seen: {} and {}", row, closedTime, status.getClosedTime());
+        }
+      }
+
       Key k = entry.getKey();
       k.getColumnFamily(colf);
       k.getColumnQualifier(colq);
@@ -172,6 +180,7 @@ public class RemoveCompleteReplicationRecords implements Runnable {
     ReplicationTarget target = ReplicationTarget.from(colq);
 
     Mutation orderMutation = OrderSection.createMutation(row.toString(), status.getClosedTime());
+    log.info("Deleting {} from order section with tableID {}", new Key(new Text(orderMutation.getRow())).toStringNoTruncate(), target.getSourceTableId());
     orderMutation.putDelete(OrderSection.NAME, new Text(target.getSourceTableId()));
 
     // Send the mutation deleting all the columns at once.

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8df4f41b/server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java
index 866450c..efcd6c2 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java
@@ -1407,7 +1407,7 @@ public class Tablet {
         if (ReplicationConfigurationUtil.isEnabled(extent, tabletServer.getTableConfiguration(extent))) {
           Status status = StatusUtil.fileClosed(System.currentTimeMillis());
           for (LogEntry logEntry : logEntries) {
-            log.debug("Writing closed status to replication table for " + logEntry.logSet + " " + ProtobufUtil.toString(status));
+            log.debug("Writing closed status to metadata table for " + logEntry.logSet + " " + ProtobufUtil.toString(status));
             ReplicationTableUtil.updateFiles(SystemCredentials.get(), extent, logEntry.logSet, status);
           }
         }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8df4f41b/test/src/test/java/org/apache/accumulo/test/replication/ReplicationSequentialIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/replication/ReplicationSequentialIT.java b/test/src/test/java/org/apache/accumulo/test/replication/ReplicationSequentialIT.java
index dce4e17..f1d25ae 100644
--- a/test/src/test/java/org/apache/accumulo/test/replication/ReplicationSequentialIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/replication/ReplicationSequentialIT.java
@@ -18,6 +18,7 @@ package org.apache.accumulo.test.replication;
 
 import java.util.Iterator;
 import java.util.Map.Entry;
+import java.util.Set;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
@@ -160,6 +161,8 @@ public class ReplicationSequentialIT extends ConfigurableMacIT {
       }
     }
 
+    final Set<String> filesNeedingReplication = connMaster.replicationOperations().referencedFiles(masterTable);
+
     for (ProcessReference proc : cluster.getProcesses().get(ServerType.TABLET_SERVER)) {
       cluster.killProcess(ServerType.TABLET_SERVER, proc);
     }
@@ -189,7 +192,7 @@ public class ReplicationSequentialIT extends ConfigurableMacIT {
 
       @Override
       public Boolean call() throws Exception {
-        connMaster.replicationOperations().drain(masterTable);
+        connMaster.replicationOperations().drain(masterTable, filesNeedingReplication);
         log.info("Drain completed");
         return true;
       }
@@ -200,7 +203,7 @@ public class ReplicationSequentialIT extends ConfigurableMacIT {
       future.get(30, TimeUnit.SECONDS);
     } catch (TimeoutException e) {
       future.cancel(true);
-      Assert.fail("Drain did not finish within 5 seconds");
+      Assert.fail("Drain did not finish within 30 seconds");
     }
 
     log.info("drain completed");
@@ -297,11 +300,13 @@ public class ReplicationSequentialIT extends ConfigurableMacIT {
 
       // Write some data to table1
       BatchWriter bw = connMaster.createBatchWriter(masterTable1, new BatchWriterConfig());
+      long masterTable1Records = 0l; 
       for (int rows = 0; rows < 2500; rows++) {
         Mutation m = new Mutation(masterTable1 + rows);
         for (int cols = 0; cols < 100; cols++) {
           String value = Integer.toString(cols);
           m.put(value, "", value);
+          masterTable1Records++;
         }
         bw.addMutation(m);
       }
@@ -310,11 +315,13 @@ public class ReplicationSequentialIT extends ConfigurableMacIT {
 
       // Write some data to table2
       bw = connMaster.createBatchWriter(masterTable2, new BatchWriterConfig());
+      long masterTable2Records = 0l;
       for (int rows = 0; rows < 2500; rows++) {
         Mutation m = new Mutation(masterTable2 + rows);
         for (int cols = 0; cols < 100; cols++) {
           String value = Integer.toString(cols);
           m.put(value, "", value);
+          masterTable2Records++;
         }
         bw.addMutation(m);
       }
@@ -327,6 +334,9 @@ public class ReplicationSequentialIT extends ConfigurableMacIT {
         Thread.sleep(500);
       }
 
+      Set<String> filesFor1 = connMaster.replicationOperations().referencedFiles(masterTable1), filesFor2 = connMaster.replicationOperations().referencedFiles(
+          masterTable2);
+
       // Restart the tserver to force a close on the WAL
       for (ProcessReference proc : cluster.getProcesses().get(ServerType.TABLET_SERVER)) {
         cluster.killProcess(ServerType.TABLET_SERVER, proc);
@@ -335,22 +345,15 @@ public class ReplicationSequentialIT extends ConfigurableMacIT {
 
       log.info("Restarted the tserver");
 
-      // Wait until we fully replicated something
-      boolean fullyReplicated = false;
-      for (int i = 0; i < 10 && !fullyReplicated; i++) {
-        UtilWaitThread.sleep(2000);
-
-        Scanner s = ReplicationTable.getScanner(connMaster);
-        WorkSection.limit(s);
-        for (Entry<Key,Value> entry : s) {
-          Status status = Status.parseFrom(entry.getValue().get());
-          if (StatusUtil.isFullyReplicated(status)) {
-            fullyReplicated |= true;
-          }
-        }
-      }
+      // Read the data -- the tserver is back up and running
+      for (@SuppressWarnings("unused") Entry<Key,Value> entry : connMaster.createScanner(masterTable1, Authorizations.EMPTY)) {}
+
+      // Wait for both tables to be replicated
+      log.info("Waiting for {} for {}", filesFor1, masterTable1);
+      connMaster.replicationOperations().drain(masterTable1, filesFor1);
 
-      Assert.assertNotEquals(0, fullyReplicated);
+      log.info("Waiting for {} for {}", filesFor2, masterTable2);
+      connMaster.replicationOperations().drain(masterTable2, filesFor2);
 
       long countTable = 0l;
       for (Entry<Key,Value> entry : connPeer.createScanner(peerTable1, Authorizations.EMPTY)) {
@@ -360,7 +363,7 @@ public class ReplicationSequentialIT extends ConfigurableMacIT {
       }
 
       log.info("Found {} records in {}", countTable, peerTable1);
-      Assert.assertTrue(countTable > 0);
+      Assert.assertEquals(masterTable1Records, countTable);
 
       countTable = 0l;
       for (Entry<Key,Value> entry : connPeer.createScanner(peerTable2, Authorizations.EMPTY)) {
@@ -370,7 +373,7 @@ public class ReplicationSequentialIT extends ConfigurableMacIT {
       }
 
       log.info("Found {} records in {}", countTable, peerTable2);
-      Assert.assertTrue(countTable > 0);
+      Assert.assertEquals(masterTable2Records, countTable);
 
     } finally {
       peer1Cluster.stop();


[13/50] [abbrv] git commit: Merge branch '1.6.1-SNAPSHOT'

Posted by el...@apache.org.
Merge branch '1.6.1-SNAPSHOT'


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

Branch: refs/heads/ACCUMULO-378
Commit: 18591bc98a42454e5c6cff0d0c0afc35a5aef34c
Parents: 246c415 01e1181
Author: Christopher Tubbs <ct...@apache.org>
Authored: Fri May 16 12:49:24 2014 -0400
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Fri May 16 12:49:24 2014 -0400

----------------------------------------------------------------------
 bin/generate_monitor_certificate.sh | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------



[06/50] [abbrv] git commit: ACCUMULO-2819 Introduce closedTime and an OrderSection to the replication table.

Posted by el...@apache.org.
ACCUMULO-2819 Introduce closedTime and an OrderSection to the replication table.

Need to have global ordering for WALs used by a table. Using this, we can make guarantees
about what data is safe to replicate concurrently, and what data must wait to be replicated.


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

Branch: refs/heads/ACCUMULO-378
Commit: f97f13ab172169d1adf246ccc84768cf3cb551e3
Parents: 18432fe
Author: Josh Elser <el...@apache.org>
Authored: Thu May 15 21:03:50 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Thu May 15 21:03:50 2014 -0400

----------------------------------------------------------------------
 .../core/replication/ReplicationSchema.java     | 130 +++++++++-
 .../accumulo/core/replication/StatusUtil.java   |  28 ++-
 .../core/replication/proto/Replication.java     | 250 ++++++++++++++++++-
 core/src/main/protobuf/replication.proto        |   9 +-
 .../apache/accumulo/server/fs/VolumeUtil.java   |   2 +-
 .../server/replication/StatusCombinerTest.java  |   4 +-
 .../server/util/ReplicationTableUtilTest.java   |   2 +-
 .../CloseWriteAheadLogReferences.java           |   2 +-
 .../gc/GarbageCollectWriteAheadLogsTest.java    |   5 +-
 .../master/replication/StatusMaker.java         |  64 ++++-
 .../accumulo/master/replication/WorkMaker.java  |   1 -
 .../master/replication/StatusMakerTest.java     |  64 +++++
 .../master/replication/WorkMakerTest.java       |  22 +-
 .../org/apache/accumulo/tserver/Tablet.java     |  15 +-
 14 files changed, 535 insertions(+), 63 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/f97f13ab/core/src/main/java/org/apache/accumulo/core/replication/ReplicationSchema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/replication/ReplicationSchema.java b/core/src/main/java/org/apache/accumulo/core/replication/ReplicationSchema.java
index 57be449..725758e 100644
--- a/core/src/main/java/org/apache/accumulo/core/replication/ReplicationSchema.java
+++ b/core/src/main/java/org/apache/accumulo/core/replication/ReplicationSchema.java
@@ -16,7 +16,10 @@
  */
 package org.apache.accumulo.core.replication;
 
+import java.nio.charset.CharacterCodingException;
+
 import org.apache.accumulo.core.client.ScannerBase;
+import org.apache.accumulo.core.client.lexicoder.ULongLexicoder;
 import org.apache.accumulo.core.data.ArrayByteSequence;
 import org.apache.accumulo.core.data.ByteSequence;
 import org.apache.accumulo.core.data.Key;
@@ -34,7 +37,7 @@ public class ReplicationSchema {
   /**
    * Portion of a file that must be replication to the given target: peer and some identifying location on that peer, e.g. remote table ID
    * <p>
-   * <code>hdfs://localhost:8020/accumulo/wal/tserver+port/WAL work:serialized_ReplicationTarget [] -> Protobuf</code>
+   * <code>hdfs://localhost:8020/accumulo/wal/tserver+port/WAL work:serialized_ReplicationTarget [] -> Status Protobuf</code>
    */
   public static class WorkSection {
     public static final Text NAME = new Text("work");
@@ -75,14 +78,14 @@ public class ReplicationSchema {
   /**
    * Holds replication markers tracking status for files
    * <p>
-   * <code>hdfs://localhost:8020/accumulo/wal/tserver+port/WAL repl:local_table_id [] -> Protobuf</code>
+   * <code>hdfs://localhost:8020/accumulo/wal/tserver+port/WAL repl:local_table_id [] -> Status Protobuf</code>
    */
   public static class StatusSection {
     public static final Text NAME = new Text("repl");
-    private static final ByteSequence BYTE_SEQ_NAME = new ArrayByteSequence("repl");
+    private static final ByteSequence BYTE_SEQ_NAME = new ArrayByteSequence("repl"); 
 
     /**
-     * Extract the table ID from the colfam (inefficiently if called repeatedly)
+     * Extract the table ID from the key (inefficiently if called repeatedly)
      * @param k Key to extract from
      * @return The table ID
      * @see #getTableId(Key,Text) 
@@ -94,7 +97,7 @@ public class ReplicationSchema {
     }
 
     /**
-     * Extract the table ID from the colfam into the given {@link Text}
+     * Extract the table ID from the key into the given {@link Text}
      * @param k Key to extract from
      * @param buff Text to place table ID into
      */
@@ -119,7 +122,7 @@ public class ReplicationSchema {
     }
 
     /**
-     * Limit the scanner to only return ingest records
+     * Limit the scanner to only return Status records
      * @param scanner
      */
     public static void limit(ScannerBase scanner) {
@@ -132,6 +135,121 @@ public class ReplicationSchema {
     }
   }
 
+  /**
+   * Holds the order in which files needed for replication were closed. The intent is to be able to guarantee
+   * that files which were closed earlier were replicated first and we don't replay data in the wrong order on our peers
+   * <p>
+   * <code>encodedTimeOfClosure_hdfs://localhost:8020/accumulo/wal/tserver+port/WAL order:source_table_id [] -> Status Protobuf</code>
+   */
+  public static class OrderSection {
+    public static final Text NAME = new Text("order");
+    public static final String ROW_SEPARATOR = "_";
+    private static final ULongLexicoder longEncoder = new ULongLexicoder();
+
+    /**
+     * Extract the table ID from the given key (inefficiently if called repeatedly)
+     * @param k OrderSection Key
+     * @return source table id
+     */
+    public static String getTableId(Key k) {
+      Text buff = new Text();
+      getTableId(k, buff);
+      return buff.toString();
+    }
+
+    /**
+     * Extract the table ID from the given key
+     * @param k OrderSection key
+     * @param buff Text to place table ID into
+     */
+    public static void getTableId(Key k, Text buff) {
+      Preconditions.checkNotNull(k);
+      Preconditions.checkNotNull(buff);
+
+      k.getColumnQualifier(buff);
+    }
+
+    /**
+     * Limit the scanner to only return Order records
+     * @param scanner
+     */
+    public static void limit(ScannerBase scanner) {
+      scanner.fetchColumnFamily(NAME);
+    }
+
+    /**
+     * Creates the Mutation for the Order section for the given file and time, adding the column
+     * as well using {@link OrderSection#add(Mutation, Text, Value)}
+     * @param file Filename
+     * @param timeInMillis Time in millis that the file was closed
+     * @param tableId Source table id
+     * @param v Serialized Status msg as a Value
+     * @return Mutation for the Order section
+     */
+    public static Mutation createMutation(String file, long timeInMillis, Text tableId, Value v) {
+      Preconditions.checkNotNull(file);
+      Preconditions.checkArgument(timeInMillis >= 0, "timeInMillis must be greater than zero");
+      Preconditions.checkNotNull(v);
+
+      // Encode the time so it sorts properly
+      byte[] rowPrefix = longEncoder.encode(timeInMillis);
+      Text row = new Text(rowPrefix);
+      // Append the file as a suffix to the row
+      row.append((ROW_SEPARATOR+file).getBytes(), 0, file.length() + ROW_SEPARATOR.length());
+
+      // Make the mutation and add the column update
+      Mutation m = new Mutation(row);
+      return add(m, tableId, v);
+    }
+
+    /**
+     * Add a column update to the given mutation with the provided tableId and value
+     * @param m Mutation for OrderSection
+     * @param tableId Source table id
+     * @param v Serialized Status msg
+     * @return The original Mutation
+     */
+    public static Mutation add(Mutation m, Text tableId, Value v) {
+      m.put(NAME, tableId, v);
+      return m;
+    }
+
+    public static long getTimeClosed(Key k) {
+      return getTimeClosed(k, new Text());
+    }
+
+    public static long getTimeClosed(Key k, Text buff) {
+      k.getRow(buff);
+      int offset = buff.find(ROW_SEPARATOR);
+      if (-1 == offset) {
+        throw new IllegalArgumentException("Row does not contain expected separator for OrderSection");
+      }
+
+      byte[] encodedLong = new byte[offset];
+      System.arraycopy(buff.getBytes(), 0, encodedLong, 0, offset);
+      return longEncoder.decode(encodedLong);
+    }
+
+    public static String getFile(Key k) {
+      Text buff = new Text();
+      return getFile(k, buff);
+    }
+
+    public static String getFile(Key k, Text buff) {
+      k.getRow(buff);
+      int offset = buff.find(ROW_SEPARATOR);
+      if (-1 == offset) {
+        throw new IllegalArgumentException("Row does not contain expected separator for OrderSection");
+      }
+
+      try {
+        return Text.decode(buff.getBytes(), offset + 1, buff.getLength() - (offset + 1));
+      } catch (CharacterCodingException e) {
+        throw new IllegalArgumentException("Could not decode file path", e);
+      }
+    }
+  }
+
   private static void _getFile(Key k, Text buff) {
     k.getRow(buff);
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f97f13ab/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 842d945..94c60ab 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
@@ -28,8 +28,10 @@ import com.google.protobuf.InvalidProtocolBufferException;
  */
 public class StatusUtil {
 
-  private static final Status NEW_REPLICATION_STATUS, CLOSED_REPLICATION_STATUS, INF_END_REPLICATION_STATUS;
-  private static final Value NEW_REPLICATION_STATUS_VALUE, CLOSED_REPLICATION_STATUS_VALUE, INF_END_REPLICATION_STATUS_VALUE;
+  private static final Status NEW_REPLICATION_STATUS, INF_END_REPLICATION_STATUS;
+  private static final Value NEW_REPLICATION_STATUS_VALUE, INF_END_REPLICATION_STATUS_VALUE;
+
+  private static final Status.Builder CLOSED_STATUS_BUILDER;
 
   static {
     Status.Builder builder = Status.newBuilder();
@@ -40,13 +42,11 @@ public class StatusUtil {
     NEW_REPLICATION_STATUS = builder.build();
     NEW_REPLICATION_STATUS_VALUE = ProtobufUtil.toValue(NEW_REPLICATION_STATUS);
 
-    builder = Status.newBuilder();
-    builder.setBegin(0);
-    builder.setEnd(0);
-    builder.setInfiniteEnd(true);
-    builder.setClosed(true);
-    CLOSED_REPLICATION_STATUS = builder.build();
-    CLOSED_REPLICATION_STATUS_VALUE = ProtobufUtil.toValue(CLOSED_REPLICATION_STATUS);
+    CLOSED_STATUS_BUILDER = Status.newBuilder();
+    CLOSED_STATUS_BUILDER.setBegin(0);
+    CLOSED_STATUS_BUILDER.setEnd(0);
+    CLOSED_STATUS_BUILDER.setInfiniteEnd(true);
+    CLOSED_STATUS_BUILDER.setClosed(true);
 
     builder = Status.newBuilder();
     builder.setBegin(0);
@@ -137,15 +137,17 @@ public class StatusUtil {
   /**
    * @return A {@link Status} for a closed file of unspecified length, all of which needs replicating.
    */
-  public static Status fileClosed() {
-    return CLOSED_REPLICATION_STATUS;
+  public static synchronized Status fileClosed(long timeClosed) {
+    // We're using a shared builder, so we need to synchronize access on it until we make a Status (which is then immutable)
+    CLOSED_STATUS_BUILDER.setClosedTime(timeClosed);
+    return CLOSED_STATUS_BUILDER.build();
   }
 
   /**
    * @return A {@link Value} for a closed file of unspecified length, all of which needs replicating.
    */
-  public static Value fileClosedValue() {
-    return CLOSED_REPLICATION_STATUS_VALUE;
+  public static Value fileClosedValue(long timeClosed) {
+    return ProtobufUtil.toValue(fileClosed(timeClosed));
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f97f13ab/core/src/main/java/org/apache/accumulo/core/replication/proto/Replication.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/replication/proto/Replication.java b/core/src/main/java/org/apache/accumulo/core/replication/proto/Replication.java
index a7e80f8..d301028 100644
--- a/core/src/main/java/org/apache/accumulo/core/replication/proto/Replication.java
+++ b/core/src/main/java/org/apache/accumulo/core/replication/proto/Replication.java
@@ -30,42 +30,92 @@ package org.apache.accumulo.core.replication.proto;
     // optional int64 begin = 1 [default = 0];
     /**
      * <code>optional int64 begin = 1 [default = 0];</code>
+     *
+     * <pre>
+     * offset where replication should start
+     * </pre>
      */
     boolean hasBegin();
     /**
      * <code>optional int64 begin = 1 [default = 0];</code>
+     *
+     * <pre>
+     * offset where replication should start
+     * </pre>
      */
     long getBegin();
 
     // optional int64 end = 2 [default = 0];
     /**
      * <code>optional int64 end = 2 [default = 0];</code>
+     *
+     * <pre>
+     * offset where data is ready for replication
+     * </pre>
      */
     boolean hasEnd();
     /**
      * <code>optional int64 end = 2 [default = 0];</code>
+     *
+     * <pre>
+     * offset where data is ready for replication
+     * </pre>
      */
     long getEnd();
 
     // optional bool infiniteEnd = 3 [default = false];
     /**
      * <code>optional bool infiniteEnd = 3 [default = false];</code>
+     *
+     * <pre>
+     * do we have a discrete 'end'
+     * </pre>
      */
     boolean hasInfiniteEnd();
     /**
      * <code>optional bool infiniteEnd = 3 [default = false];</code>
+     *
+     * <pre>
+     * do we have a discrete 'end'
+     * </pre>
      */
     boolean getInfiniteEnd();
 
     // optional bool closed = 4 [default = false];
     /**
      * <code>optional bool closed = 4 [default = false];</code>
+     *
+     * <pre>
+     * will more data be appended to the file
+     * </pre>
      */
     boolean hasClosed();
     /**
      * <code>optional bool closed = 4 [default = false];</code>
+     *
+     * <pre>
+     * will more data be appended to the file
+     * </pre>
      */
     boolean getClosed();
+
+    // optional int64 closedTime = 5 [default = 0];
+    /**
+     * <code>optional int64 closedTime = 5 [default = 0];</code>
+     *
+     * <pre>
+     * when, in ms, was the file closed?
+     * </pre>
+     */
+    boolean hasClosedTime();
+    /**
+     * <code>optional int64 closedTime = 5 [default = 0];</code>
+     *
+     * <pre>
+     * when, in ms, was the file closed?
+     * </pre>
+     */
+    long getClosedTime();
   }
   /**
    * Protobuf type {@code Status}
@@ -138,6 +188,11 @@ package org.apache.accumulo.core.replication.proto;
               closed_ = input.readBool();
               break;
             }
+            case 40: {
+              bitField0_ |= 0x00000010;
+              closedTime_ = input.readInt64();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -183,12 +238,20 @@ package org.apache.accumulo.core.replication.proto;
     private long begin_;
     /**
      * <code>optional int64 begin = 1 [default = 0];</code>
+     *
+     * <pre>
+     * offset where replication should start
+     * </pre>
      */
     public boolean hasBegin() {
       return ((bitField0_ & 0x00000001) == 0x00000001);
     }
     /**
      * <code>optional int64 begin = 1 [default = 0];</code>
+     *
+     * <pre>
+     * offset where replication should start
+     * </pre>
      */
     public long getBegin() {
       return begin_;
@@ -199,12 +262,20 @@ package org.apache.accumulo.core.replication.proto;
     private long end_;
     /**
      * <code>optional int64 end = 2 [default = 0];</code>
+     *
+     * <pre>
+     * offset where data is ready for replication
+     * </pre>
      */
     public boolean hasEnd() {
       return ((bitField0_ & 0x00000002) == 0x00000002);
     }
     /**
      * <code>optional int64 end = 2 [default = 0];</code>
+     *
+     * <pre>
+     * offset where data is ready for replication
+     * </pre>
      */
     public long getEnd() {
       return end_;
@@ -215,12 +286,20 @@ package org.apache.accumulo.core.replication.proto;
     private boolean infiniteEnd_;
     /**
      * <code>optional bool infiniteEnd = 3 [default = false];</code>
+     *
+     * <pre>
+     * do we have a discrete 'end'
+     * </pre>
      */
     public boolean hasInfiniteEnd() {
       return ((bitField0_ & 0x00000004) == 0x00000004);
     }
     /**
      * <code>optional bool infiniteEnd = 3 [default = false];</code>
+     *
+     * <pre>
+     * do we have a discrete 'end'
+     * </pre>
      */
     public boolean getInfiniteEnd() {
       return infiniteEnd_;
@@ -231,22 +310,55 @@ package org.apache.accumulo.core.replication.proto;
     private boolean closed_;
     /**
      * <code>optional bool closed = 4 [default = false];</code>
+     *
+     * <pre>
+     * will more data be appended to the file
+     * </pre>
      */
     public boolean hasClosed() {
       return ((bitField0_ & 0x00000008) == 0x00000008);
     }
     /**
      * <code>optional bool closed = 4 [default = false];</code>
+     *
+     * <pre>
+     * will more data be appended to the file
+     * </pre>
      */
     public boolean getClosed() {
       return closed_;
     }
 
+    // optional int64 closedTime = 5 [default = 0];
+    public static final int CLOSEDTIME_FIELD_NUMBER = 5;
+    private long closedTime_;
+    /**
+     * <code>optional int64 closedTime = 5 [default = 0];</code>
+     *
+     * <pre>
+     * when, in ms, was the file closed?
+     * </pre>
+     */
+    public boolean hasClosedTime() {
+      return ((bitField0_ & 0x00000010) == 0x00000010);
+    }
+    /**
+     * <code>optional int64 closedTime = 5 [default = 0];</code>
+     *
+     * <pre>
+     * when, in ms, was the file closed?
+     * </pre>
+     */
+    public long getClosedTime() {
+      return closedTime_;
+    }
+
     private void initFields() {
       begin_ = 0L;
       end_ = 0L;
       infiniteEnd_ = false;
       closed_ = false;
+      closedTime_ = 0L;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -272,6 +384,9 @@ package org.apache.accumulo.core.replication.proto;
       if (((bitField0_ & 0x00000008) == 0x00000008)) {
         output.writeBool(4, closed_);
       }
+      if (((bitField0_ & 0x00000010) == 0x00000010)) {
+        output.writeInt64(5, closedTime_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -297,6 +412,10 @@ package org.apache.accumulo.core.replication.proto;
         size += com.google.protobuf.CodedOutputStream
           .computeBoolSize(4, closed_);
       }
+      if (((bitField0_ & 0x00000010) == 0x00000010)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeInt64Size(5, closedTime_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -421,6 +540,8 @@ package org.apache.accumulo.core.replication.proto;
         bitField0_ = (bitField0_ & ~0x00000004);
         closed_ = false;
         bitField0_ = (bitField0_ & ~0x00000008);
+        closedTime_ = 0L;
+        bitField0_ = (bitField0_ & ~0x00000010);
         return this;
       }
 
@@ -465,6 +586,10 @@ package org.apache.accumulo.core.replication.proto;
           to_bitField0_ |= 0x00000008;
         }
         result.closed_ = closed_;
+        if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
+          to_bitField0_ |= 0x00000010;
+        }
+        result.closedTime_ = closedTime_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -493,6 +618,9 @@ package org.apache.accumulo.core.replication.proto;
         if (other.hasClosed()) {
           setClosed(other.getClosed());
         }
+        if (other.hasClosedTime()) {
+          setClosedTime(other.getClosedTime());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -524,18 +652,30 @@ package org.apache.accumulo.core.replication.proto;
       private long begin_ ;
       /**
        * <code>optional int64 begin = 1 [default = 0];</code>
+       *
+       * <pre>
+       * offset where replication should start
+       * </pre>
        */
       public boolean hasBegin() {
         return ((bitField0_ & 0x00000001) == 0x00000001);
       }
       /**
        * <code>optional int64 begin = 1 [default = 0];</code>
+       *
+       * <pre>
+       * offset where replication should start
+       * </pre>
        */
       public long getBegin() {
         return begin_;
       }
       /**
        * <code>optional int64 begin = 1 [default = 0];</code>
+       *
+       * <pre>
+       * offset where replication should start
+       * </pre>
        */
       public Builder setBegin(long value) {
         bitField0_ |= 0x00000001;
@@ -545,6 +685,10 @@ package org.apache.accumulo.core.replication.proto;
       }
       /**
        * <code>optional int64 begin = 1 [default = 0];</code>
+       *
+       * <pre>
+       * offset where replication should start
+       * </pre>
        */
       public Builder clearBegin() {
         bitField0_ = (bitField0_ & ~0x00000001);
@@ -557,18 +701,30 @@ package org.apache.accumulo.core.replication.proto;
       private long end_ ;
       /**
        * <code>optional int64 end = 2 [default = 0];</code>
+       *
+       * <pre>
+       * offset where data is ready for replication
+       * </pre>
        */
       public boolean hasEnd() {
         return ((bitField0_ & 0x00000002) == 0x00000002);
       }
       /**
        * <code>optional int64 end = 2 [default = 0];</code>
+       *
+       * <pre>
+       * offset where data is ready for replication
+       * </pre>
        */
       public long getEnd() {
         return end_;
       }
       /**
        * <code>optional int64 end = 2 [default = 0];</code>
+       *
+       * <pre>
+       * offset where data is ready for replication
+       * </pre>
        */
       public Builder setEnd(long value) {
         bitField0_ |= 0x00000002;
@@ -578,6 +734,10 @@ package org.apache.accumulo.core.replication.proto;
       }
       /**
        * <code>optional int64 end = 2 [default = 0];</code>
+       *
+       * <pre>
+       * offset where data is ready for replication
+       * </pre>
        */
       public Builder clearEnd() {
         bitField0_ = (bitField0_ & ~0x00000002);
@@ -590,18 +750,30 @@ package org.apache.accumulo.core.replication.proto;
       private boolean infiniteEnd_ ;
       /**
        * <code>optional bool infiniteEnd = 3 [default = false];</code>
+       *
+       * <pre>
+       * do we have a discrete 'end'
+       * </pre>
        */
       public boolean hasInfiniteEnd() {
         return ((bitField0_ & 0x00000004) == 0x00000004);
       }
       /**
        * <code>optional bool infiniteEnd = 3 [default = false];</code>
+       *
+       * <pre>
+       * do we have a discrete 'end'
+       * </pre>
        */
       public boolean getInfiniteEnd() {
         return infiniteEnd_;
       }
       /**
        * <code>optional bool infiniteEnd = 3 [default = false];</code>
+       *
+       * <pre>
+       * do we have a discrete 'end'
+       * </pre>
        */
       public Builder setInfiniteEnd(boolean value) {
         bitField0_ |= 0x00000004;
@@ -611,6 +783,10 @@ package org.apache.accumulo.core.replication.proto;
       }
       /**
        * <code>optional bool infiniteEnd = 3 [default = false];</code>
+       *
+       * <pre>
+       * do we have a discrete 'end'
+       * </pre>
        */
       public Builder clearInfiniteEnd() {
         bitField0_ = (bitField0_ & ~0x00000004);
@@ -623,18 +799,30 @@ package org.apache.accumulo.core.replication.proto;
       private boolean closed_ ;
       /**
        * <code>optional bool closed = 4 [default = false];</code>
+       *
+       * <pre>
+       * will more data be appended to the file
+       * </pre>
        */
       public boolean hasClosed() {
         return ((bitField0_ & 0x00000008) == 0x00000008);
       }
       /**
        * <code>optional bool closed = 4 [default = false];</code>
+       *
+       * <pre>
+       * will more data be appended to the file
+       * </pre>
        */
       public boolean getClosed() {
         return closed_;
       }
       /**
        * <code>optional bool closed = 4 [default = false];</code>
+       *
+       * <pre>
+       * will more data be appended to the file
+       * </pre>
        */
       public Builder setClosed(boolean value) {
         bitField0_ |= 0x00000008;
@@ -644,6 +832,10 @@ package org.apache.accumulo.core.replication.proto;
       }
       /**
        * <code>optional bool closed = 4 [default = false];</code>
+       *
+       * <pre>
+       * will more data be appended to the file
+       * </pre>
        */
       public Builder clearClosed() {
         bitField0_ = (bitField0_ & ~0x00000008);
@@ -652,6 +844,55 @@ package org.apache.accumulo.core.replication.proto;
         return this;
       }
 
+      // optional int64 closedTime = 5 [default = 0];
+      private long closedTime_ ;
+      /**
+       * <code>optional int64 closedTime = 5 [default = 0];</code>
+       *
+       * <pre>
+       * when, in ms, was the file closed?
+       * </pre>
+       */
+      public boolean hasClosedTime() {
+        return ((bitField0_ & 0x00000010) == 0x00000010);
+      }
+      /**
+       * <code>optional int64 closedTime = 5 [default = 0];</code>
+       *
+       * <pre>
+       * when, in ms, was the file closed?
+       * </pre>
+       */
+      public long getClosedTime() {
+        return closedTime_;
+      }
+      /**
+       * <code>optional int64 closedTime = 5 [default = 0];</code>
+       *
+       * <pre>
+       * when, in ms, was the file closed?
+       * </pre>
+       */
+      public Builder setClosedTime(long value) {
+        bitField0_ |= 0x00000010;
+        closedTime_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional int64 closedTime = 5 [default = 0];</code>
+       *
+       * <pre>
+       * when, in ms, was the file closed?
+       * </pre>
+       */
+      public Builder clearClosedTime() {
+        bitField0_ = (bitField0_ & ~0x00000010);
+        closedTime_ = 0L;
+        onChanged();
+        return this;
+      }
+
       // @@protoc_insertion_point(builder_scope:Status)
     }
 
@@ -677,11 +918,12 @@ package org.apache.accumulo.core.replication.proto;
       descriptor;
   static {
     java.lang.String[] descriptorData = {
-      "\n#src/main/protobuf/replication.proto\"]\n" +
+      "\n#src/main/protobuf/replication.proto\"t\n" +
       "\006Status\022\020\n\005begin\030\001 \001(\003:\0010\022\016\n\003end\030\002 \001(\003:\001" +
       "0\022\032\n\013infiniteEnd\030\003 \001(\010:\005false\022\025\n\006closed\030" +
-      "\004 \001(\010:\005falseB.\n*org.apache.accumulo.core" +
-      ".replication.protoH\001"
+      "\004 \001(\010:\005false\022\025\n\nclosedTime\030\005 \001(\003:\0010B.\n*o" +
+      "rg.apache.accumulo.core.replication.prot" +
+      "oH\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -693,7 +935,7 @@ package org.apache.accumulo.core.replication.proto;
           internal_static_Status_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_Status_descriptor,
-              new java.lang.String[] { "Begin", "End", "InfiniteEnd", "Closed", });
+              new java.lang.String[] { "Begin", "End", "InfiniteEnd", "Closed", "ClosedTime", });
           return null;
         }
       };

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f97f13ab/core/src/main/protobuf/replication.proto
----------------------------------------------------------------------
diff --git a/core/src/main/protobuf/replication.proto b/core/src/main/protobuf/replication.proto
index bdcda61..be801b0 100644
--- a/core/src/main/protobuf/replication.proto
+++ b/core/src/main/protobuf/replication.proto
@@ -18,8 +18,9 @@ option java_package = "org.apache.accumulo.core.replication.proto";
 option optimize_for = SPEED;
 
 message Status {
-	optional int64 begin = 1 [default = 0];
-	optional int64 end = 2 [default = 0];
-	optional bool infiniteEnd = 3 [default = false];
-	optional bool closed = 4 [default = false];
+	optional int64 begin = 1 [default = 0]; // offset where replication should start
+	optional int64 end = 2 [default = 0]; // offset where data is ready for replication
+	optional bool infiniteEnd = 3 [default = false]; // do we have a discrete 'end'
+	optional bool closed = 4 [default = false]; // will more data be appended to the file
+	optional int64 closedTime = 5 [default = 0]; // when, in ms, was the file closed?
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f97f13ab/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 f83fcbe..400156c 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
@@ -232,7 +232,7 @@ public class VolumeUtil {
       MetadataTableUtil.updateTabletVolumes(extent, logsToRemove, logsToAdd, filesToRemove, filesToAdd, switchedDir, zooLock, creds);
       if (replicate) {
         // Before deleting these logs, we need to mark them for replication
-        ReplicationTableUtil.updateLogs(creds, extent, logsToRemove, StatusUtil.fileClosed());
+        ReplicationTableUtil.updateLogs(creds, extent, logsToRemove, StatusUtil.fileClosed(System.currentTimeMillis()));
       }
     }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f97f13ab/server/base/src/test/java/org/apache/accumulo/server/replication/StatusCombinerTest.java
----------------------------------------------------------------------
diff --git a/server/base/src/test/java/org/apache/accumulo/server/replication/StatusCombinerTest.java b/server/base/src/test/java/org/apache/accumulo/server/replication/StatusCombinerTest.java
index 0a5f5f5..8b8b72c 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/replication/StatusCombinerTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/replication/StatusCombinerTest.java
@@ -146,7 +146,7 @@ public class StatusCombinerTest {
 
   @Test
   public void commutativeWithClose() {
-    Status newFile = StatusUtil.newFile(), closed = StatusUtil.fileClosed(), secondSync = StatusUtil.ingestedUntil(200);
+    Status newFile = StatusUtil.newFile(), closed = StatusUtil.fileClosed(System.currentTimeMillis()), secondSync = StatusUtil.ingestedUntil(200);
 
     Status order1 = combiner.typedReduce(key, Arrays.asList(newFile, closed, secondSync).iterator()), order2 = combiner.typedReduce(key,
         Arrays.asList(newFile, secondSync, closed).iterator());
@@ -156,7 +156,7 @@ public class StatusCombinerTest {
 
   @Test
   public void commutativeWithCloseSingleBuilder() {
-    Status newFile = StatusUtil.newFile(), closed = StatusUtil.fileClosed(), secondSync = StatusUtil.ingestedUntil(builder, 200);
+    Status newFile = StatusUtil.newFile(), closed = StatusUtil.fileClosed(System.currentTimeMillis()), secondSync = StatusUtil.ingestedUntil(builder, 200);
 
     Status order1 = combiner.typedReduce(key, Arrays.asList(newFile, closed, secondSync).iterator()), order2 = combiner.typedReduce(key,
         Arrays.asList(newFile, secondSync, closed).iterator());

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f97f13ab/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 d1a43c3..be8e7ed 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
@@ -107,7 +107,7 @@ public class ReplicationTableUtilTest {
   @Test
   public void replEntryMutation() {
     // We stopped using a WAL -- we need a reference that this WAL needs to be replicated completely
-    Status stat = StatusUtil.fileClosed();
+    Status stat = StatusUtil.fileClosed(System.currentTimeMillis());
     String file = "file:///accumulo/wal/127.0.0.1+9997" + UUID.randomUUID();
     Path filePath = new Path(file);
     Text row = new Text(filePath.toString());

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f97f13ab/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 d649c3e..294883e 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
@@ -250,7 +250,7 @@ public class CloseWriteAheadLogReferences implements Runnable {
   protected void closeWal(BatchWriter bw, Key k) throws MutationsRejectedException {
     log.debug("Closing unreferenced WAL ({}) in metadata table", k.toStringNoTruncate());
     Mutation m = new Mutation(k.getRow());
-    m.put(k.getColumnFamily(), k.getColumnQualifier(), StatusUtil.fileClosedValue());
+    m.put(k.getColumnFamily(), k.getColumnQualifier(), StatusUtil.fileClosedValue(System.currentTimeMillis()));
     bw.addMutation(m);
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f97f13ab/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
index 1a3995b..2faa8a2 100644
--- a/server/gc/src/test/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogsTest.java
+++ b/server/gc/src/test/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogsTest.java
@@ -29,13 +29,10 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
 import java.util.LinkedList;
 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.BatchWriter;
@@ -341,7 +338,7 @@ public class GarbageCollectWriteAheadLogsTest {
     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()));
+    replData.add(Maps.immutableEntry(new Key("/wals/" + file1, StatusSection.NAME.toString(), "1"), StatusUtil.fileClosedValue(System.currentTimeMillis())));
     assertTrue(replGC.neededByReplication(conn, "/wals/" + file1));
 
     // File is closed and fully replicated, can be deleted

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f97f13ab/server/master/src/main/java/org/apache/accumulo/master/replication/StatusMaker.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/replication/StatusMaker.java b/server/master/src/main/java/org/apache/accumulo/master/replication/StatusMaker.java
index e44f184..8941a56 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/replication/StatusMaker.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/replication/StatusMaker.java
@@ -31,6 +31,7 @@ import org.apache.accumulo.core.metadata.MetadataTable;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.ReplicationSection;
 import org.apache.accumulo.core.protobuf.ProtobufUtil;
+import org.apache.accumulo.core.replication.ReplicationSchema.OrderSection;
 import org.apache.accumulo.core.replication.ReplicationSchema.StatusSection;
 import org.apache.accumulo.core.replication.proto.Replication.Status;
 import org.apache.accumulo.core.security.Authorizations;
@@ -104,28 +105,39 @@ public class StatusMaker {
         MetadataSchema.ReplicationSection.getFile(entry.getKey(), row);
         MetadataSchema.ReplicationSection.getTableId(entry.getKey(), tableId);
 
-        String rowStr = row.toString();
-        rowStr = rowStr.substring(ReplicationSection.getRowPrefix().length());
+        String file = row.toString();
+        file = file.substring(ReplicationSection.getRowPrefix().length());
 
         Status status;
         try {
           status = Status.parseFrom(entry.getValue().get());
         } catch (InvalidProtocolBufferException e) {
-          log.warn("Could not deserialize protobuf for {}", rowStr);
+          log.warn("Could not deserialize protobuf for {}", file);
           continue;
         }
 
-        log.debug("Creating replication status record for {} on table {} with {}.", rowStr, tableId, ProtobufUtil.toString(status));
+        log.debug("Creating replication status record for {} on table {} with {}.", file, tableId, ProtobufUtil.toString(status));
 
         Span workSpan = Trace.start("createStatusMutations");
         try {
           // Create entries in the replication table from the metadata table
-          addStatusRecord(rowStr, tableId, entry.getValue());
+          if (!addStatusRecord(file, tableId, entry.getValue())) {
+            continue;
+          }
         } finally {
           workSpan.stop();
         }
 
         if (status.getClosed()) {
+          Span orderSpan = Trace.start("recordStatusOrder");
+          try {
+            if (!addOrderRecord(file, tableId, status, entry.getValue())) {
+              continue;
+            }
+          } finally {
+            orderSpan.stop();
+          }
+
           Span deleteSpan = Trace.start("deleteClosedStatus");
           try {
             deleteStatusRecord(entry.getKey());
@@ -149,8 +161,7 @@ public class StatusMaker {
    * @param tableId
    * @param v
    */
-  protected void addStatusRecord(String file, Text tableId, Value v) {
-    // TODO come up with something that tries to avoid creating a new BatchWriter all the time
+  protected boolean addStatusRecord(String file, Text tableId, Value v) {
     try {
       Mutation m = new Mutation(file);
       m.put(StatusSection.NAME, tableId, v);
@@ -159,14 +170,53 @@ public class StatusMaker {
         replicationWriter.addMutation(m);
       } catch (MutationsRejectedException e) {
         log.warn("Failed to write work mutations for replication, will retry", e);
+        return false;
       }
     } finally {
       try {
         replicationWriter.flush();
       } catch (MutationsRejectedException e) {
         log.warn("Failed to write work mutations for replication, will retry", e);
+        return false;
       }
     }
+
+    return true;
+  }
+
+  /**
+   * Create a record to track when the file was closed to ensure that replication preference
+   * is given to files that have been closed the longest and allow the work assigner to try to
+   * replicate in order that data was ingested (avoid replay in different order)
+   * @param file File being replicated
+   * @param tableId Table ID the file was used by
+   * @param stat Status msg
+   * @param value Serialized version of the Status msg
+   */
+  protected boolean addOrderRecord(String file, Text tableId, Status stat, Value value) {
+    try {
+      if (!stat.hasClosedTime()) {
+        log.warn("Status record ({}) for {} in table {} was written to metadata table which was closed but lacked closedTime", ProtobufUtil.toString(stat), file, tableId);
+      }
+
+      Mutation m = OrderSection.createMutation(file, stat.getClosedTime(), tableId, value);
+
+      try {
+        replicationWriter.addMutation(m);
+      } catch (MutationsRejectedException e) {
+        log.warn("Failed to write order mutation for replication, will retry", e);
+        return false;
+      }
+    } finally {
+      try {
+        replicationWriter.flush();
+      } catch (MutationsRejectedException e) {
+        log.warn("Failed to write order mutation for replication, will retry", e);
+        return false;
+      }
+    }
+
+    return true;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f97f13ab/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 1aaae51..2dfddc2 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
@@ -156,7 +156,6 @@ public class WorkMaker {
   }
 
   protected void addWorkRecord(Text file, Value v, Map<String,String> targets, String sourceTableId) {
-    // TODO come up with something that tries to avoid creating a new BatchWriter all the time
     log.info("Adding work records for " + file + " to targets " + targets);
     try {
       Mutation m = new Mutation(file);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f97f13ab/server/master/src/test/java/org/apache/accumulo/master/replication/StatusMakerTest.java
----------------------------------------------------------------------
diff --git a/server/master/src/test/java/org/apache/accumulo/master/replication/StatusMakerTest.java b/server/master/src/test/java/org/apache/accumulo/master/replication/StatusMakerTest.java
index 496318d..e0fc421 100644
--- a/server/master/src/test/java/org/apache/accumulo/master/replication/StatusMakerTest.java
+++ b/server/master/src/test/java/org/apache/accumulo/master/replication/StatusMakerTest.java
@@ -16,7 +16,10 @@
  */
 package org.apache.accumulo.master.replication;
 
+import java.util.Arrays;
 import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
@@ -33,6 +36,7 @@ import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.metadata.schema.MetadataSchema.ReplicationSection;
 import org.apache.accumulo.core.protobuf.ProtobufUtil;
+import org.apache.accumulo.core.replication.ReplicationSchema.OrderSection;
 import org.apache.accumulo.core.replication.ReplicationSchema.StatusSection;
 import org.apache.accumulo.core.replication.StatusUtil;
 import org.apache.accumulo.core.replication.proto.Replication.Status;
@@ -187,4 +191,64 @@ public class StatusMakerTest {
     
   }
 
+  @Test
+  public void closedMessagesCreateOrderRecords() throws Exception {
+    MockInstance inst = new MockInstance(test.getMethodName());
+    Credentials creds = new Credentials("root", new PasswordToken(""));
+    Connector conn = inst.getConnector(creds.getPrincipal(), creds.getToken());
+
+    String sourceTable = "source";
+    conn.tableOperations().create(sourceTable);
+    ReplicationTableUtil.configureMetadataTable(conn, sourceTable);
+
+    BatchWriter bw = conn.createBatchWriter(sourceTable, new BatchWriterConfig());
+    String walPrefix = "hdfs://localhost:8020/accumulo/wals/tserver+port/";
+    List<String> files = Arrays.asList(walPrefix + UUID.randomUUID(), walPrefix + UUID.randomUUID(), walPrefix + UUID.randomUUID(),
+        walPrefix + UUID.randomUUID());
+    Map<String,Integer> fileToTableId = new HashMap<>();
+
+    Status.Builder statBuilder = Status.newBuilder().setBegin(0).setEnd(0).setInfiniteEnd(true).setClosed(true);
+
+    int index = 1;
+    long time = System.currentTimeMillis();
+    for (String file : files) {
+      statBuilder.setClosedTime(time++);
+      Mutation m = new Mutation(ReplicationSection.getRowPrefix() + file);
+      m.put(ReplicationSection.COLF, new Text(Integer.toString(index)), ProtobufUtil.toValue(statBuilder.build()));
+      bw.addMutation(m);
+      fileToTableId.put(file, index);
+      index++;
+    }
+
+    bw.close();
+
+    StatusMaker statusMaker = new StatusMaker(conn);
+    statusMaker.setSourceTableName(sourceTable);
+
+    statusMaker.run();
+
+    Scanner s = conn.createScanner(sourceTable, Authorizations.EMPTY);
+    s.setRange(ReplicationSection.getRange());
+    s.fetchColumnFamily(ReplicationSection.COLF);
+    Assert.assertEquals(0, Iterables.size(s));
+
+    s = ReplicationTable.getScanner(conn);
+    OrderSection.limit(s);
+    Iterator<Entry<Key,Value>> iter = s.iterator();
+    Assert.assertTrue("Found no order records in replication table", iter.hasNext());
+
+    Iterator<String> expectedFiles = files.iterator();
+    Text buff = new Text();
+    while (expectedFiles.hasNext() && iter.hasNext()) {
+      String file = expectedFiles.next();
+      Entry<Key,Value> entry  = iter.next();
+
+      Assert.assertEquals(file, OrderSection.getFile(entry.getKey(), buff));
+      OrderSection.getTableId(entry.getKey(), buff);
+      Assert.assertEquals(fileToTableId.get(file).intValue(), Integer.parseInt(buff.toString()));
+    }
+
+    Assert.assertFalse("Found more files unexpectedly", expectedFiles.hasNext());
+    Assert.assertFalse("Found more entries in replication table unexpectedly", iter.hasNext());
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f97f13ab/server/master/src/test/java/org/apache/accumulo/master/replication/WorkMakerTest.java
----------------------------------------------------------------------
diff --git a/server/master/src/test/java/org/apache/accumulo/master/replication/WorkMakerTest.java b/server/master/src/test/java/org/apache/accumulo/master/replication/WorkMakerTest.java
index cd313f9..337aa12 100644
--- a/server/master/src/test/java/org/apache/accumulo/master/replication/WorkMakerTest.java
+++ b/server/master/src/test/java/org/apache/accumulo/master/replication/WorkMakerTest.java
@@ -18,7 +18,6 @@ package org.apache.accumulo.master.replication;
 
 import java.util.HashMap;
 import java.util.HashSet;
-import java.util.Iterator;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
@@ -84,8 +83,10 @@ public class WorkMakerTest {
     String tableId = conn.tableOperations().tableIdMap().get(table);
     String file = "hdfs://localhost:8020/accumulo/wal/123456-1234-1234-12345678";
 
+    // Create a status record for a file
+    long timeClosed = System.currentTimeMillis();
     Mutation m = new Mutation(new Path(file).toString());
-    m.put(StatusSection.NAME, new Text(tableId), StatusUtil.fileClosedValue());
+    m.put(StatusSection.NAME, new Text(tableId), StatusUtil.fileClosedValue(timeClosed));
     BatchWriter bw = ReplicationTable.getBatchWriter(conn);
     bw.addMutation(m);
     bw.flush();
@@ -97,26 +98,23 @@ public class WorkMakerTest {
 
     WorkMaker workMaker = new WorkMaker(conn);
 
+    // Invoke the addWorkRecord method to create a Work record from the Status record earlier
     ReplicationTarget expected = new ReplicationTarget("remote_cluster_1", "4", tableId);
     workMaker.setBatchWriter(bw);
-    workMaker.addWorkRecord(new Text(file), StatusUtil.fileClosedValue(), ImmutableMap.of("remote_cluster_1", "4"), tableId);
+    workMaker.addWorkRecord(new Text(file), StatusUtil.fileClosedValue(timeClosed), ImmutableMap.of("remote_cluster_1", "4"), tableId);
 
+    // Scan over just the WorkSection
     s = ReplicationTable.getScanner(conn);
     WorkSection.limit(s);
 
-    Iterator<Entry<Key,Value>> iter = s.iterator();
-    Assert.assertTrue(iter.hasNext());
-
-    Entry<Key,Value> workEntry = iter.next();
+    Entry<Key,Value> workEntry = Iterables.getOnlyElement(s);
     Key workKey = workEntry.getKey();
     ReplicationTarget actual = ReplicationTarget.from(workKey.getColumnQualifier());
 
     Assert.assertEquals(file, workKey.getRow().toString());
     Assert.assertEquals(WorkSection.NAME, workKey.getColumnFamily());
     Assert.assertEquals(expected, actual);
-    Assert.assertEquals(workEntry.getValue(), StatusUtil.fileClosedValue());
-
-    Assert.assertFalse(iter.hasNext());
+    Assert.assertEquals(workEntry.getValue(), StatusUtil.fileClosedValue(timeClosed));
   }
 
   @Test
@@ -129,7 +127,7 @@ public class WorkMakerTest {
     String file = "hdfs://localhost:8020/accumulo/wal/123456-1234-1234-12345678";
 
     Mutation m = new Mutation(new Path(file).toString());
-    m.put(StatusSection.NAME, new Text(tableId), StatusUtil.fileClosedValue());
+    m.put(StatusSection.NAME, new Text(tableId), StatusUtil.fileClosedValue(System.currentTimeMillis()));
     BatchWriter bw = ReplicationTable.getBatchWriter(conn);
     bw.addMutation(m);
     bw.flush();
@@ -147,7 +145,7 @@ public class WorkMakerTest {
       expectedTargets.add(new ReplicationTarget(cluster.getKey(), cluster.getValue(), tableId));
     }
     workMaker.setBatchWriter(bw);
-    workMaker.addWorkRecord(new Text(file), StatusUtil.fileClosedValue(), targetClusters, tableId);
+    workMaker.addWorkRecord(new Text(file), StatusUtil.fileClosedValue(System.currentTimeMillis()), targetClusters, tableId);
 
     s = ReplicationTable.getScanner(conn);
     WorkSection.limit(s);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/f97f13ab/server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java
index 43c6c34..2b9c326 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java
@@ -1378,13 +1378,6 @@ public class Tablet {
         for (FileRef ref : datafiles.keySet())
           absPaths.add(ref.path().toString());
 
-        // Ensure that we write a record marking each WAL as requiring replication to make sure we don't abandon the data
-        if (ReplicationConfigurationUtil.isEnabled(extent, tabletServer.getTableConfiguration(extent))) {
-          for (LogEntry logEntry : logEntries) {
-            ReplicationTableUtil.updateFiles(SystemCredentials.get(), extent, logEntry.logSet, StatusUtil.fileClosed());
-          }
-        }
-
         tabletServer.recover(this.tabletServer.getFileSystem(), extent, acuTableConf, logEntries, absPaths, new MutationReceiver() {
           @Override
           public void receive(Mutation m) {
@@ -1407,6 +1400,14 @@ public class Tablet {
         }
         commitSession.updateMaxCommittedTime(tabletTime.getTime());
 
+        // Ensure that we write a record marking each WAL as requiring replication to make sure we don't abandon the data
+        if (ReplicationConfigurationUtil.isEnabled(extent, tabletServer.getTableConfiguration(extent))) {
+          long timeClosed = System.currentTimeMillis();
+          for (LogEntry logEntry : logEntries) {
+            ReplicationTableUtil.updateFiles(SystemCredentials.get(), extent, logEntry.logSet, StatusUtil.fileClosed(timeClosed));
+          }
+        }
+
         if (count[0] == 0) {
           MetadataTableUtil.removeUnusedWALEntries(extent, logEntries, tabletServer.getLock());
           logEntries.clear();


[16/50] [abbrv] git commit: ACCUMULO-2762 Use all optimizations by default

Posted by el...@apache.org.
ACCUMULO-2762 Use all optimizations by default


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

Branch: refs/heads/ACCUMULO-378
Commit: 10ce7af89ddaaef1863eed789a6df67946e09c59
Parents: e86bfc2
Author: Christopher Tubbs <ct...@apache.org>
Authored: Wed May 14 12:30:55 2014 -0400
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Fri May 16 12:58:12 2014 -0400

----------------------------------------------------------------------
 server/native/src/main/resources/Makefile | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/10ce7af8/server/native/src/main/resources/Makefile
----------------------------------------------------------------------
diff --git a/server/native/src/main/resources/Makefile b/server/native/src/main/resources/Makefile
index 1ca79eb..33c2fe9 100644
--- a/server/native/src/main/resources/Makefile
+++ b/server/native/src/main/resources/Makefile
@@ -25,7 +25,7 @@ ifeq ($(shell uname),Linux)
 		JAVA_HOME=$(shell dirname $$(dirname $$(readlink -ef $$(which javah))))
 	endif
 	NATIVE_LIB := libaccumulo.so
-	CXXFLAGS=-m64 -g -fPIC -shared -O2 -fno-omit-frame-pointer -fno-strict-aliasing -Wall -I$(JAVA_HOME)/include -I$(JAVA_HOME)/include/linux -Ijavah
+	CXXFLAGS=-m64 -g -fPIC -shared -O3 -Wall -I$(JAVA_HOME)/include -I$(JAVA_HOME)/include/linux -Ijavah
 endif
 
 ifeq ($(shell uname),Darwin)


[21/50] [abbrv] git commit: Merge branch '1.6.1-SNAPSHOT'

Posted by el...@apache.org.
Merge branch '1.6.1-SNAPSHOT'

Conflicts:
	docs/src/main/asciidoc/chapters/administration.txt


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

Branch: refs/heads/ACCUMULO-378
Commit: 6b36d53a90f817af6c53d4324f0d731ef7537450
Parents: c312fa6 f61abc0
Author: Christopher Tubbs <ct...@apache.org>
Authored: Fri May 16 15:37:08 2014 -0400
Committer: Christopher Tubbs <ct...@apache.org>
Committed: Fri May 16 15:37:08 2014 -0400

----------------------------------------------------------------------
 README                                           |  7 +++++++
 .../main/asciidoc/chapters/administration.txt    | 19 +++++++++++++------
 2 files changed, 20 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/6b36d53a/README
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/6b36d53a/docs/src/main/asciidoc/chapters/administration.txt
----------------------------------------------------------------------
diff --cc docs/src/main/asciidoc/chapters/administration.txt
index 4dbcd1b,0000000..b4c1a71
mode 100644,000000..100644
--- a/docs/src/main/asciidoc/chapters/administration.txt
+++ b/docs/src/main/asciidoc/chapters/administration.txt
@@@ -1,396 -1,0 +1,403 @@@
 +// 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.
 +
 +== Administration
 +
 +=== Hardware
 +
 +Because we are running essentially two or three systems simultaneously layered
 +across the cluster: HDFS, Accumulo and MapReduce, it is typical for hardware to
 +consist of 4 to 8 cores, and 8 to 32 GB RAM. This is so each running process can have
 +at least one core and 2 - 4 GB each.
 +
 +One core running HDFS can typically keep 2 to 4 disks busy, so each machine may
 +typically have as little as 2 x 300GB disks and as much as 4 x 1TB or 2TB disks.
 +
 +It is possible to do with less than this, such as with 1u servers with 2 cores and 4GB
 +each, but in this case it is recommended to only run up to two processes per
 +machine -- i.e. DataNode and TabletServer or DataNode and MapReduce worker but
 +not all three. The constraint here is having enough available heap space for all the
 +processes on a machine.
 +
 +=== Network
 +
 +Accumulo communicates via remote procedure calls over TCP/IP for both passing
 +data and control messages. In addition, Accumulo uses HDFS clients to
 +communicate with HDFS. To achieve good ingest and query performance, sufficient
 +network bandwidth must be available between any two machines.
 +
 +In addition to needing access to ports associated with HDFS and ZooKeeper, Accumulo will
 +use the following default ports. Please make sure that they are open, or change
 +their value in conf/accumulo-site.xml.
 +
 +.Accumulo default ports
 +[width="75%",cols=">,^2,^2"]
 +[options="header"]
 +|====
 +|Port | Description | Property Name
 +|4445 | Shutdown Port (Accumulo MiniCluster) | n/a
 +|4560 | Accumulo monitor (for centralized log display) | monitor.port.log4j
 +|9997 | Tablet Server | tserver.port.client
 +|9999 | Master Server | master.port.client
 +|12234 | Accumulo Tracer | trace.port.client
 +|42424 | Accumulo Proxy Server | n/a
 +|50091 | Accumulo GC | gc.port.client
 +|50095 | Accumulo HTTP monitor | monitor.port.client
 +|====
 +
 +In addition, the user can provide +0+ and an ephemeral port will be chosen instead. This
 +ephemeral port is likely to be unique and not already bound. Thus, configuring ports to
 +use +0+ instead of an explicit value, should, in most cases, work around any issues of
 +running multiple distinct Accumulo instances (or any other process which tries to use the
 +same default ports) on the same hardware.
 +
 +=== Installation
 +Choose a directory for the Accumulo installation. This directory will be referenced
 +by the environment variable +$ACCUMULO_HOME+. Run the following:
 +
 +  $ tar xzf accumulo-1.6.0-bin.tar.gz    # unpack to subdirectory
 +  $ mv accumulo-1.6.0 $ACCUMULO_HOME # move to desired location
 +
 +Repeat this step at each machine within the cluster. Usually all machines have the
 +same +$ACCUMULO_HOME+.
 +
 +=== Dependencies
 +Accumulo requires HDFS and ZooKeeper to be configured and running
 +before starting. Password-less SSH should be configured between at least the
 +Accumulo master and TabletServer machines. It is also a good idea to run Network
 +Time Protocol (NTP) within the cluster to ensure nodes' clocks don't get too out of
 +sync, which can cause problems with automatically timestamped data.
 +
 +=== Configuration
 +
 +Accumulo is configured by editing several Shell and XML files found in
 ++$ACCUMULO_HOME/conf+. The structure closely resembles Hadoop's configuration
 +files.
 +
 +==== Edit conf/accumulo-env.sh
 +
 +Accumulo needs to know where to find the software it depends on. Edit accumulo-env.sh
 +and specify the following:
 +
 +. Enter the location of the installation directory of Accumulo for +$ACCUMULO_HOME+
 +. Enter your system's Java home for +$JAVA_HOME+
 +. Enter the location of Hadoop for +$HADOOP_PREFIX+
 +. Choose a location for Accumulo logs and enter it for +$ACCUMULO_LOG_DIR+
 +. Enter the location of ZooKeeper for +$ZOOKEEPER_HOME+
 +
 +By default Accumulo TabletServers are set to use 1GB of memory. You may change
 +this by altering the value of +$ACCUMULO_TSERVER_OPTS+. Note the syntax is that of
 +the Java JVM command line options. This value should be less than the physical
 +memory of the machines running TabletServers.
 +
 +There are similar options for the master's memory usage and the garbage collector
 +process. Reduce these if they exceed the physical RAM of your hardware and
 +increase them, within the bounds of the physical RAM, if a process fails because of
 +insufficient memory.
 +
 +Note that you will be specifying the Java heap space in accumulo-env.sh. You should
 +make sure that the total heap space used for the Accumulo tserver and the Hadoop
 +DataNode and TaskTracker is less than the available memory on each slave node in
 +the cluster. On large clusters, it is recommended that the Accumulo master, Hadoop
 +NameNode, secondary NameNode, and Hadoop JobTracker all be run on separate
 +machines to allow them to use more heap space. If you are running these on the
 +same machine on a small cluster, likewise make sure their heap space settings fit
 +within the available memory.
 +
 +==== Native Map
 +
 +The tablet server uses a data structure called a MemTable to store sorted key/value
 +pairs in memory when they are first received from the client. When a minor compaction
 +occurs, this data structure is written to HDFS. The MemTable will default to using
 +memory in the JVM but a JNI version, called the native map, can be used to significantly
 +speed up performance by utilizing the memory space of the native operating system. The
 +native map also avoids the performance implications brought on by garbage collection
 +in the JVM by causing it to pause much less frequently.
 +
- 32-bit and 64-bit Linux versions of the native map ship with the Accumulo dist package.
- For other operating systems, the native map can be built from the codebase in two ways-
- from maven or from the Makefile.
++32-bit and 64-bit Linux and Mac OS X versions of the native map can be built
++from the Accumulo bin package by executing
+++$ACCUMULO_HOME/bin/build_native_library.sh+. If your system's
++default compiler options are insufficient, you can add additional compiler
++options to the command line, such as options for the architecture. These will be
++passed to the Makefile in the environment variable +USERFLAGS+.
 +
- . Build from maven using the following command: +mvn clean package -Pnative+.
- . Build from the c++ source by running +make+ in the `$ACCUMULO_HOME/server/src/main/c++` directory.
++Examples:
++
++. +$ACCUMULO_HOME/bin/build_native_library.sh+
++. +$ACCUMULO_HOME/bin/build_native_library.sh -m32+
 +
 +After building the native map from the source, you will find the artifact in
 ++$ACCUMULO_HOME/lib/native+. Upon starting up, the tablet server will look
 +in this directory for the map library. If the file is renamed or moved from its
- target directory, the tablet server may not be able to find it.
++target directory, the tablet server may not be able to find it. The system can
++also locate the native maps shared library by setting +LD_LIBRARY_PATH+
++(or +DYLD_LIBRARY_PATH+ on Mac OS X) in +$ACCUMULO_HOME/conf/accumulo-env.sh+.
 +
 +==== Cluster Specification
 +
 +On the machine that will serve as the Accumulo master:
 +
 +. Write the IP address or domain name of the Accumulo Master to the +$ACCUMULO_HOME/conf/masters+ file.
 +. Write the IP addresses or domain name of the machines that will be TabletServers in +$ACCUMULO_HOME/conf/slaves+, one per line.
 +
 +Note that if using domain names rather than IP addresses, DNS must be configured
 +properly for all machines participating in the cluster. DNS can be a confusing source
 +of errors.
 +
 +==== Accumulo Settings
 +Specify appropriate values for the following settings in
 ++$ACCUMULO_HOME/conf/accumulo-site.xml+ :
 +
 +[source,xml]
 +<property>
 +    <name>instance.zookeeper.host</name>
 +    <value>zooserver-one:2181,zooserver-two:2181</value>
 +    <description>list of zookeeper servers</description>
 +</property>
 +
 +This enables Accumulo to find ZooKeeper. Accumulo uses ZooKeeper to coordinate
 +settings between processes and helps finalize TabletServer failure.
 +
 +[source,xml]
 +<property>
 +    <name>instance.secret</name>
 +    <value>DEFAULT</value>
 +</property>
 +
 +The instance needs a secret to enable secure communication between servers. Configure your
 +secret and make sure that the +accumulo-site.xml+ file is not readable to other users.
 +
 +Some settings can be modified via the Accumulo shell and take effect immediately, but
 +some settings require a process restart to take effect. See the configuration documentation
 +(available in the docs directory of the tarball and in <<configuration>>) for details.
 +
 +==== Deploy Configuration
 +
 +Copy the masters, slaves, accumulo-env.sh, and if necessary, accumulo-site.xml
 +from the +$ACCUMULO_HOME/conf/+ directory on the master to all the machines
 +specified in the slaves file.
 +
 +=== Initialization
 +
 +Accumulo must be initialized to create the structures it uses internally to locate
 +data across the cluster. HDFS is required to be configured and running before
 +Accumulo can be initialized.
 +
 +Once HDFS is started, initialization can be performed by executing
 ++$ACCUMULO_HOME/bin/accumulo init+ . This script will prompt for a name
 +for this instance of Accumulo. The instance name is used to identify a set of tables
 +and instance-specific settings. The script will then write some information into
 +HDFS so Accumulo can start properly.
 +
 +The initialization script will prompt you to set a root password. Once Accumulo is
 +initialized it can be started.
 +
 +=== Running
 +
 +==== Starting Accumulo
 +
 +Make sure Hadoop is configured on all of the machines in the cluster, including
 +access to a shared HDFS instance. Make sure HDFS and ZooKeeper are running.
 +Make sure ZooKeeper is configured and running on at least one machine in the
 +cluster.
 +Start Accumulo using the +bin/start-all.sh+ script.
 +
 +To verify that Accumulo is running, check the Status page as described under
 +_Monitoring_. In addition, the Shell can provide some information about the status of
 +tables via reading the metadata tables.
 +
 +==== Stopping Accumulo
 +
 +To shutdown cleanly, run +bin/stop-all.sh+ and the master will orchestrate the
 +shutdown of all the tablet servers. Shutdown waits for all minor compactions to finish, so it may
 +take some time for particular configurations.
 +
 +==== Adding a Node
 +
 +Update your +$ACCUMULO_HOME/conf/slaves+ (or +$ACCUMULO_CONF_DIR/slaves+) file to account for the addition.
 +
 +  $ACCUMULO_HOME/bin/accumulo admin start <host(s)> {<host> ...}
 +
 +Alternatively, you can ssh to each of the hosts you want to add and run:
 +
 +  $ACCUMULO_HOME/bin/start-here.sh
 +
 +Make sure the host in question has the new configuration, or else the tablet
 +server won't start; at a minimum this needs to be on the host(s) being added,
 +but in practice it's good to ensure consistent configuration across all nodes.
 +
 +==== Decomissioning a Node
 +
 +If you need to take a node out of operation, you can trigger a graceful shutdown of a tablet
 +server. Accumulo will automatically rebalance the tablets across the available tablet servers.
 +
 +  $ACCUMULO_HOME/bin/accumulo admin stop <host(s)> {<host> ...}
 +
 +Alternatively, you can ssh to each of the hosts you want to remove and run:
 +
 +  $ACCUMULO_HOME/bin/stop-here.sh
 +
 +Be sure to update your +$ACCUMULO_HOME/conf/slaves+ (or +$ACCUMULO_CONF_DIR/slaves+) file to
 +account for the removal of these hosts. Bear in mind that the monitor will not re-read the
 +slaves file automatically, so it will report the decomissioned servers as down; it's
 +recommended that you restart the monitor so that the node list is up to date.
 +
 +=== Monitoring
 +
 +The Accumulo Master provides an interface for monitoring the status and health of
 +Accumulo components. This interface can be accessed by pointing a web browser to
 ++http://accumulomaster:50095/status+
 +
 +=== Tracing
 +It can be difficult to determine why some operations are taking longer
 +than expected. For example, you may be looking up items with very low
 +latency, but sometimes the lookups take much longer. Determining the
 +cause of the delay is difficult because the system is distributed, and
 +the typical lookup is fast.
 +
 +Accumulo has been instrumented to record the time that various
 +operations take when tracing is turned on. The fact that tracing is
 +enabled follows all the requests made on behalf of the user throughout
 +the distributed infrastructure of accumulo, and across all threads of
 +execution.
 +
 +These time spans will be inserted into the +trace+ table in
 +Accumulo. You can browse recent traces from the Accumulo monitor
 +page. You can also read the +trace+ table directly like any
 +other table.
 +
 +The design of Accumulo's distributed tracing follows that of
 +http://research.google.com/pubs/pub36356.html[Google's Dapper].
 +
 +==== Tracers
 +To collect traces, Accumulo needs at least one server listed in
 + +$ACCUMULO_HOME/conf/tracers+. The server collects traces
 +from clients and writes them to the +trace+ table. The Accumulo
 +user that the tracer connects to Accumulo with can be configured with
 +the following properties
 +
 +  trace.user
 +  trace.token.property.password
 +
 +==== Instrumenting a Client
 +Tracing can be used to measure a client operation, such as a scan, as
 +the operation traverses the distributed system. To enable tracing for
 +your application call
 +
 +[source,java]
 +DistributedTrace.enable(instance, new ZooReader(instance), hostname, "myApplication");
 +
 +Once tracing has been enabled, a client can wrap an operation in a trace.
 +
 +[source,java]
 +Trace.on("Client Scan");
 +BatchScanner scanner = conn.createBatchScanner(...);
 +// Configure your scanner
 +for (Entry entry : scanner) {
 +}
 +Trace.off();
 +
 +Additionally, the user can create additional Spans within a Trace.
 +
 +[source,java]
 +Trace.on("Client Update");
 +...
 +Span readSpan = Trace.start("Read");
 +...
 +readSpan.stop();
 +...
 +Span writeSpan = Trace.start("Write");
 +...
 +writeSpan.stop();
 +Trace.off();
 +
 +Like Dapper, Accumulo tracing supports user defined annotations to associate additional data with a Trace.
 +
 +[source,java]
 +...
 +int numberOfEntriesRead = 0;
 +Span readSpan = Trace.start("Read");
 +// Do the read, update the counter
 +...
 +readSpan.data("Number of Entries Read", String.valueOf(numberOfEntriesRead));
 +
 +Some client operations may have a high volume within your
 +application. As such, you may wish to only sample a percentage of
 +operations for tracing. As seen below, the CountSampler can be used to
 +help enable tracing for 1-in-1000 operations
 +
 +[source,java]
 +Sampler sampler = new CountSampler(1000);
 +...
 +if (sampler.next()) {
 +  Trace.on("Read");
 +}
 +...
 +Trace.offNoFlush();
 +
 +It should be noted that it is safe to turn off tracing even if it
 +isn't currently active. The +Trace.offNoFlush()+ should be used if the
 +user does not wish to have +Trace.off()+ block while flushing trace
 +data.
 +
 +==== Viewing Collected Traces
 +To view collected traces, use the "Recent Traces" link on the Monitor
 +UI. You can also programmatically access and print traces using the
 ++TraceDump+ class.
 +
 +==== Tracing from the Shell
 +You can enable tracing for operations run from the shell by using the
 ++trace on+ and +trace off+ commands.
 +
 +----
 +root@test test> trace on
 +
 +root@test test> scan
 +a b:c []    d
 +
 +root@test test> trace off
 +Waiting for trace information
 +Waiting for trace information
 +Trace started at 2013/08/26 13:24:08.332
 +Time  Start  Service@Location       Name
 + 3628+0      shell@localhost shell:root
 +    8+1690     shell@localhost scan
 +    7+1691       shell@localhost scan:location
 +    6+1692         tserver@localhost startScan
 +    5+1692           tserver@localhost tablet read ahead 6
 +----
 +
 +=== Logging
 +Accumulo processes each write to a set of log files. By default these are found under
 ++$ACCUMULO/logs/+.
 +
 +=== Recovery
 +
 +In the event of TabletServer failure or error on shutting Accumulo down, some
 +mutations may not have been minor compacted to HDFS properly. In this case,
 +Accumulo will automatically reapply such mutations from the write-ahead log
 +either when the tablets from the failed server are reassigned by the Master (in the
 +case of a single TabletServer failure) or the next time Accumulo starts (in the event of
 +failure during shutdown).
 +
 +Recovery is performed by asking a tablet server to sort the logs so that tablets can easily find their missing
 +updates. The sort status of each file is displayed on
 +Accumulo monitor status page. Once the recovery is complete any
 +tablets involved should return to an ``online'' state. Until then those tablets will be
 +unavailable to clients.
 +
 +The Accumulo client library is configured to retry failed mutations and in many
 +cases clients will be able to continue processing after the recovery process without
 +throwing an exception.