You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by cj...@apache.org on 2014/09/19 02:38:13 UTC

[1/6] git commit: ACCUMULO-3078 ACCUMULO-3138 Remove the getStaticCluster reference again

Repository: accumulo
Updated Branches:
  refs/heads/1.6.1-rc1 d439f92ab -> 723443e7d (forced update)


ACCUMULO-3078 ACCUMULO-3138 Remove the getStaticCluster reference again

The AccumuloConfiguration from the Instance is at least the
ClientConfiguration. We can use that to remove the reliance
on needing the shared MiniAccumuloCluster.


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

Branch: refs/heads/1.6.1-rc1
Commit: 255ce5f25164dc1c429ee57bbeecfdb1f976cacb
Parents: d0f95f8
Author: Josh Elser <el...@apache.org>
Authored: Thu Sep 18 12:23:26 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Thu Sep 18 12:23:26 2014 -0400

----------------------------------------------------------------------
 .../accumulo/test/functional/AccumuloInputFormatIT.java     | 9 ++++++---
 1 file changed, 6 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/255ce5f2/test/src/test/java/org/apache/accumulo/test/functional/AccumuloInputFormatIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/AccumuloInputFormatIT.java b/test/src/test/java/org/apache/accumulo/test/functional/AccumuloInputFormatIT.java
index acd7440..a3427bb 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/AccumuloInputFormatIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/AccumuloInputFormatIT.java
@@ -36,6 +36,8 @@ import org.apache.accumulo.core.client.TableExistsException;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.mapreduce.AccumuloInputFormat;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
+import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Mutation;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
@@ -71,10 +73,11 @@ public class AccumuloInputFormatIT extends SimpleMacIT {
     insertData(table, currentTimeMillis());
 
     ClientConfiguration clientConf = new ClientConfiguration().withInstance(conn.getInstance().getInstanceName()).withZkHosts(
-        conn.getInstance().getZooKeepers()), clusterClientConf = getStaticCluster().getClientConfig();
+        conn.getInstance().getZooKeepers());
+    AccumuloConfiguration clusterClientConf = conn.getInstance().getConfiguration();
 
     // Pass SSL and CredentialProvider options into the ClientConfiguration given to AccumuloInputFormat
-    boolean sslEnabled = Boolean.valueOf(clusterClientConf.get(ClientProperty.INSTANCE_RPC_SSL_ENABLED));
+    boolean sslEnabled = Boolean.valueOf(clusterClientConf.get(Property.INSTANCE_RPC_SSL_ENABLED));
     if (sslEnabled) {
       ClientProperty[] sslProperties = new ClientProperty[] {ClientProperty.INSTANCE_RPC_SSL_ENABLED, ClientProperty.INSTANCE_RPC_SSL_CLIENT_AUTH,
           ClientProperty.RPC_SSL_KEYSTORE_PATH, ClientProperty.RPC_SSL_KEYSTORE_TYPE, ClientProperty.RPC_SSL_KEYSTORE_PASSWORD,
@@ -83,7 +86,7 @@ public class AccumuloInputFormatIT extends SimpleMacIT {
 
       for (ClientProperty prop : sslProperties) {
         // The default property is returned if it's not in the ClientConfiguration so we don't have to check if the value is actually defined
-        clientConf.setProperty(prop, clusterClientConf.get(prop));
+        clientConf.setProperty(prop, clusterClientConf.get(prop.getAccumuloProperty()));
       }
     }
 


[2/6] git commit: ACCUMULO-3144 Give the servers some time to flush out their buffers to the MAC log pipe.

Posted by cj...@apache.org.
ACCUMULO-3144 Give the servers some time to flush out their buffers to the MAC log pipe.


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

Branch: refs/heads/1.6.1-rc1
Commit: 32570d5a554503d7af6c76379edd8d46175637e2
Parents: 255ce5f
Author: Josh Elser <el...@apache.org>
Authored: Thu Sep 18 14:33:21 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Thu Sep 18 14:33:21 2014 -0400

----------------------------------------------------------------------
 .../java/org/apache/accumulo/test/AuditMessageIT.java   | 12 ++++++++++--
 1 file changed, 10 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/32570d5a/test/src/test/java/org/apache/accumulo/test/AuditMessageIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/AuditMessageIT.java b/test/src/test/java/org/apache/accumulo/test/AuditMessageIT.java
index 1730b14..3ed1f02 100644
--- a/test/src/test/java/org/apache/accumulo/test/AuditMessageIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/AuditMessageIT.java
@@ -70,7 +70,8 @@ public class AuditMessageIT extends ConfigurableMacIT {
   private static final String NEW_TEST_TABLE_NAME = "oranges";
   private static final String THIRD_TEST_TABLE_NAME = "pears";
   private static final Authorizations auths = new Authorizations("private", "public");
-  
+
+  @Override
   public void beforeClusterStart(MiniAccumuloConfigImpl cfg) throws Exception {
     new File(cfg.getConfDir(), "auditLog.xml").delete();
   }
@@ -91,12 +92,19 @@ public class AuditMessageIT extends ConfigurableMacIT {
 
   /**
    * Returns a List of Audit messages that have been grep'd out of the MiniAccumuloCluster output.
-   * 
+   *
    * @param stepName
    *          A unique name for the test being executed, to identify the System.out messages.
    * @return A List of the Audit messages, sorted (so in chronological order).
    */
   private ArrayList<String> getAuditMessages(String stepName) throws IOException {
+    // ACCUMULO-3144 Make sure we give the processes enough time to flush the write buffer
+    try {
+      Thread.sleep(2000);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new IOException("Interrupted waiting for data to be flushed to output streams");
+    }
 
     for (MiniAccumuloClusterImpl.LogWriter lw : getCluster().getLogWriters()) {
       lw.flush();


[3/6] git commit: ACCUMULO-3145 Fix loop invariants in listscans

Posted by cj...@apache.org.
ACCUMULO-3145 Fix loop invariants in listscans

Added some extra logging and (automatically) fixed formatting.


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

Branch: refs/heads/1.6.1-rc1
Commit: 992414ebba8109b601846b5c279bf72b8165485d
Parents: 32570d5
Author: Josh Elser <el...@apache.org>
Authored: Thu Sep 18 15:01:02 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Thu Sep 18 15:01:02 2014 -0400

----------------------------------------------------------------------
 .../org/apache/accumulo/test/ShellServerIT.java | 89 +++++++++++---------
 1 file changed, 48 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/992414eb/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java b/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
index 1cd787e..7449148 100644
--- a/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/ShellServerIT.java
@@ -94,7 +94,7 @@ public class ShellServerIT extends SimpleMacIT {
       sb.setLength(0);
     }
   }
-  
+
   private static final Logger log = Logger.getLogger(ShellServerIT.class);
 
   public static class StringInputStream extends InputStream {
@@ -118,10 +118,11 @@ public class ShellServerIT extends SimpleMacIT {
   private static abstract class ErrorMessageCallback {
     public abstract String getErrorMessage();
   }
-  
+
   private static class NoOpErrorMessageCallback extends ErrorMessageCallback {
     private static final String empty = "";
-    public String getErrorMessage() { 
+    @Override
+    public String getErrorMessage() {
       return empty;
     }
   }
@@ -235,7 +236,7 @@ public class ShellServerIT extends SimpleMacIT {
     // give the tracer some time to start
     UtilWaitThread.sleep(1000);
   }
-  
+
   @Before
   public void setupShell() throws Exception {
     ts = new TestShell(ROOT_PASSWORD, getStaticCluster().getConfig().getInstanceName(), getStaticCluster().getConfig().getZooKeepers(),
@@ -268,7 +269,7 @@ public class ShellServerIT extends SimpleMacIT {
   @Test(timeout = 60000)
   public void exporttableImporttable() throws Exception {
     final String table = name.getMethodName(), table2 = table + "2";
-    
+
     // exporttable / importtable
     ts.exec("createtable " + table + " -evc", true);
     make10();
@@ -312,7 +313,7 @@ public class ShellServerIT extends SimpleMacIT {
   @Test(timeout = 45000)
   public void setscaniterDeletescaniter() throws Exception {
     final String table = name.getMethodName();
-    
+
     // setscaniter, deletescaniter
     ts.exec("createtable " + table);
     ts.exec("insert a cf cq 1");
@@ -341,7 +342,7 @@ public class ShellServerIT extends SimpleMacIT {
   @Test(timeout = 45000)
   public void egrep() throws Exception {
     final String table = name.getMethodName();
-    
+
     // egrep
     ts.exec("createtable " + table);
     make10();
@@ -353,7 +354,7 @@ public class ShellServerIT extends SimpleMacIT {
   @Test(timeout = 45000)
   public void du() throws Exception {
     final String table = name.getMethodName();
-    
+
     // create and delete a table so we get out of a table context in the shell
     ts.exec("notable", true);
 
@@ -388,7 +389,7 @@ public class ShellServerIT extends SimpleMacIT {
   @Test(timeout = 45000)
   public void user() throws Exception {
     final String table = name.getMethodName();
-    
+
     // createuser, deleteuser, user, users, droptable, grant, revoke
     ts.input.set("secret\nsecret\n");
     ts.exec("createuser xyzzy", true);
@@ -424,7 +425,7 @@ public class ShellServerIT extends SimpleMacIT {
   @Test(timeout = 45000)
   public void iter() throws Exception {
     final String table = name.getMethodName();
-    
+
     // setshelliter, listshelliter, deleteshelliter
     ts.exec("createtable " + table);
     ts.exec("insert a cf cq 1");
@@ -557,7 +558,7 @@ public class ShellServerIT extends SimpleMacIT {
   @Test(timeout = 45000)
   public void notable() throws Exception {
     final String table = name.getMethodName();
-    
+
     // notable
     ts.exec("createtable " + table, true);
     ts.exec("scan", true, " " + table + ">", true);
@@ -587,6 +588,7 @@ public class ShellServerIT extends SimpleMacIT {
     for (int i = 0; i < 9 && !success; i++) {
       try {
         ts.exec("insert a b c d -l foo", false, "does not have authorization", true, new ErrorMessageCallback() {
+          @Override
           public String getErrorMessage() {
             try {
               Connector c = getConnector();
@@ -602,6 +604,7 @@ public class ShellServerIT extends SimpleMacIT {
     }
     if (!success) {
       ts.exec("insert a b c d -l foo", false, "does not have authorization", true, new ErrorMessageCallback() {
+        @Override
         public String getErrorMessage() {
           try {
             Connector c = getConnector();
@@ -662,7 +665,7 @@ public class ShellServerIT extends SimpleMacIT {
   @Test(timeout = 45000)
   public void clonetable() throws Exception {
     final String table = name.getMethodName(), clone = table + "_clone";
-    
+
     // clonetable
     ts.exec("createtable " + table + " -evc");
     ts.exec("config -t " + table + " -s table.split.threshold=123M", true);
@@ -677,16 +680,16 @@ public class ShellServerIT extends SimpleMacIT {
     ts.exec("deletetable -f " + table);
     ts.exec("deletetable -f " + clone);
   }
-  
+
   @Test(timeout = 45000)
   public void testCompactions() throws Exception {
     final String table = name.getMethodName();
-    
+
     // compact
     ts.exec("createtable " + table);
-    
+
     String tableId = getTableId(table);
-    
+
     // make two files
     ts.exec("insert a b c d");
     ts.exec("flush -w");
@@ -706,7 +709,7 @@ public class ShellServerIT extends SimpleMacIT {
 
     // at this point there are 4 files in the default tablet
     assertEquals("Files that were found: " + oldFiles, 4, oldFiles.size());
-    
+
     // compact some data:
     ts.exec("compact -b g -e z -w");
     assertEquals(2, countFiles(tableId));
@@ -721,7 +724,7 @@ public class ShellServerIT extends SimpleMacIT {
   @Test(timeout = 45000)
   public void constraint() throws Exception {
     final String table = name.getMethodName();
-    
+
     // constraint
     ts.exec("constraint -l -t " + MetadataTable.NAME + "", true, "MetadataConstraints=1", true);
     ts.exec("createtable " + table + " -evc");
@@ -740,7 +743,7 @@ public class ShellServerIT extends SimpleMacIT {
   @Test(timeout = 45000)
   public void deletemany() throws Exception {
     final String table = name.getMethodName();
-    
+
     // deletemany
     ts.exec("createtable " + table);
     make10();
@@ -772,11 +775,11 @@ public class ShellServerIT extends SimpleMacIT {
 
     ts.exec("createtable " + table);
     final String tableId = getTableId(table);
-    
+
     // deleterows
     int base = countFiles(tableId);
     assertEquals(0, base);
-    
+
     ts.exec("addsplits row5 row7");
     make10();
     ts.exec("flush -w -t " + table);
@@ -801,7 +804,7 @@ public class ShellServerIT extends SimpleMacIT {
   @Test(timeout = 45000)
   public void groups() throws Exception {
     final String table = name.getMethodName();
-    
+
     ts.exec("createtable " + table);
     ts.exec("setgroups -t " + table + " alpha=a,b,c num=3,2,1");
     ts.exec("getgroups -t " + table, true, "alpha=a,b,c", true);
@@ -812,7 +815,7 @@ public class ShellServerIT extends SimpleMacIT {
   @Test(timeout = 45000)
   public void grep() throws Exception {
     final String table = name.getMethodName();
-    
+
     ts.exec("createtable " + table, true);
     make10();
     ts.exec("grep row[123]", true, "row1", false);
@@ -836,7 +839,7 @@ public class ShellServerIT extends SimpleMacIT {
   // @Test(timeout = 45000)
   public void history() throws Exception {
     final String table = name.getMethodName();
-    
+
     ts.exec("history -c", true);
     ts.exec("createtable " + table);
     ts.exec("deletetable -f " + table);
@@ -847,7 +850,7 @@ public class ShellServerIT extends SimpleMacIT {
   @Test(timeout = 45000)
   public void importDirectory() throws Exception {
     final String table = name.getMethodName();
-    
+
     Configuration conf = new Configuration();
     FileSystem fs = FileSystem.get(conf);
     File importDir = new File(getFolder(), "import");
@@ -890,7 +893,7 @@ public class ShellServerIT extends SimpleMacIT {
   @Test(timeout = 45000)
   public void interpreter() throws Exception {
     final String table = name.getMethodName();
-    
+
     ts.exec("createtable " + table, true);
     ts.exec("interpreter -l", true, "HexScan", false);
     ts.exec("insert \\x02 cf cq value", true);
@@ -907,7 +910,7 @@ public class ShellServerIT extends SimpleMacIT {
   @Test(timeout = 45000)
   public void listcompactions() throws Exception {
     final String table = name.getMethodName();
-    
+
     ts.exec("createtable " + table, true);
     ts.exec("config -t " + table + " -s table.iterator.minc.slow=30,org.apache.accumulo.test.functional.SlowIterator", true);
     ts.exec("config -t " + table + " -s table.iterator.minc.slow.opt.sleepTime=1000", true);
@@ -928,7 +931,7 @@ public class ShellServerIT extends SimpleMacIT {
   @Test(timeout = 45000)
   public void maxrow() throws Exception {
     final String table = name.getMethodName();
-    
+
     ts.exec("createtable " + table, true);
     ts.exec("insert a cf cq value", true);
     ts.exec("insert b cf cq value", true);
@@ -943,7 +946,7 @@ public class ShellServerIT extends SimpleMacIT {
   @Test(timeout = 45000)
   public void merge() throws Exception {
     final String table = name.getMethodName();
-    
+
     ts.exec("createtable " + table);
     ts.exec("addsplits a m z");
     ts.exec("getsplits", true, "z", true);
@@ -975,7 +978,7 @@ public class ShellServerIT extends SimpleMacIT {
   @Test(timeout = 45000)
   public void renametable() throws Exception {
     final String table = name.getMethodName() + "1", rename = name.getMethodName() + "2";
-    
+
     ts.exec("createtable " + table);
     ts.exec("insert this is a value");
     ts.exec("renametable " + table + " " + rename);
@@ -1008,7 +1011,7 @@ public class ShellServerIT extends SimpleMacIT {
   @Test(timeout = 45000)
   public void listscans() throws Exception {
     final String table = name.getMethodName();
-    
+
     ts.exec("createtable " + table, true);
 
     // Should be about a 3 second scan
@@ -1037,11 +1040,15 @@ public class ShellServerIT extends SimpleMacIT {
     // Try to find the active scan for about 15seconds
     for (int i = 0; i < 50 && scans.isEmpty(); i++) {
       String currentScans = ts.exec("listscans", true);
+      log.info("Got output from listscans:\n" + currentScans);
       String[] lines = currentScans.split("\n");
-      for (int scanOffset = 2; i < lines.length; i++) {
+      for (int scanOffset = 2; scanOffset < lines.length; scanOffset++) {
         String currentScan = lines[scanOffset];
         if (currentScan.contains(table)) {
+          log.info("Retaining scan: " + currentScan);
           scans.add(currentScan);
+        } else {
+          log.info("Ignoring scan because of wrong table: " + currentScan);
         }
       }
       UtilWaitThread.sleep(300);
@@ -1062,7 +1069,7 @@ public class ShellServerIT extends SimpleMacIT {
       String client = parts[1].trim();
       assertTrue(client.matches(hostPortPattern));
     }
-    
+
     ts.exec("deletetable -f " + table, true);
   }
 
@@ -1071,7 +1078,7 @@ public class ShellServerIT extends SimpleMacIT {
     final String table = name.getMethodName();
 
     File fooFilterJar = File.createTempFile("FooFilter", ".jar", getFolder());
-    
+
     FileUtils.copyURLToFile(this.getClass().getResource("/FooFilter.jar"), fooFilterJar);
     fooFilterJar.deleteOnExit();
 
@@ -1117,7 +1124,7 @@ public class ShellServerIT extends SimpleMacIT {
   public void trace() throws Exception {
     // Make sure to not collide with the "trace" table
     final String table = name.getMethodName() + "Test";
-    
+
     ts.exec("trace on", true);
     ts.exec("createtable " + table, true);
     ts.exec("insert a b c value", true);
@@ -1253,14 +1260,14 @@ public class ShellServerIT extends SimpleMacIT {
       ts.exec(String.format("insert row%d cf col%d value", i, i));
     }
   }
-  
+
   private List<String> getFiles(String tableId) throws IOException {
     ts.output.clear();
 
     ts.exec("scan -t " + MetadataTable.NAME + " -np -c file -b " + tableId + " -e " + tableId + "~");
-    
+
     log.debug("countFiles(): " + ts.output.get());
-    
+
     String[] lines = StringUtils.split(ts.output.get(), "\n");
     ts.output.clear();
 
@@ -1274,10 +1281,10 @@ public class ShellServerIT extends SimpleMacIT {
   private int countFiles(String tableId) throws IOException {
     return getFiles(tableId).size();
   }
-  
+
   private String getTableId(String tableName) throws Exception {
     Connector conn = getConnector();
-    
+
     for (int i = 0; i < 5; i++) {
       Map<String,String> nameToId = conn.tableOperations().tableIdMap();
       if (nameToId.containsKey(tableName)) {
@@ -1286,7 +1293,7 @@ public class ShellServerIT extends SimpleMacIT {
         Thread.sleep(1000);
       }
     }
-    
+
     fail("Could not find ID for table: " + tableName);
     // Will never get here
     return null;


[4/6] git commit: ACCUMULO-3146 Try to stabilize BalanceAfterCommsFailureIT and SimpleBalancerFairnessIT

Posted by cj...@apache.org.
ACCUMULO-3146 Try to stabilize BalanceAfterCommsFailureIT and SimpleBalancerFairnessIT


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

Branch: refs/heads/1.6.1-rc1
Commit: 7a40f7bfa4c6e30467a24105da974687ac442cdf
Parents: 992414e
Author: Josh Elser <el...@apache.org>
Authored: Thu Sep 18 18:12:12 2014 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Thu Sep 18 18:12:12 2014 -0400

----------------------------------------------------------------------
 .../functional/BalanceAfterCommsFailureIT.java  | 30 +++++++++----
 .../functional/SimpleBalancerFairnessIT.java    | 44 ++++++++++++++------
 2 files changed, 53 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/7a40f7bf/test/src/test/java/org/apache/accumulo/test/functional/BalanceAfterCommsFailureIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/BalanceAfterCommsFailureIT.java b/test/src/test/java/org/apache/accumulo/test/functional/BalanceAfterCommsFailureIT.java
index 39e7e70..7b4a774 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/BalanceAfterCommsFailureIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/BalanceAfterCommsFailureIT.java
@@ -92,22 +92,33 @@ public class BalanceAfterCommsFailureIT extends ConfigurableMacIT {
     c.tableOperations().addSplits("test", splits);
     // Ensure all of the tablets are actually assigned
     assertEquals(0, Iterables.size(c.createScanner("test", Authorizations.EMPTY)));
-    UtilWaitThread.sleep(10 * 1000);
+    UtilWaitThread.sleep(30 * 1000);
     checkBalance(c);
   }
 
   private void checkBalance(Connector c) throws Exception {
     Credentials creds = new Credentials("root", new PasswordToken(ROOT_PASSWORD));
 
-    MasterClientService.Iface client = null;
     MasterMonitorInfo stats = null;
-    try {
-      client = MasterClient.getConnectionWithRetry(c.getInstance());
-      stats = client.getMasterStats(Tracer.traceInfo(), creds.toThrift(c.getInstance()));
-    } finally {
-      if (client != null)
-        MasterClient.close(client);
+    int unassignedTablets = 1;
+    for (int i = 0; unassignedTablets > 0 && i < 10; i++) {
+      MasterClientService.Iface client = null;
+      try {
+        client = MasterClient.getConnectionWithRetry(c.getInstance());
+        stats = client.getMasterStats(Tracer.traceInfo(), creds.toThrift(c.getInstance()));
+      } finally {
+        if (client != null)
+          MasterClient.close(client);
+      }
+      unassignedTablets = stats.getUnassignedTablets();
+      if (unassignedTablets > 0) {
+        log.info("Found " + unassignedTablets + " unassigned tablets, sleeping 3 seconds for tablet assignment");
+        Thread.sleep(3000);
+      }
     }
+
+    assertEquals("Unassigned tablets were not assigned within 30 seconds", 0, unassignedTablets);
+
     List<Integer> counts = new ArrayList<Integer>();
     for (TabletServerStatus server : stats.tServerInfo) {
       int count = 0;
@@ -119,7 +130,8 @@ public class BalanceAfterCommsFailureIT extends ConfigurableMacIT {
     assertTrue("Expected to have at least two TabletServers", counts.size() > 1);
     for (int i = 1; i < counts.size(); i++) {
       int diff = Math.abs(counts.get(0) - counts.get(i));
-      assertTrue("Expected difference in tablets to be less than or equal to " + counts.size() + " but was " + diff, diff <= counts.size());
+      assertTrue("Expected difference in tablets to be less than or equal to " + counts.size() + " but was " + diff + ". Counts " + counts,
+          diff <= counts.size());
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7a40f7bf/test/src/test/java/org/apache/accumulo/test/functional/SimpleBalancerFairnessIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/SimpleBalancerFairnessIT.java b/test/src/test/java/org/apache/accumulo/test/functional/SimpleBalancerFairnessIT.java
index 6904563..7d37615 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/SimpleBalancerFairnessIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/SimpleBalancerFairnessIT.java
@@ -16,12 +16,14 @@
  */
 package org.apache.accumulo.test.functional;
 
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.TreeSet;
 
 import org.apache.accumulo.core.cli.BatchWriterOpts;
 import org.apache.accumulo.core.client.Connector;
@@ -38,6 +40,7 @@ import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
 import org.apache.accumulo.test.TestIngest;
 import org.apache.accumulo.trace.instrument.Tracer;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Text;
 import org.junit.Test;
 
 public class SimpleBalancerFairnessIT extends ConfigurableMacIT {
@@ -61,24 +64,38 @@ public class SimpleBalancerFairnessIT extends ConfigurableMacIT {
     c.tableOperations().create("test_ingest");
     c.tableOperations().setProperty("test_ingest", Property.TABLE_SPLIT_THRESHOLD.getKey(), "10K");
     c.tableOperations().create("unused");
-    c.tableOperations().addSplits("unused", TestIngest.getSplitPoints(0, 10000000, 2000));
+    TreeSet<Text> splits = TestIngest.getSplitPoints(0, 10000000, 2000);
+    log.info("Creating " + splits.size() + " splits");
+    c.tableOperations().addSplits("unused", splits);
     List<String> tservers = c.instanceOperations().getTabletServers();
     TestIngest.Opts opts = new TestIngest.Opts();
     opts.rows = 200000;
     TestIngest.ingest(c, opts, new BatchWriterOpts());
     c.tableOperations().flush("test_ingest", null, null, false);
-    UtilWaitThread.sleep(15 * 1000);
+    UtilWaitThread.sleep(45 * 1000);
     Credentials creds = new Credentials("root", new PasswordToken(ROOT_PASSWORD));
 
-    MasterClientService.Iface client = null;
     MasterMonitorInfo stats = null;
-    try {
-      client = MasterClient.getConnectionWithRetry(c.getInstance());
-      stats = client.getMasterStats(Tracer.traceInfo(), creds.toThrift(c.getInstance()));
-    } finally {
-      if (client != null)
-        MasterClient.close(client);
+    int unassignedTablets = 1;
+    for (int i = 0; unassignedTablets > 0 && i < 10; i++) {
+      MasterClientService.Iface client = null;
+      try {
+        client = MasterClient.getConnectionWithRetry(c.getInstance());
+        stats = client.getMasterStats(Tracer.traceInfo(), creds.toThrift(c.getInstance()));
+      } finally {
+        if (client != null)
+          MasterClient.close(client);
+      }
+      unassignedTablets = stats.getUnassignedTablets();
+      if (unassignedTablets > 0) {
+        log.info("Found " + unassignedTablets + " unassigned tablets, sleeping 3 seconds for tablet assignment");
+        Thread.sleep(3000);
+      }
     }
+
+    assertEquals("Unassigned tablets were not assigned within 30 seconds", 0, unassignedTablets);
+
+    // Compute online tablets per tserver
     List<Integer> counts = new ArrayList<Integer>();
     for (TabletServerStatus server : stats.tServerInfo) {
       int count = 0;
@@ -87,9 +104,12 @@ public class SimpleBalancerFairnessIT extends ConfigurableMacIT {
       }
       counts.add(count);
     }
-    assertTrue(counts.size() > 1);
-    for (int i = 1; i < counts.size(); i++)
-      assertTrue(Math.abs(counts.get(0) - counts.get(i)) <= tservers.size());
+    assertTrue("Expected to have at least two TabletServers", counts.size() > 1);
+    for (int i = 1; i < counts.size(); i++) {
+      int diff = Math.abs(counts.get(0) - counts.get(i));
+      assertTrue("Expected difference in tablets to be less than or equal to " + counts.size() + " but was " + diff + ". Counts " + counts,
+          diff <= tservers.size());
+    }
   }
 
 }


[5/6] git commit: ACCUMULO-3143 Offline scan property is now being properly serialized. Added verification that all other booleans are being serialized.

Posted by cj...@apache.org.
ACCUMULO-3143 Offline scan property is now being properly serialized. Added verification that all other booleans are being serialized.


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

Branch: refs/heads/1.6.1-rc1
Commit: ddd2c3bc098531d8dadee1b5142cfb80c50f4c41
Parents: 7a40f7b
Author: Corey J. Nolet <cj...@gmail.com>
Authored: Thu Sep 18 20:11:41 2014 -0400
Committer: Corey J. Nolet <cj...@gmail.com>
Committed: Thu Sep 18 20:14:30 2014 -0400

----------------------------------------------------------------------
 .../core/client/mapreduce/InputTableConfig.java |  2 ++
 .../client/mapreduce/InputTableConfigTest.java  | 20 ++++++++++++++++----
 2 files changed, 18 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/ddd2c3bc/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputTableConfig.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputTableConfig.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputTableConfig.java
index e59451e..fa3b7eb 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputTableConfig.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputTableConfig.java
@@ -281,6 +281,7 @@ public class InputTableConfig implements Writable {
     dataOutput.writeBoolean(autoAdjustRanges);
     dataOutput.writeBoolean(useLocalIterators);
     dataOutput.writeBoolean(useIsolatedScanners);
+    dataOutput.writeBoolean(offlineScan);
   }
 
   /**
@@ -325,6 +326,7 @@ public class InputTableConfig implements Writable {
     autoAdjustRanges = dataInput.readBoolean();
     useLocalIterators = dataInput.readBoolean();
     useIsolatedScanners = dataInput.readBoolean();
+    offlineScan = dataInput.readBoolean();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ddd2c3bc/core/src/test/java/org/apache/accumulo/core/client/mapreduce/InputTableConfigTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/InputTableConfigTest.java b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/InputTableConfigTest.java
index 7f5c7d8..4855094 100644
--- a/core/src/test/java/org/apache/accumulo/core/client/mapreduce/InputTableConfigTest.java
+++ b/core/src/test/java/org/apache/accumulo/core/client/mapreduce/InputTableConfigTest.java
@@ -16,8 +16,6 @@
  */
 package org.apache.accumulo.core.client.mapreduce;
 
-import static org.junit.Assert.assertEquals;
-
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.DataInputStream;
@@ -29,13 +27,14 @@ import java.util.List;
 import java.util.Set;
 
 import org.apache.accumulo.core.client.IteratorSetting;
-import org.apache.accumulo.core.client.mapreduce.InputTableConfig;
 import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.hadoop.io.Text;
 import org.junit.Before;
 import org.junit.Test;
 
+import static org.junit.Assert.assertEquals;
+
 public class InputTableConfigTest {
   
   private InputTableConfig tableQueryConfig;
@@ -52,7 +51,20 @@ public class InputTableConfigTest {
     
     assertEquals(tableQueryConfig, actualConfig);
   }
-  
+
+  @Test
+  public void testSerialization_allBooleans() throws IOException {
+    tableQueryConfig.setAutoAdjustRanges(false);
+    tableQueryConfig.setOfflineScan(true);
+    tableQueryConfig.setUseIsolatedScanners(true);
+    tableQueryConfig.setUseLocalIterators(true);
+    byte[] serialized = serialize(tableQueryConfig);
+    InputTableConfig actualConfig = deserialize(serialized);
+
+    assertEquals(tableQueryConfig, actualConfig);
+  }
+
+
   @Test
   public void testSerialization_ranges() throws IOException {
     List<Range> ranges = new ArrayList<Range>();


[6/6] git commit: ACCUMULO-3142 Setting version to 1.6.1

Posted by cj...@apache.org.
ACCUMULO-3142 Setting version to 1.6.1


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

Branch: refs/heads/1.6.1-rc1
Commit: 723443e7da0a5c533ee03e59a3ce5ccea978c7d0
Parents: ddd2c3b
Author: Corey J. Nolet <cj...@gmail.com>
Authored: Wed Sep 17 22:57:18 2014 -0400
Committer: Corey J. Nolet <cj...@gmail.com>
Committed: Thu Sep 18 20:39:43 2014 -0400

----------------------------------------------------------------------
 assemble/pom.xml        | 2 +-
 core/pom.xml            | 2 +-
 docs/pom.xml            | 2 +-
 examples/simple/pom.xml | 2 +-
 fate/pom.xml            | 2 +-
 maven-plugin/pom.xml    | 2 +-
 minicluster/pom.xml     | 2 +-
 pom.xml                 | 2 +-
 proxy/pom.xml           | 2 +-
 server/base/pom.xml     | 2 +-
 server/gc/pom.xml       | 2 +-
 server/master/pom.xml   | 2 +-
 server/monitor/pom.xml  | 2 +-
 server/native/pom.xml   | 2 +-
 server/tracer/pom.xml   | 2 +-
 server/tserver/pom.xml  | 2 +-
 start/pom.xml           | 2 +-
 test/pom.xml            | 2 +-
 trace/pom.xml           | 2 +-
 19 files changed, 19 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/723443e7/assemble/pom.xml
----------------------------------------------------------------------
diff --git a/assemble/pom.xml b/assemble/pom.xml
index 81afc0f..6c88765 100644
--- a/assemble/pom.xml
+++ b/assemble/pom.xml
@@ -20,7 +20,7 @@
   <parent>
     <groupId>org.apache.accumulo</groupId>
     <artifactId>accumulo-project</artifactId>
-    <version>1.6.1-SNAPSHOT</version>
+    <version>1.6.1</version>
   </parent>
   <artifactId>accumulo</artifactId>
   <packaging>pom</packaging>

http://git-wip-us.apache.org/repos/asf/accumulo/blob/723443e7/core/pom.xml
----------------------------------------------------------------------
diff --git a/core/pom.xml b/core/pom.xml
index 5a8c347..c385ab0 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -20,7 +20,7 @@
   <parent>
     <groupId>org.apache.accumulo</groupId>
     <artifactId>accumulo-project</artifactId>
-    <version>1.6.1-SNAPSHOT</version>
+    <version>1.6.1</version>
   </parent>
   <artifactId>accumulo-core</artifactId>
   <name>Core</name>

http://git-wip-us.apache.org/repos/asf/accumulo/blob/723443e7/docs/pom.xml
----------------------------------------------------------------------
diff --git a/docs/pom.xml b/docs/pom.xml
index a6260c6..9edba52 100644
--- a/docs/pom.xml
+++ b/docs/pom.xml
@@ -20,7 +20,7 @@
   <parent>
     <groupId>org.apache.accumulo</groupId>
     <artifactId>accumulo-project</artifactId>
-    <version>1.6.1-SNAPSHOT</version>
+    <version>1.6.1</version>
   </parent>
   <artifactId>accumulo-docs</artifactId>
   <packaging>pom</packaging>

http://git-wip-us.apache.org/repos/asf/accumulo/blob/723443e7/examples/simple/pom.xml
----------------------------------------------------------------------
diff --git a/examples/simple/pom.xml b/examples/simple/pom.xml
index 472aeb4..fb96420 100644
--- a/examples/simple/pom.xml
+++ b/examples/simple/pom.xml
@@ -20,7 +20,7 @@
   <parent>
     <groupId>org.apache.accumulo</groupId>
     <artifactId>accumulo-project</artifactId>
-    <version>1.6.1-SNAPSHOT</version>
+    <version>1.6.1</version>
     <relativePath>../../pom.xml</relativePath>
   </parent>
   <artifactId>accumulo-examples-simple</artifactId>

http://git-wip-us.apache.org/repos/asf/accumulo/blob/723443e7/fate/pom.xml
----------------------------------------------------------------------
diff --git a/fate/pom.xml b/fate/pom.xml
index dfc3f1c..63bd73d 100644
--- a/fate/pom.xml
+++ b/fate/pom.xml
@@ -20,7 +20,7 @@
   <parent>
     <groupId>org.apache.accumulo</groupId>
     <artifactId>accumulo-project</artifactId>
-    <version>1.6.1-SNAPSHOT</version>
+    <version>1.6.1</version>
   </parent>
   <artifactId>accumulo-fate</artifactId>
   <name>Fate</name>

http://git-wip-us.apache.org/repos/asf/accumulo/blob/723443e7/maven-plugin/pom.xml
----------------------------------------------------------------------
diff --git a/maven-plugin/pom.xml b/maven-plugin/pom.xml
index 9d286d8..502e7ed 100644
--- a/maven-plugin/pom.xml
+++ b/maven-plugin/pom.xml
@@ -20,7 +20,7 @@
   <parent>
     <groupId>org.apache.accumulo</groupId>
     <artifactId>accumulo-project</artifactId>
-    <version>1.6.1-SNAPSHOT</version>
+    <version>1.6.1</version>
   </parent>
   <artifactId>accumulo-maven-plugin</artifactId>
   <packaging>maven-plugin</packaging>

http://git-wip-us.apache.org/repos/asf/accumulo/blob/723443e7/minicluster/pom.xml
----------------------------------------------------------------------
diff --git a/minicluster/pom.xml b/minicluster/pom.xml
index 726f95f..2c6d8d3 100644
--- a/minicluster/pom.xml
+++ b/minicluster/pom.xml
@@ -20,7 +20,7 @@
   <parent>
     <groupId>org.apache.accumulo</groupId>
     <artifactId>accumulo-project</artifactId>
-    <version>1.6.1-SNAPSHOT</version>
+    <version>1.6.1</version>
   </parent>
   <artifactId>accumulo-minicluster</artifactId>
   <name>MiniCluster</name>

http://git-wip-us.apache.org/repos/asf/accumulo/blob/723443e7/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 98501c7..fd8a5b5 100644
--- a/pom.xml
+++ b/pom.xml
@@ -24,7 +24,7 @@
   </parent>
   <groupId>org.apache.accumulo</groupId>
   <artifactId>accumulo-project</artifactId>
-  <version>1.6.1-SNAPSHOT</version>
+  <version>1.6.1</version>
   <packaging>pom</packaging>
   <name>Apache Accumulo</name>
   <description>Apache Accumulo is a sorted, distributed key/value store based on Google's BigTable design. It is built on top of Apache Hadoop, Zookeeper, and Thrift. It features a few novel improvements on the BigTable design in the form of cell-level access labels and a server-side programming mechanism that can modify key/value pairs at various points in the data management process.</description>

http://git-wip-us.apache.org/repos/asf/accumulo/blob/723443e7/proxy/pom.xml
----------------------------------------------------------------------
diff --git a/proxy/pom.xml b/proxy/pom.xml
index 0e72921..a322a3e 100644
--- a/proxy/pom.xml
+++ b/proxy/pom.xml
@@ -20,7 +20,7 @@
   <parent>
     <groupId>org.apache.accumulo</groupId>
     <artifactId>accumulo-project</artifactId>
-    <version>1.6.1-SNAPSHOT</version>
+    <version>1.6.1</version>
   </parent>
   <artifactId>accumulo-proxy</artifactId>
   <name>Proxy</name>

http://git-wip-us.apache.org/repos/asf/accumulo/blob/723443e7/server/base/pom.xml
----------------------------------------------------------------------
diff --git a/server/base/pom.xml b/server/base/pom.xml
index 6eb91a0..0e38565 100644
--- a/server/base/pom.xml
+++ b/server/base/pom.xml
@@ -20,7 +20,7 @@
   <parent>
     <groupId>org.apache.accumulo</groupId>
     <artifactId>accumulo-project</artifactId>
-    <version>1.6.1-SNAPSHOT</version>
+    <version>1.6.1</version>
     <relativePath>../../pom.xml</relativePath>
   </parent>
   <artifactId>accumulo-server-base</artifactId>

http://git-wip-us.apache.org/repos/asf/accumulo/blob/723443e7/server/gc/pom.xml
----------------------------------------------------------------------
diff --git a/server/gc/pom.xml b/server/gc/pom.xml
index a96e9e4..0fd238a 100644
--- a/server/gc/pom.xml
+++ b/server/gc/pom.xml
@@ -20,7 +20,7 @@
   <parent>
     <groupId>org.apache.accumulo</groupId>
     <artifactId>accumulo-project</artifactId>
-    <version>1.6.1-SNAPSHOT</version>
+    <version>1.6.1</version>
     <relativePath>../../pom.xml</relativePath>
   </parent>
   <artifactId>accumulo-gc</artifactId>

http://git-wip-us.apache.org/repos/asf/accumulo/blob/723443e7/server/master/pom.xml
----------------------------------------------------------------------
diff --git a/server/master/pom.xml b/server/master/pom.xml
index 7a848b8..b40fb4b 100644
--- a/server/master/pom.xml
+++ b/server/master/pom.xml
@@ -20,7 +20,7 @@
   <parent>
     <groupId>org.apache.accumulo</groupId>
     <artifactId>accumulo-project</artifactId>
-    <version>1.6.1-SNAPSHOT</version>
+    <version>1.6.1</version>
     <relativePath>../../pom.xml</relativePath>
   </parent>
   <artifactId>accumulo-master</artifactId>

http://git-wip-us.apache.org/repos/asf/accumulo/blob/723443e7/server/monitor/pom.xml
----------------------------------------------------------------------
diff --git a/server/monitor/pom.xml b/server/monitor/pom.xml
index 221d6b3..fcccc89 100644
--- a/server/monitor/pom.xml
+++ b/server/monitor/pom.xml
@@ -20,7 +20,7 @@
   <parent>
     <groupId>org.apache.accumulo</groupId>
     <artifactId>accumulo-project</artifactId>
-    <version>1.6.1-SNAPSHOT</version>
+    <version>1.6.1</version>
     <relativePath>../../pom.xml</relativePath>
   </parent>
   <artifactId>accumulo-monitor</artifactId>

http://git-wip-us.apache.org/repos/asf/accumulo/blob/723443e7/server/native/pom.xml
----------------------------------------------------------------------
diff --git a/server/native/pom.xml b/server/native/pom.xml
index e0ace33..2c35baf 100644
--- a/server/native/pom.xml
+++ b/server/native/pom.xml
@@ -20,7 +20,7 @@
   <parent>
     <groupId>org.apache.accumulo</groupId>
     <artifactId>accumulo-project</artifactId>
-    <version>1.6.1-SNAPSHOT</version>
+    <version>1.6.1</version>
     <relativePath>../../pom.xml</relativePath>
   </parent>
   <artifactId>accumulo-native</artifactId>

http://git-wip-us.apache.org/repos/asf/accumulo/blob/723443e7/server/tracer/pom.xml
----------------------------------------------------------------------
diff --git a/server/tracer/pom.xml b/server/tracer/pom.xml
index 78fc7b4..35ad456 100644
--- a/server/tracer/pom.xml
+++ b/server/tracer/pom.xml
@@ -20,7 +20,7 @@
   <parent>
     <groupId>org.apache.accumulo</groupId>
     <artifactId>accumulo-project</artifactId>
-    <version>1.6.1-SNAPSHOT</version>
+    <version>1.6.1</version>
     <relativePath>../../pom.xml</relativePath>
   </parent>
   <artifactId>accumulo-tracer</artifactId>

http://git-wip-us.apache.org/repos/asf/accumulo/blob/723443e7/server/tserver/pom.xml
----------------------------------------------------------------------
diff --git a/server/tserver/pom.xml b/server/tserver/pom.xml
index d4dc605..8e00b1f 100644
--- a/server/tserver/pom.xml
+++ b/server/tserver/pom.xml
@@ -20,7 +20,7 @@
   <parent>
     <groupId>org.apache.accumulo</groupId>
     <artifactId>accumulo-project</artifactId>
-    <version>1.6.1-SNAPSHOT</version>
+    <version>1.6.1</version>
     <relativePath>../../pom.xml</relativePath>
   </parent>
   <artifactId>accumulo-tserver</artifactId>

http://git-wip-us.apache.org/repos/asf/accumulo/blob/723443e7/start/pom.xml
----------------------------------------------------------------------
diff --git a/start/pom.xml b/start/pom.xml
index 20612c0..e6d0241 100644
--- a/start/pom.xml
+++ b/start/pom.xml
@@ -20,7 +20,7 @@
   <parent>
     <groupId>org.apache.accumulo</groupId>
     <artifactId>accumulo-project</artifactId>
-    <version>1.6.1-SNAPSHOT</version>
+    <version>1.6.1</version>
   </parent>
   <artifactId>accumulo-start</artifactId>
   <name>Start</name>

http://git-wip-us.apache.org/repos/asf/accumulo/blob/723443e7/test/pom.xml
----------------------------------------------------------------------
diff --git a/test/pom.xml b/test/pom.xml
index 196742a..64ee738 100644
--- a/test/pom.xml
+++ b/test/pom.xml
@@ -20,7 +20,7 @@
   <parent>
     <groupId>org.apache.accumulo</groupId>
     <artifactId>accumulo-project</artifactId>
-    <version>1.6.1-SNAPSHOT</version>
+    <version>1.6.1</version>
   </parent>
   <artifactId>accumulo-test</artifactId>
   <name>Testing</name>

http://git-wip-us.apache.org/repos/asf/accumulo/blob/723443e7/trace/pom.xml
----------------------------------------------------------------------
diff --git a/trace/pom.xml b/trace/pom.xml
index db386ff..d705ec7 100644
--- a/trace/pom.xml
+++ b/trace/pom.xml
@@ -20,7 +20,7 @@
   <parent>
     <groupId>org.apache.accumulo</groupId>
     <artifactId>accumulo-project</artifactId>
-    <version>1.6.1-SNAPSHOT</version>
+    <version>1.6.1</version>
   </parent>
   <artifactId>accumulo-trace</artifactId>
   <name>Trace</name>