You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by ec...@apache.org on 2015/05/26 21:50:19 UTC

[1/3] accumulo git commit: ACCUMULO-3423 fixing all the ITs I broke

Repository: accumulo
Updated Branches:
  refs/heads/master 0138249ca -> fdd7d09a0


ACCUMULO-3423 fixing all the ITs I broke


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

Branch: refs/heads/master
Commit: ecf8e07c388054c8715f13112596fbf195034e01
Parents: 56e211a
Author: Eric C. Newton <er...@gmail.com>
Authored: Tue May 26 15:18:14 2015 -0400
Committer: Eric C. Newton <er...@gmail.com>
Committed: Tue May 26 15:18:14 2015 -0400

----------------------------------------------------------------------
 .../accumulo/core/client/impl/Tables.java       |  2 +-
 .../gc/GarbageCollectWriteAheadLogsTest.java    |  3 +-
 .../accumulo/master/tableOps/DeleteTable.java   | 28 +++++++++---
 .../tserver/log/TabletServerLogger.java         |  4 +-
 .../org/apache/accumulo/test/UnusedWALIT.java   | 14 ++++--
 .../java/org/apache/accumulo/test/VolumeIT.java |  5 ++-
 .../accumulo/test/functional/WALSunnyDayIT.java | 46 +++++++++-----------
 ...bageCollectorCommunicatesWithTServersIT.java |  7 ++-
 .../test/replication/ReplicationIT.java         |  9 +++-
 9 files changed, 72 insertions(+), 46 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/ecf8e07c/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java b/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java
index beacea9..cf388bf 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java
@@ -236,7 +236,7 @@ public class Tables {
 
     // We might get null out of ZooCache if this tableID doesn't exist
     if (null == n) {
-      throw new IllegalArgumentException("Table with id " + tableId + " does not exist");
+      throw new IllegalArgumentException(new TableNotFoundException(tableId, null, null));
     }
 
     return new String(n, UTF_8);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ecf8e07c/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 1ab8eb6..a40d390 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
@@ -192,8 +192,7 @@ public class GarbageCollectWriteAheadLogsTest {
     String row = MetadataSchema.ReplicationSection.getRowPrefix() + path.toString();
     String colf = MetadataSchema.ReplicationSection.COLF.toString();
     String colq = "1";
-    Map<Key, Value> replicationWork = Collections.singletonMap(new Key(row, colf, colq), new Value(new byte[0]));
-
+    Map<Key,Value> replicationWork = Collections.singletonMap(new Key(row, colf, colq), new Value(new byte[0]));
 
     GCStatus status = new GCStatus(null, null, null, new GcCycleStats());
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ecf8e07c/server/master/src/main/java/org/apache/accumulo/master/tableOps/DeleteTable.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/tableOps/DeleteTable.java b/server/master/src/main/java/org/apache/accumulo/master/tableOps/DeleteTable.java
index a1158f4..8ee385c 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/tableOps/DeleteTable.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/tableOps/DeleteTable.java
@@ -16,6 +16,7 @@
  */
 package org.apache.accumulo.master.tableOps;
 
+import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.client.impl.Tables;
 import org.apache.accumulo.core.client.impl.thrift.TableOperation;
 import org.apache.accumulo.core.master.state.tables.TableState;
@@ -35,16 +36,31 @@ public class DeleteTable extends MasterRepo {
 
   @Override
   public long isReady(long tid, Master environment) throws Exception {
-    String namespaceId = Tables.getNamespaceId(environment.getInstance(), tableId);
-    return Utils.reserveNamespace(namespaceId, tid, false, false, TableOperation.DELETE) + Utils.reserveTable(tableId, tid, true, true, TableOperation.DELETE);
+    try {
+      String namespaceId = Tables.getNamespaceId(environment.getInstance(), tableId);
+      return Utils.reserveNamespace(namespaceId, tid, false, false, TableOperation.DELETE)
+          + Utils.reserveTable(tableId, tid, true, true, TableOperation.DELETE);
+    } catch (IllegalArgumentException ex) {
+      if (ex.getCause() != null && ex.getCause() instanceof TableNotFoundException) {
+        return 0;
+      }
+      throw ex;
+    }
   }
 
   @Override
   public Repo<Master> call(long tid, Master environment) throws Exception {
-    String namespaceId = Tables.getNamespaceId(environment.getInstance(), tableId);
-    TableManager.getInstance().transitionTableState(tableId, TableState.DELETING);
-    environment.getEventCoordinator().event("deleting table %s ", tableId);
-    return new CleanUp(tableId, namespaceId);
+    try {
+      String namespaceId = Tables.getNamespaceId(environment.getInstance(), tableId);
+      TableManager.getInstance().transitionTableState(tableId, TableState.DELETING);
+      environment.getEventCoordinator().event("deleting table %s ", tableId);
+      return new CleanUp(tableId, namespaceId);
+    } catch (IllegalArgumentException ex) {
+      if (ex.getCause() != null && ex.getCause() instanceof TableNotFoundException) {
+        return null;
+      }
+      throw ex;
+    }
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ecf8e07c/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 4836d99..304b8c3 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
@@ -260,7 +260,7 @@ public class TabletServerLogger {
               // object before trying to create a new one.
               try {
                 alog.close();
-              } catch (IOException e) {
+              } catch (Exception e) {
                 log.error("Failed to close WAL after it failed to open", e);
               }
               // Try to avoid leaving a bunch of empty WALs lying around
@@ -269,7 +269,7 @@ public class TabletServerLogger {
                 if (fs.exists(path)) {
                   fs.delete(path);
                 }
-              } catch (IOException e) {
+              } catch (Exception e) {
                 log.warn("Failed to delete a WAL that failed to open", e);
               }
             }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ecf8e07c/test/src/test/java/org/apache/accumulo/test/UnusedWALIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/UnusedWALIT.java b/test/src/test/java/org/apache/accumulo/test/UnusedWALIT.java
index 8d22ad3..ac68abd 100644
--- a/test/src/test/java/org/apache/accumulo/test/UnusedWALIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/UnusedWALIT.java
@@ -25,6 +25,7 @@ import java.util.UUID;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Key;
@@ -51,6 +52,8 @@ import com.google.common.collect.Iterators;
 // It would be useful to have an IT that will test this situation.
 public class UnusedWALIT extends ConfigurableMacIT {
 
+  private ZooReaderWriter zk;
+
   @Override
   protected void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
     final long logSize = 1024 * 1024 * 10;
@@ -75,16 +78,19 @@ public class UnusedWALIT extends ConfigurableMacIT {
     c.tableOperations().create(bigTable);
     c.tableOperations().create(lilTable);
 
+    Instance i = c.getInstance();
+    zk = new ZooReaderWriter(i.getZooKeepers(), i.getZooKeepersSessionTimeOut(), "");
+
     // put some data in a log that should be replayed for both tables
     writeSomeData(c, bigTable, 0, 10, 0, 10);
     scanSomeData(c, bigTable, 0, 10, 0, 10);
     writeSomeData(c, lilTable, 0, 1, 0, 1);
     scanSomeData(c, lilTable, 0, 1, 0, 1);
-    assertEquals(1, getWALCount(c));
+    assertEquals(2, getWALCount(i, zk));
 
     // roll the logs by pushing data into bigTable
     writeSomeData(c, bigTable, 0, 3000, 0, 1000);
-    assertEquals(2, getWALCount(c));
+    assertEquals(3, getWALCount(i, zk));
 
     // put some data in the latest log
     writeSomeData(c, lilTable, 1, 10, 0, 10);
@@ -121,8 +127,8 @@ public class UnusedWALIT extends ConfigurableMacIT {
     assertEquals(row, startRow + rowCount);
   }
 
-  private int getWALCount(Connector c) throws Exception {
-    WalStateManager wals = new WalStateManager(c.getInstance(), ZooReaderWriter.getInstance());
+  private int getWALCount(Instance i, ZooReaderWriter zk) throws Exception {
+    WalStateManager wals = new WalStateManager(i, zk);
     int result = 0;
     for (Entry<TServerInstance,List<UUID>> entry : wals.getAllMarkers().entrySet()) {
       result += entry.getValue().size();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ecf8e07c/test/src/test/java/org/apache/accumulo/test/VolumeIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/VolumeIT.java b/test/src/test/java/org/apache/accumulo/test/VolumeIT.java
index b66d13f..ae472d1 100644
--- a/test/src/test/java/org/apache/accumulo/test/VolumeIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/VolumeIT.java
@@ -39,6 +39,7 @@ import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.MutationsRejectedException;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableExistsException;
@@ -428,7 +429,9 @@ public class VolumeIT extends ConfigurableMacIT {
       Assert.fail("Unexpected volume " + path);
     }
 
-    WalStateManager wals = new WalStateManager(conn.getInstance(), ZooReaderWriter.getInstance());
+    Instance i = conn.getInstance();
+    ZooReaderWriter zk = new ZooReaderWriter(i.getZooKeepers(), i.getZooKeepersSessionTimeOut(), "");
+    WalStateManager wals = new WalStateManager(i, zk);
     outer: for (Entry<Path,WalState> entry : wals.getAllState().entrySet()) {
       for (Path path : paths) {
         if (entry.getKey().toString().startsWith(path.toString())) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ecf8e07c/test/src/test/java/org/apache/accumulo/test/functional/WALSunnyDayIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/WALSunnyDayIT.java b/test/src/test/java/org/apache/accumulo/test/functional/WALSunnyDayIT.java
index 8d7dd62..166d149 100644
--- a/test/src/test/java/org/apache/accumulo/test/functional/WALSunnyDayIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/functional/WALSunnyDayIT.java
@@ -38,6 +38,7 @@ import java.util.Random;
 
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Mutation;
@@ -59,9 +60,6 @@ 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.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
-import org.apache.zookeeper.ZooKeeper;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -97,29 +95,23 @@ public class WALSunnyDayIT extends ConfigurableMacIT {
     MiniAccumuloClusterControl control = mac.getClusterControl();
     control.stop(GARBAGE_COLLECTOR);
     Connector c = getConnector();
-    ZooKeeper zoo = new ZooKeeper(c.getInstance().getZooKeepers(), c.getInstance().getZooKeepersSessionTimeOut(), new Watcher() {
-      @Override
-      public void process(WatchedEvent event) {
-        log.info(event.toString());
-      }
-    });
     String tableName = getUniqueNames(1)[0];
     c.tableOperations().create(tableName);
     writeSomeData(c, tableName, 1, 1);
 
     // wal markers are added lazily
-    Map<String,Boolean> wals = getWals(c, zoo);
-    assertEquals(wals.toString(), 1, wals.size());
+    Map<String,Boolean> wals = getWals(c);
+    assertEquals(wals.toString(), 2, wals.size());
     for (Boolean b : wals.values()) {
       assertTrue("logs should be in use", b.booleanValue());
     }
 
     // roll log, get a new next
     writeSomeData(c, tableName, 1000, 50);
-    Map<String,Boolean> walsAfterRoll = getWals(c, zoo);
-    assertEquals("should have 3 WALs after roll", 2, walsAfterRoll.size());
+    Map<String,Boolean> walsAfterRoll = getWals(c);
+    assertEquals("should have 3 WALs after roll", 3, walsAfterRoll.size());
     assertTrue("new WALs should be a superset of the old WALs", walsAfterRoll.keySet().containsAll(wals.keySet()));
-    assertEquals("all WALs should be in use", 2, countTrue(walsAfterRoll.values()));
+    assertEquals("all WALs should be in use", 3, countTrue(walsAfterRoll.values()));
 
     // flush the tables
     for (String table : new String[] {tableName, MetadataTable.NAME, RootTable.NAME}) {
@@ -127,16 +119,16 @@ public class WALSunnyDayIT extends ConfigurableMacIT {
     }
     UtilWaitThread.sleep(1000);
     // rolled WAL is no longer in use, but needs to be GC'd
-    Map<String,Boolean> walsAfterflush = getWals(c, zoo);
-    assertEquals(walsAfterflush.toString(), 2, walsAfterflush.size());
-    assertEquals("inUse should be 1", 1, countTrue(walsAfterflush.values()));
+    Map<String,Boolean> walsAfterflush = getWals(c);
+    assertEquals(walsAfterflush.toString(), 3, walsAfterflush.size());
+    assertEquals("inUse should be 2", 2, countTrue(walsAfterflush.values()));
 
     // let the GC run for a little bit
     control.start(GARBAGE_COLLECTOR);
     UtilWaitThread.sleep(5 * 1000);
     // make sure the unused WAL goes away
-    Map<String,Boolean> walsAfterGC = getWals(c, zoo);
-    assertEquals(walsAfterGC.toString(), 1, walsAfterGC.size());
+    Map<String,Boolean> walsAfterGC = getWals(c);
+    assertEquals(walsAfterGC.toString(), 2, walsAfterGC.size());
     control.stop(GARBAGE_COLLECTOR);
     // restart the tserver, but don't run recovery on all tablets
     control.stop(TABLET_SERVER);
@@ -158,14 +150,14 @@ public class WALSunnyDayIT extends ConfigurableMacIT {
     verifySomeData(c, tableName, 1000 * 50 + 1);
     writeSomeData(c, tableName, 100, 100);
 
-    Map<String,Boolean> walsAfterRestart = getWals(c, zoo);
+    Map<String,Boolean> walsAfterRestart = getWals(c);
     // log.debug("wals after " + walsAfterRestart);
-    assertEquals("used WALs after restart should be 1", 1, countTrue(walsAfterRestart.values()));
+    assertEquals("used WALs after restart should be 4", 4, countTrue(walsAfterRestart.values()));
     control.start(GARBAGE_COLLECTOR);
     UtilWaitThread.sleep(5 * 1000);
-    Map<String,Boolean> walsAfterRestartAndGC = getWals(c, zoo);
-    assertEquals("wals left should be 1", 1, walsAfterRestartAndGC.size());
-    assertEquals("logs in use should be 1", 1, countTrue(walsAfterRestartAndGC.values()));
+    Map<String,Boolean> walsAfterRestartAndGC = getWals(c);
+    assertEquals("wals left should be 2", 2, walsAfterRestartAndGC.size());
+    assertEquals("logs in use should be 2", 2, countTrue(walsAfterRestartAndGC.values()));
   }
 
   private void verifySomeData(Connector c, String tableName, int expected) throws Exception {
@@ -198,9 +190,11 @@ public class WALSunnyDayIT extends ConfigurableMacIT {
     bw.close();
   }
 
-  private Map<String,Boolean> getWals(Connector c, ZooKeeper zoo) throws Exception {
+  private Map<String,Boolean> getWals(Connector c) throws Exception {
     Map<String,Boolean> result = new HashMap<>();
-    WalStateManager wals = new WalStateManager(c.getInstance(), ZooReaderWriter.getInstance());
+    Instance i = c.getInstance();
+    ZooReaderWriter zk = new ZooReaderWriter(i.getZooKeepers(), i.getZooKeepersSessionTimeOut(), "");
+    WalStateManager wals = new WalStateManager(c.getInstance(), zk);
     for (Entry<Path,WalState> entry : wals.getAllState().entrySet()) {
       // WALs are in use if they are not unreferenced
       result.put(entry.getKey().toString(), entry.getValue() != WalState.UNREFERENCED);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ecf8e07c/test/src/test/java/org/apache/accumulo/test/replication/GarbageCollectorCommunicatesWithTServersIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/replication/GarbageCollectorCommunicatesWithTServersIT.java b/test/src/test/java/org/apache/accumulo/test/replication/GarbageCollectorCommunicatesWithTServersIT.java
index ddaef00..309de80 100644
--- a/test/src/test/java/org/apache/accumulo/test/replication/GarbageCollectorCommunicatesWithTServersIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/replication/GarbageCollectorCommunicatesWithTServersIT.java
@@ -25,6 +25,7 @@ import java.util.Set;
 
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.impl.ClientContext;
 import org.apache.accumulo.core.client.impl.ClientExecReturn;
@@ -105,7 +106,9 @@ public class GarbageCollectorCommunicatesWithTServersIT extends ConfigurableMacI
 
     Assert.assertNotNull("Could not determine table ID for " + tableName, tableId);
 
-    WalStateManager wals = new WalStateManager(conn.getInstance(), ZooReaderWriter.getInstance());
+    Instance i = conn.getInstance();
+    ZooReaderWriter zk = new ZooReaderWriter(i.getZooKeepers(), i.getZooKeepersSessionTimeOut(), "");
+    WalStateManager wals = new WalStateManager(conn.getInstance(), zk);
 
     Set<String> result = new HashSet<String>();
     for (Entry<Path,WalState> entry : wals.getAllState().entrySet()) {
@@ -285,7 +288,7 @@ public class GarbageCollectorCommunicatesWithTServersIT extends ConfigurableMacI
     log.info("Checking that metadata only has one WAL recorded for this table");
 
     Set<String> wals = getWalsForTable(table);
-    Assert.assertEquals("Expected to only find one WAL for the table", 1, wals.size());
+    Assert.assertEquals("Expected to only find two WAL for the table", 2, wals.size());
 
     log.info("Compacting the table which will remove all WALs from the tablets");
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ecf8e07c/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 da9dd24..5040295 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
@@ -39,6 +39,7 @@ import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.IteratorSetting;
 import org.apache.accumulo.core.client.IteratorSetting.Column;
 import org.apache.accumulo.core.client.Scanner;
@@ -149,7 +150,9 @@ public class ReplicationIT extends ConfigurableMacIT {
     }
     // Map of logs to tableId
     Multimap<String,String> logs = HashMultimap.create();
-    WalStateManager wals = new WalStateManager(conn.getInstance(), ZooReaderWriter.getInstance());
+    Instance i = conn.getInstance();
+    ZooReaderWriter zk = new ZooReaderWriter(i.getZooKeepers(), i.getZooKeepersSessionTimeOut(), "");
+    WalStateManager wals = new WalStateManager(conn.getInstance(), zk);
     for (Entry<TServerInstance,List<UUID>> entry : wals.getAllMarkers().entrySet()) {
       for (UUID id : entry.getValue()) {
         Pair<WalState,Path> state = wals.state(entry.getKey(), id);
@@ -332,8 +335,10 @@ public class ReplicationIT extends ConfigurableMacIT {
 
     Set<String> wals = Sets.newHashSet();
     attempts = 5;
+    Instance i = conn.getInstance();
+    ZooReaderWriter zk = new ZooReaderWriter(i.getZooKeepers(), i.getZooKeepersSessionTimeOut(), "");
     while (wals.isEmpty() && attempts > 0) {
-      WalStateManager markers = new WalStateManager(conn.getInstance(), ZooReaderWriter.getInstance());
+      WalStateManager markers = new WalStateManager(i, zk);
       for (Entry<Path,WalState> entry : markers.getAllState().entrySet()) {
         wals.add(entry.getKey().toString());
       }


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

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


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

Branch: refs/heads/master
Commit: 724504a15477cd9e9e877785f81ab8c261f40d37
Parents: ecf8e07 0138249
Author: Eric C. Newton <er...@gmail.com>
Authored: Tue May 26 15:49:38 2015 -0400
Committer: Eric C. Newton <er...@gmail.com>
Committed: Tue May 26 15:49:38 2015 -0400

----------------------------------------------------------------------
 .../monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------



[3/3] accumulo git commit: ACCUMULO-3423 fixing another IT: WALogs show up before they are used

Posted by ec...@apache.org.
ACCUMULO-3423 fixing another IT: WALogs show up before they are used


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

Branch: refs/heads/master
Commit: fdd7d09a003bedca31db2e3602f8486b263d6ad1
Parents: 724504a
Author: Eric C. Newton <er...@gmail.com>
Authored: Tue May 26 15:50:18 2015 -0400
Committer: Eric C. Newton <er...@gmail.com>
Committed: Tue May 26 15:50:18 2015 -0400

----------------------------------------------------------------------
 .../replication/GarbageCollectorCommunicatesWithTServersIT.java    | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/fdd7d09a/test/src/test/java/org/apache/accumulo/test/replication/GarbageCollectorCommunicatesWithTServersIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/replication/GarbageCollectorCommunicatesWithTServersIT.java b/test/src/test/java/org/apache/accumulo/test/replication/GarbageCollectorCommunicatesWithTServersIT.java
index 309de80..4f9c69f 100644
--- a/test/src/test/java/org/apache/accumulo/test/replication/GarbageCollectorCommunicatesWithTServersIT.java
+++ b/test/src/test/java/org/apache/accumulo/test/replication/GarbageCollectorCommunicatesWithTServersIT.java
@@ -201,7 +201,7 @@ public class GarbageCollectorCommunicatesWithTServersIT extends ConfigurableMacI
     log.info("Checking that metadata only has one WAL recorded for this table");
 
     Set<String> wals = getWalsForTable(table);
-    Assert.assertEquals("Expected to only find one WAL for the table", 1, wals.size());
+    Assert.assertEquals("Expected to only find two WALs for the table", 2, wals.size());
 
     log.info("Compacting the table which will remove all WALs from the tablets");