You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by bu...@apache.org on 2014/08/03 09:36:04 UTC

[1/9] git commit: ACCUMULO-2694 Fix handling of tablet migrations for offline tables.

Repository: accumulo
Updated Branches:
  refs/heads/1.5.2-SNAPSHOT f848178e7 -> 6bbe12165
  refs/heads/1.6.1-SNAPSHOT 39f405a3f -> 533983f10
  refs/heads/master 3c11090ff -> f34f1d892


ACCUMULO-2694 Fix handling of tablet migrations for offline tables.

* Adds a funtional test that fails due to not rebalancing
* Fix master to clear migrations when it learns that a table has gone offline
* Update master to periodically clean up migrations for offline tables
* Fix balancers to make sure they log if they can't balance.


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

Branch: refs/heads/1.5.2-SNAPSHOT
Commit: 6bbe12165d81067651cc2d8c1c545fb31580d1a3
Parents: f848178
Author: Sean Busbey <bu...@cloudera.com>
Authored: Fri Apr 18 16:44:54 2014 -0500
Committer: Sean Busbey <bu...@cloudera.com>
Committed: Fri Aug 1 02:46:55 2014 -0500

----------------------------------------------------------------------
 server/pom.xml                                  |  4 +
 .../apache/accumulo/server/master/Master.java   | 34 +++++--
 .../master/balancer/ChaoticLoadBalancer.java    | 10 +-
 .../master/balancer/DefaultLoadBalancer.java    | 12 ++-
 .../server/master/balancer/TabletBalancer.java  | 91 ++++++++++++++++++
 test/system/auto/stress/migrations.py           | 98 ++++++++++++++++++++
 6 files changed, 240 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/6bbe1216/server/pom.xml
----------------------------------------------------------------------
diff --git a/server/pom.xml b/server/pom.xml
index bd61fe6..fb526af 100644
--- a/server/pom.xml
+++ b/server/pom.xml
@@ -34,6 +34,10 @@
       <artifactId>gson</artifactId>
     </dependency>
     <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+    <dependency>
       <groupId>jline</groupId>
       <artifactId>jline</artifactId>
     </dependency>

http://git-wip-us.apache.org/repos/asf/accumulo/blob/6bbe1216/server/src/main/java/org/apache/accumulo/server/master/Master.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/master/Master.java b/server/src/main/java/org/apache/accumulo/server/master/Master.java
index a2ad2e6..12f8fed 100644
--- a/server/src/main/java/org/apache/accumulo/server/master/Master.java
+++ b/server/src/main/java/org/apache/accumulo/server/master/Master.java
@@ -1924,7 +1924,8 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
       while (stillMaster()) {
         if (!migrations.isEmpty()) {
           try {
-            cleanupMutations();
+            cleanupOfflineMigrations();
+            cleanupNonexistentMigrations(getConnector());
           } catch (Exception ex) {
             log.error("Error cleaning up migrations", ex);
           }
@@ -1933,12 +1934,13 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
       }
     }
     
-    // If a migrating tablet splits, and the tablet dies before sending the
-    // master a message, the migration will refer to a non-existing tablet,
-    // so it can never complete. Periodically scan the metadata table and
-    // remove any migrating tablets that no longer exist.
-    private void cleanupMutations() throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-      Connector connector = getConnector();
+    /**
+     * If a migrating tablet splits, and the tablet dies before sending the
+     * master a message, the migration will refer to a non-existing tablet,
+     * so it can never complete. Periodically scan the metadata table and
+     * remove any migrating tablets that no longer exist.
+     */
+    private void cleanupNonexistentMigrations(final Connector connector) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
       Scanner scanner = connector.createScanner(Constants.METADATA_TABLE_NAME, Constants.NO_AUTHS);
       Constants.METADATA_PREV_ROW_COLUMN.fetch(scanner);
       Set<KeyExtent> found = new HashSet<KeyExtent>();
@@ -1950,6 +1952,21 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
       }
       migrations.keySet().retainAll(found);
     }
+
+    /**
+     * If migrating a tablet for a table that is offline, the migration
+     * can never succeed because no tablet server will load the tablet.
+     * check for offline tables and remove their migrations.
+     */
+    private void cleanupOfflineMigrations() {
+      TableManager manager = TableManager.getInstance();
+      for (String tableId : Tables.getIdToNameMap(instance).keySet()) {
+        TableState state = manager.getTableState(tableId);
+        if (TableState.OFFLINE == state) {
+          clearMigrations(tableId);
+        }
+      }
+    }
   }
   
   private class StatusThread extends Daemon {
@@ -2418,6 +2435,9 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
   @Override
   public void stateChanged(String tableId, TableState state) {
     nextEvent.event("Table state in zookeeper changed for %s to %s", tableId, state);
+    if (TableState.OFFLINE == state) {
+      clearMigrations(tableId);
+    }
   }
   
   @Override

http://git-wip-us.apache.org/repos/asf/accumulo/blob/6bbe1216/server/src/main/java/org/apache/accumulo/server/master/balancer/ChaoticLoadBalancer.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/master/balancer/ChaoticLoadBalancer.java b/server/src/main/java/org/apache/accumulo/server/master/balancer/ChaoticLoadBalancer.java
index e14008a..92eca13 100644
--- a/server/src/main/java/org/apache/accumulo/server/master/balancer/ChaoticLoadBalancer.java
+++ b/server/src/main/java/org/apache/accumulo/server/master/balancer/ChaoticLoadBalancer.java
@@ -33,6 +33,7 @@ import org.apache.accumulo.core.tabletserver.thrift.TabletStats;
 import org.apache.accumulo.server.conf.ServerConfiguration;
 import org.apache.accumulo.server.master.state.TServerInstance;
 import org.apache.accumulo.server.master.state.TabletMigration;
+import org.apache.log4j.Logger;
 import org.apache.thrift.TException;
 
 /**
@@ -40,6 +41,7 @@ import org.apache.thrift.TException;
  * designed for performance, do not use on production systems. I'm calling it the LokiLoadBalancer.
  */
 public class ChaoticLoadBalancer extends TabletBalancer {
+  private static final Logger log = Logger.getLogger(ChaoticLoadBalancer.class);
   Random r = new Random();
   
   @Override
@@ -75,6 +77,8 @@ public class ChaoticLoadBalancer extends TabletBalancer {
     }
   }
   
+  protected final OutstandingMigrations outstandingMigrations = new OutstandingMigrations(log);
+
   /**
    * Will balance randomly, maintaining distribution
    */
@@ -83,8 +87,12 @@ public class ChaoticLoadBalancer extends TabletBalancer {
     Map<TServerInstance,Long> numTablets = new HashMap<TServerInstance,Long>();
     List<TServerInstance> underCapacityTServer = new ArrayList<TServerInstance>();
 
-    if (!migrations.isEmpty())
+    if (!migrations.isEmpty()) {
+      outstandingMigrations.migrations = migrations;
+      constraintNotMet(outstandingMigrations);
       return 100;
+    }
+    resetBalancerErrors();
 
     boolean moveMetadata = r.nextInt(4) == 0;
     long totalTablets = 0;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/6bbe1216/server/src/main/java/org/apache/accumulo/server/master/balancer/DefaultLoadBalancer.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/master/balancer/DefaultLoadBalancer.java b/server/src/main/java/org/apache/accumulo/server/master/balancer/DefaultLoadBalancer.java
index 1fcab46..9a970e7 100644
--- a/server/src/main/java/org/apache/accumulo/server/master/balancer/DefaultLoadBalancer.java
+++ b/server/src/main/java/org/apache/accumulo/server/master/balancer/DefaultLoadBalancer.java
@@ -302,16 +302,26 @@ public class DefaultLoadBalancer extends TabletBalancer {
       assignments.put(entry.getKey(), getAssignment(current, entry.getKey(), entry.getValue()));
     }
   }
-  
+
+  private static final NoTservers NO_SERVERS = new NoTservers(log);
+
+  protected final OutstandingMigrations outstandingMigrations = new OutstandingMigrations(log);
+
   @Override
   public long balance(SortedMap<TServerInstance,TabletServerStatus> current, Set<KeyExtent> migrations, List<TabletMigration> migrationsOut) {
     // do we have any servers?
     if (current.size() > 0) {
       // Don't migrate if we have migrations in progress
       if (migrations.size() == 0) {
+        resetBalancerErrors();
         if (getMigrations(current, migrationsOut))
           return 1 * 1000;
+      } else {
+        outstandingMigrations.migrations = migrations;
+        constraintNotMet(outstandingMigrations);
       }
+    } else {
+      constraintNotMet(NO_SERVERS);
     }
     return 5 * 1000;
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/6bbe1216/server/src/main/java/org/apache/accumulo/server/master/balancer/TabletBalancer.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/master/balancer/TabletBalancer.java b/server/src/main/java/org/apache/accumulo/server/master/balancer/TabletBalancer.java
index 69387d3..16c5dbc 100644
--- a/server/src/main/java/org/apache/accumulo/server/master/balancer/TabletBalancer.java
+++ b/server/src/main/java/org/apache/accumulo/server/master/balancer/TabletBalancer.java
@@ -17,11 +17,14 @@
 package org.apache.accumulo.server.master.balancer;
 
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.SortedMap;
 
+import com.google.common.collect.Iterables;
+
 import org.apache.accumulo.trace.instrument.Tracer;
 import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.data.KeyExtent;
@@ -67,6 +70,11 @@ public abstract class TabletBalancer {
   
   /**
    * Ask the balancer if any migrations are necessary.
+   *
+   * If the balancer is going to self-abort due to some environmental constraint (e.g. it requires some minimum number of tservers, or a maximum number
+   * of outstanding migrations), it should issue a log message to alert operators. The message should be at WARN normally and at ERROR if the balancer knows that the
+   * problem can not self correct. It should not issue these messages more than once a minute. Subclasses can use the convenience methods of {@link #constraintNotMet()} and
+   * {@link #balanceSuccessful()} to accomplish this logging.
    * 
    * @param current
    *          The current table-summary state of all the online tablet servers. Read-only.
@@ -79,6 +87,89 @@ public abstract class TabletBalancer {
    *         This method will not be called when there are unassigned tablets.
    */
   public abstract long balance(SortedMap<TServerInstance,TabletServerStatus> current, Set<KeyExtent> migrations, List<TabletMigration> migrationsOut);
+
+  private static final long ONE_SECOND = 1000l;
+  private boolean stuck = false;
+  private long stuckNotificationTime = -1l;
+
+  protected static final long TIME_BETWEEN_BALANCER_WARNINGS = 60 * ONE_SECOND;
+
+  /**
+   * A deferred call descendent TabletBalancers use to log why they can't continue.
+   * The call is deferred so that TabletBalancer can limit how often messages happen.
+   *
+   * Implementations should be reused as much as possible.
+   *
+   * Be sure to pass in a properly scoped Logger instance so that messages indicate
+   * what part of the system is having trouble.
+   */
+  protected static abstract class BalancerProblem implements Runnable {
+    protected final Logger balancerLog;
+    public BalancerProblem(Logger logger) {
+      balancerLog = logger;
+    }
+  }
+
+  /**
+   * If a TabletBalancer requires active tservers, it should use this problem to indicate when there are none.
+   * NoTservers is safe to share with anyone who uses the same Logger. TabletBalancers should have a single
+   * static instance.
+   */
+  protected static class NoTservers extends BalancerProblem {
+    public NoTservers(Logger logger) {
+      super(logger);
+    }
+
+    @Override
+    public void run() {
+      balancerLog.warn("Not balancing because we don't have any tservers");
+    }
+  }
+
+  /**
+   * If a TabletBalancer only balances when there are no outstanding migrations, it should use this problem
+   * to indicate when they exist.
+   *
+   * Iff a TabletBalancer makes use of the migrations member to provide samples, then OutstandingMigrations
+   * is not thread safe.
+   */
+  protected static class OutstandingMigrations extends BalancerProblem {
+    public Set<KeyExtent> migrations = Collections.<KeyExtent>emptySet();
+
+    public OutstandingMigrations(Logger logger) {
+      super(logger);
+    }
+
+    @Override
+    public void run() {
+      balancerLog.warn("Not balancing due to " + migrations.size() + " outstanding migrations.");
+      /* TODO ACCUMULO-2938 redact key extents in this output to avoid leaking protected information. */
+      balancerLog.debug("Sample up to 10 outstanding migrations: " + Iterables.limit(migrations, 10));
+    }
+  }
+
+  /**
+   * Warn that a Balancer can't work because of some external restriction.
+   * Will not call the provided logging handler  more often than TIME_BETWEEN_BALANCER_WARNINGS
+   */
+  protected void constraintNotMet(BalancerProblem cause) {
+    if (!stuck) {
+      stuck = true;
+      stuckNotificationTime = System.currentTimeMillis();
+    } else {
+      if ((System.currentTimeMillis() - stuckNotificationTime) > TIME_BETWEEN_BALANCER_WARNINGS) {
+        cause.run();
+        stuckNotificationTime = System.currentTimeMillis();
+      }
+    }
+  }
+
+  /**
+   * Resets logging about problems meeting an external constraint on balancing.
+   */
+  protected void resetBalancerErrors() {
+    stuck = false;
+  }
   
   /**
    * Fetch the tablets for the given table by asking the tablet server. Useful if your balance strategy needs details at the tablet level to decide what tablets

http://git-wip-us.apache.org/repos/asf/accumulo/blob/6bbe1216/test/system/auto/stress/migrations.py
----------------------------------------------------------------------
diff --git a/test/system/auto/stress/migrations.py b/test/system/auto/stress/migrations.py
index d07d7a8..9c94b88 100755
--- a/test/system/auto/stress/migrations.py
+++ b/test/system/auto/stress/migrations.py
@@ -76,7 +76,105 @@ class ChaoticBalancerIntegrity(SunnyDayTest):
         self.shell(self.masterHost(), 'flush -t test_ingest')
         self.waitForStop(self.verify(self.masterHost(), self.options.rows), 60)
 
+# Check for ACCUMULO-2694
+class BalanceInPresenceOfOfflineTable(SunnyDayTest):
+    """Start a new table, create many splits, and offline before they can rebalance. Then try to have a different table balance"""
+
+    order = 98
+
+    settings = TestUtilsMixin.settings.copy()
+    settings.update({
+        'tserver.memory.maps.max':'10K',
+        'tserver.compaction.major.delay': 0,
+        })
+    tableSettings = SunnyDayTest.tableSettings.copy()
+    tableSettings['test_ingest'] = {
+        'table.split.threshold': '10K',
+        }
+    def setUp(self):
+        # ensure we have two servers
+        if len(self.options.hosts) == 1:
+            self.options.hosts.append('localhost')
+        self.options.hosts = self.options.hosts[:2]
+
+        TestUtilsMixin.setUp(self);
+
+        # create a table with 200 splits
+        import tempfile
+        fileno, filename = tempfile.mkstemp()
+        fp = os.fdopen(fileno, "wb")
+        try:
+            for i in range(200):
+                fp.write("%08x\n" % (i * 1000))
+        finally:
+            fp.close()
+        self.createTable('unused', filename)
+        out,err,code = self.shell(self.masterHost(), 'offline -t unused\n')
+        self.processResult(out,err,code);
+
+        # create an empty table
+        self.createTable('test_ingest')
+
+    def runTest(self):
+
+        # start test ingestion
+        log.info("Starting Test Ingester")
+        self.ingester = self.ingest(self.masterHost(),
+                                    200000,
+                                    size=self.options.size)
+        self.waitForStop(self.ingester, self.timeout_factor * 120)
+        self.shell(self.masterHost(), 'flush -t test_ingest\n')
+        self.waitForStop(self.verify(self.masterHost(), self.options.rows), 60)
+
+        # let the server split tablets and move them around
+        # Keep retrying until the wait period for migration cleanup has passed
+        # which is hard coded to 5 minutes. :/
+        startTime = time.clock()
+        currentWait = 10
+        balancingWorked = False
+        while ((time.clock() - startTime) < 5*60+15):
+            self.sleep(currentWait)
+            # If we end up needing to sleep again, back off.
+            currentWait *= 2
+
+            # fetch the list of tablets from each server
+            h = self.runOn(self.masterHost(),
+                           [self.accumulo_sh(),
+                            'org.apache.accumulo.test.GetMasterStats'])
+            out, err = h.communicate()
+            servers = {}
+            server = None
+            # if balanced based on ingest, the table that split due to ingest
+            # will be split evenly on both servers, not just one
+            table = ''
+            tableId = self.getTableId('test_ingest');
+            for line in out.split('\n'):
+                if line.find(' Name: ') == 0:
+                    server = line[7:]
+                    servers.setdefault(server, 0)
+                if line.find('Table: ') >= 0:
+                    table = line.split(' ')[-1]
+                if line.find('    Tablets: ') == 0:
+                    if table == tableId:
+                       servers[server] += int(line.split()[-1])
+            log.info("Tablet counts " + repr(servers))
+
+            # we have two servers
+            if len(servers.values()) == 2:
+              servers = servers.values()
+              # a server has more than 10 splits
+              if servers[0] > 10:
+                # the ratio is roughly even
+                ratio = min(servers) / float(max(servers))
+                if ratio > 0.5:
+                  balancingWorked = True
+                  break
+            log.debug("tablets not balanced, sleeping for %d seconds" % currentWait)
+
+        self.assert_(balancingWorked)
+
 def suite():
     result = unittest.TestSuite()
     result.addTest(ChaoticBalancerIntegrity())
+    result.addTest(BalanceInPresenceOfOfflineTable())
     return result


[6/9] git commit: ACCUMULO-2694 update migration cleanup for 1.6 branch.

Posted by bu...@apache.org.
ACCUMULO-2694 update migration cleanup for 1.6 branch.

- manually redo changes to Master
- convert python functional test to an IT
-- Add programmatic access to MasterMonitorInfo to MiniAccumuloClusterImpl
-- Make SystemCredentials work with MiniAccumuloClusterImpl


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

Branch: refs/heads/master
Commit: 533983f10136ce581a8b877379b19eb2302fa578
Parents: dedc9cd
Author: Sean Busbey <bu...@cloudera.com>
Authored: Thu Jun 26 11:11:39 2014 -0500
Committer: Sean Busbey <bu...@cloudera.com>
Committed: Fri Aug 1 19:32:25 2014 -0500

----------------------------------------------------------------------
 .../core/client/AccumuloSecurityException.java  |   8 +
 .../impl/MiniAccumuloClusterImpl.java           |  32 ++++
 .../impl/MiniAccumuloClusterImplTest.java       |  37 +++++
 minicluster/src/test/resources/log4j.properties |  22 ++-
 .../server/security/SystemCredentials.java      |  21 ++-
 .../java/org/apache/accumulo/master/Master.java |  34 ++++-
 test/pom.xml                                    |   5 +
 .../BalanceInPresenceOfOfflineTableIT.java      | 152 +++++++++++++++++++
 8 files changed, 300 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/533983f1/core/src/main/java/org/apache/accumulo/core/client/AccumuloSecurityException.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/AccumuloSecurityException.java b/core/src/main/java/org/apache/accumulo/core/client/AccumuloSecurityException.java
index 06b148d..35ea188 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/AccumuloSecurityException.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/AccumuloSecurityException.java
@@ -75,6 +75,14 @@ public class AccumuloSecurityException extends Exception {
   }
   
   /**
+   * Construct a user-facing exception from a serialized version.
+   * @param thrift a serialized version
+   */
+  public AccumuloSecurityException(final ThriftSecurityException thrift) {
+    this(thrift.getUser(), thrift.getCode(), thrift);
+  }
+
+  /**
    * @param user
    *          the relevant user for the security violation
    * @param errorcode

http://git-wip-us.apache.org/repos/asf/accumulo/blob/533983f1/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImpl.java
----------------------------------------------------------------------
diff --git a/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImpl.java b/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImpl.java
index 9a8efa6..deb04d9 100644
--- a/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImpl.java
+++ b/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImpl.java
@@ -56,9 +56,13 @@ import org.apache.accumulo.core.client.ClientConfiguration;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.ZooKeeperInstance;
+import org.apache.accumulo.core.client.impl.MasterClient;
+import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.master.thrift.MasterGoalState;
+import org.apache.accumulo.core.master.thrift.MasterClientService;
+import org.apache.accumulo.core.master.thrift.MasterMonitorInfo;
 import org.apache.accumulo.core.util.Daemon;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.core.util.StringUtil;
@@ -68,10 +72,12 @@ import org.apache.accumulo.master.Master;
 import org.apache.accumulo.master.state.SetGoalState;
 import org.apache.accumulo.minicluster.ServerType;
 import org.apache.accumulo.server.init.Initialize;
+import org.apache.accumulo.server.security.SystemCredentials;
 import org.apache.accumulo.server.util.PortUtils;
 import org.apache.accumulo.server.util.time.SimpleTimer;
 import org.apache.accumulo.start.Main;
 import org.apache.accumulo.start.classloader.vfs.MiniDFSUtil;
+import org.apache.accumulo.trace.instrument.Tracer;
 import org.apache.accumulo.tserver.TabletServer;
 import org.apache.commons.configuration.MapConfiguration;
 import org.apache.commons.io.FileUtils;
@@ -82,6 +88,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.log4j.Logger;
+import org.apache.thrift.TException;
 import org.apache.zookeeper.server.ZooKeeperServerMain;
 
 import com.google.common.base.Predicate;
@@ -742,4 +749,29 @@ public class MiniAccumuloClusterImpl implements AccumuloCluster {
 
     return future.get(timeout, unit);
   }
+
+  /**
+   * Get programmatic interface to information available in a normal monitor.
+   * XXX the returned structure won't contain information about the metadata table until there is data in it.
+   * e.g. if you want to see the metadata table you should create a table.
+   * @since 1.6.1
+   */
+  public MasterMonitorInfo getMasterMonitorInfo() throws AccumuloException, AccumuloSecurityException {
+    MasterClientService.Iface client = null;
+    MasterMonitorInfo stats = null;
+    try {
+      Instance instance = new ZooKeeperInstance(getClientConfig());
+      client = MasterClient.getConnectionWithRetry(instance);
+      stats = client.getMasterStats(Tracer.traceInfo(), SystemCredentials.get(instance).toThrift(instance));
+    } catch (ThriftSecurityException exception) {
+      throw new AccumuloSecurityException(exception);
+    } catch (TException exception) {
+      throw new AccumuloException(exception);
+    } finally {
+      if (client != null) {
+        MasterClient.close(client);
+      }
+    }
+    return stats;
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/533983f1/minicluster/src/test/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImplTest.java
----------------------------------------------------------------------
diff --git a/minicluster/src/test/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImplTest.java b/minicluster/src/test/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImplTest.java
index 99f8d7d..32b20b0 100644
--- a/minicluster/src/test/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImplTest.java
+++ b/minicluster/src/test/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImplTest.java
@@ -17,9 +17,17 @@
 package org.apache.accumulo.minicluster.impl;
 
 import java.io.File;
+import java.util.Arrays;
 import java.util.Collection;
+import java.util.List;
 import java.util.Map;
 
+import org.apache.accumulo.core.client.admin.TableOperations;
+import org.apache.accumulo.core.master.thrift.MasterMonitorInfo;
+import org.apache.accumulo.core.master.thrift.MasterGoalState;
+import org.apache.accumulo.core.master.thrift.MasterState;
+import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.RootTable;
 import org.apache.accumulo.minicluster.ServerType;
 import org.apache.commons.io.FileUtils;
 import org.apache.log4j.Level;
@@ -31,10 +39,17 @@ import org.junit.Test;
 
 public class MiniAccumuloClusterImplTest {
 
+  private static final Logger log = Logger.getLogger(MiniAccumuloClusterImplTest.class);
+
   public static File testDir;
 
   private static MiniAccumuloClusterImpl accumulo;
 
+  private static final int NUM_TSERVERS = 2;
+
+  private static String TEST_TABLE = "test";
+  private static String testTableID;
+
   @BeforeClass
   public static void setupMiniCluster() throws Exception {
     Logger.getLogger("org.apache.zookeeper").setLevel(Level.ERROR);
@@ -46,8 +61,14 @@ public class MiniAccumuloClusterImplTest {
     testDir.mkdir();
 
     MiniAccumuloConfigImpl config = new MiniAccumuloConfigImpl(testDir, "superSecret").setJDWPEnabled(true);
+    // expressly set number of tservers since we assert it later, in case the default changes
+    config.setNumTservers(NUM_TSERVERS);
     accumulo = new MiniAccumuloClusterImpl(config);
     accumulo.start();
+    // create a table to ensure there are some entries in the !0 table
+    TableOperations tableops = accumulo.getConnector("root","superSecret").tableOperations();
+    tableops.create(TEST_TABLE);
+    testTableID = tableops.tableIdMap().get(TEST_TABLE);
   }
 
   @Test(timeout = 10000)
@@ -67,6 +88,22 @@ public class MiniAccumuloClusterImplTest {
     }
   }
 
+  @Test
+  public void saneMonitorInfo() throws Exception {
+    log.info("ensure monitor info includes some base information.");
+    MasterMonitorInfo stats = accumulo.getMasterMonitorInfo();
+    List<MasterState> validStates = Arrays.asList(MasterState.values());
+    List<MasterGoalState> validGoals = Arrays.asList(MasterGoalState.values());
+    Assert.assertTrue("master state should be valid.", validStates.contains(stats.state));
+    Assert.assertTrue("master goal state should be in " + validGoals + ". is " + stats.goalState, validGoals.contains(stats.goalState));
+    Assert.assertNotNull("should have a table map.", stats.tableMap);
+    Assert.assertTrue("root table should exist in " + stats.tableMap.keySet(), stats.tableMap.keySet().contains(RootTable.ID));
+    Assert.assertTrue("meta table should exist in " + stats.tableMap.keySet(), stats.tableMap.keySet().contains(MetadataTable.ID));
+    Assert.assertTrue("our test table should exist in " + stats.tableMap.keySet(), stats.tableMap.keySet().contains(testTableID));
+    Assert.assertNotNull("there should be tservers.", stats.tServerInfo);
+    Assert.assertEquals(NUM_TSERVERS, stats.tServerInfo.size());
+  }
+
   @AfterClass
   public static void tearDownMiniCluster() throws Exception {
     accumulo.stop();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/533983f1/minicluster/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/minicluster/src/test/resources/log4j.properties b/minicluster/src/test/resources/log4j.properties
index b5efe8d..75dfd78 100644
--- a/minicluster/src/test/resources/log4j.properties
+++ b/minicluster/src/test/resources/log4j.properties
@@ -1 +1,21 @@
-# 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.

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} [%-8c{2}] %-5p: %m%n

log4j
 .logger.org.apache.accumulo.core.client.impl.ServerClient=ERROR
\ No newline at end of file
+# 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.
+
+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} [%-8c{2}] %-5p: %m%n
+
+log4j.logger.org.apache.accumulo.core.client.impl.ServerClient=ERROR

http://git-wip-us.apache.org/repos/asf/accumulo/blob/533983f1/server/base/src/main/java/org/apache/accumulo/server/security/SystemCredentials.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/security/SystemCredentials.java b/server/base/src/main/java/org/apache/accumulo/server/security/SystemCredentials.java
index 767ed25..19e7ff7 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/security/SystemCredentials.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/security/SystemCredentials.java
@@ -57,14 +57,29 @@ public final class SystemCredentials extends Credentials {
   }
 
   public static SystemCredentials get() {
+    check_permission();
+    if (SYSTEM_CREDS == null) {
+      SYSTEM_CREDS = new SystemCredentials(HdfsZooInstance.getInstance());
+    }
+    return SYSTEM_CREDS;
+  }
+
+  private static void check_permission() {
     SecurityManager sm = System.getSecurityManager();
     if (sm != null) {
       sm.checkPermission(SYSTEM_CREDENTIALS_PERMISSION);
     }
-    if (SYSTEM_CREDS == null) {
-      SYSTEM_CREDS = new SystemCredentials(HdfsZooInstance.getInstance());
+  }
+
+  public static SystemCredentials get(Instance instance) {
+    check_permission();
+    /* Special case to avoid duplicating SYSTEM_CREDS */
+    if (null != SYSTEM_CREDS) {
+      if (SYSTEM_CREDS.AS_THRIFT.getInstanceId().equals(instance.getInstanceID())) {
+        return SYSTEM_CREDS;
+      }
     }
-    return SYSTEM_CREDS;
+    return new SystemCredentials(instance);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/accumulo/blob/533983f1/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 30b1f2e..2f04922 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
@@ -715,7 +715,8 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
       while (stillMaster()) {
         if (!migrations.isEmpty()) {
           try {
-            cleanupMutations();
+            cleanupOfflineMigrations();
+            cleanupNonexistentMigrations(getConnector());
           } catch (Exception ex) {
             log.error("Error cleaning up migrations", ex);
           }
@@ -724,12 +725,13 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
       }
     }
 
-    // If a migrating tablet splits, and the tablet dies before sending the
-    // master a message, the migration will refer to a non-existing tablet,
-    // so it can never complete. Periodically scan the metadata table and
-    // remove any migrating tablets that no longer exist.
-    private void cleanupMutations() throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-      Connector connector = getConnector();
+    /**
+     * If a migrating tablet splits, and the tablet dies before sending the
+     * master a message, the migration will refer to a non-existing tablet,
+     * so it can never complete. Periodically scan the metadata table and
+     * remove any migrating tablets that no longer exist.
+     */
+    private void cleanupNonexistentMigrations(final Connector connector) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
       Scanner scanner = connector.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
       TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN.fetch(scanner);
       Set<KeyExtent> found = new HashSet<KeyExtent>();
@@ -741,6 +743,21 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
       }
       migrations.keySet().retainAll(found);
     }
+
+    /**
+     * If migrating a tablet for a table that is offline, the migration
+     * can never succeed because no tablet server will load the tablet.
+     * check for offline tables and remove their migrations.
+     */
+    private void cleanupOfflineMigrations() {
+      TableManager manager = TableManager.getInstance();
+      for (String tableId : Tables.getIdToNameMap(instance).keySet()) {
+        TableState state = manager.getTableState(tableId);
+        if (TableState.OFFLINE == state) {
+          clearMigrations(tableId);
+        }
+      }
+    }
   }
 
   private class StatusThread extends Daemon {
@@ -1206,6 +1223,9 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
   @Override
   public void stateChanged(String tableId, TableState state) {
     nextEvent.event("Table state in zookeeper changed for %s to %s", tableId, state);
+    if (TableState.OFFLINE == state) {
+      clearMigrations(tableId);
+    }
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/accumulo/blob/533983f1/test/pom.xml
----------------------------------------------------------------------
diff --git a/test/pom.xml b/test/pom.xml
index f3cc927..17c8527 100644
--- a/test/pom.xml
+++ b/test/pom.xml
@@ -137,6 +137,11 @@
       <scope>test</scope>
     </dependency>
     <dependency>
+      <groupId>commons-lang</groupId>
+      <artifactId>commons-lang</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>
       <scope>test</scope>

http://git-wip-us.apache.org/repos/asf/accumulo/blob/533983f1/test/src/test/java/org/apache/accumulo/test/functional/BalanceInPresenceOfOfflineTableIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/BalanceInPresenceOfOfflineTableIT.java b/test/src/test/java/org/apache/accumulo/test/functional/BalanceInPresenceOfOfflineTableIT.java
new file mode 100644
index 0000000..37aa8a1
--- /dev/null
+++ b/test/src/test/java/org/apache/accumulo/test/functional/BalanceInPresenceOfOfflineTableIT.java
@@ -0,0 +1,152 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.test.functional;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.TableExistsException;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.master.thrift.MasterMonitorInfo;
+import org.apache.accumulo.core.master.thrift.TableInfo;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.test.TestIngest;
+import org.apache.accumulo.test.VerifyIngest;
+import org.apache.commons.lang.math.NumberUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Text;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Start a new table, create many splits, and offline before they can rebalance. Then try to have a different table balance
+ */
+public class BalanceInPresenceOfOfflineTableIT extends ConfigurableMacIT {
+
+  private static Logger log = LoggerFactory.getLogger(BalanceInPresenceOfOfflineTableIT.class);
+
+  @Override
+  public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
+    Map<String,String> siteConfig = new HashMap<String, String>();
+    siteConfig.put(Property.TSERV_MAXMEM.getKey(), "10K");
+    siteConfig.put(Property.TSERV_MAJC_DELAY.getKey(), "0");
+    cfg.setSiteConfig(siteConfig );
+    // ensure we have two tservers
+    if (cfg.getNumTservers() < 2) {
+      cfg.setNumTservers(2);
+    }
+  }
+
+  @Override
+  protected int defaultTimeoutSeconds() {
+    return 10 * 60;
+  }
+
+  private static final int NUM_SPLITS = 200;
+  private static final String UNUSED_TABLE = "unused";
+  private static final String TEST_TABLE = "test_ingest";
+
+  private Connector connector;
+
+  @Before
+  public void setupTables() throws AccumuloException, AccumuloSecurityException, TableExistsException, TableNotFoundException {
+    // set up splits
+    final SortedSet<Text> splits = new TreeSet<Text>();
+    for (int i = 0; i < NUM_SPLITS; i++) {
+      splits.add(new Text(String.format("%08x", i * 1000)));
+    }
+    // load into a table we won't use
+    connector = getConnector();
+    connector.tableOperations().create(UNUSED_TABLE);
+    connector.tableOperations().addSplits(UNUSED_TABLE, splits);
+    // mark the table offline before it can rebalance.
+    connector.tableOperations().offline(UNUSED_TABLE);
+
+    // actual test table
+    connector.tableOperations().create(TEST_TABLE);
+    connector.tableOperations().setProperty(TEST_TABLE, Property.TABLE_SPLIT_THRESHOLD.getKey(), "10K");
+  }
+
+  @Test
+  public void test() throws Exception {
+    log.info("Test that balancing is not stopped by an offline table with outstanding migrations.");
+
+    log.debug("starting test ingestion");
+
+    TestIngest.Opts opts = new TestIngest.Opts();
+    VerifyIngest.Opts vopts = new VerifyIngest.Opts();
+    vopts.rows = opts.rows = 200000;
+    TestIngest.ingest(connector, opts, BWOPTS);
+    connector.tableOperations().flush(TEST_TABLE, null, null, true);
+    VerifyIngest.verifyIngest(connector, vopts, SOPTS);
+
+    log.debug("waiting for balancing, up to ~5 minutes to allow for migration cleanup.");
+    final long startTime = System.currentTimeMillis();
+    long currentWait = 10*1000;
+    boolean balancingWorked = false;
+
+    while (!balancingWorked && (System.currentTimeMillis() - startTime) < ((5*60 + 15)*1000)) {
+      Thread.sleep(currentWait);
+      currentWait *= 2;
+
+      log.debug("fetch the list of tablets assigned to each tserver.");
+      MasterMonitorInfo stats = getCluster().getMasterMonitorInfo();
+
+      if (stats.getTServerInfoSize() < 2) {
+        log.debug("we need >= 2 servers. sleeping for " + currentWait + "ms");
+        continue;
+      }
+      if (stats.getUnassignedTablets() != 0) {
+        log.debug("We shouldn't have unassigned tablets. sleeping for " + currentWait + "ms");
+        continue;
+      }
+
+      int numTablets = 0;
+      long[] tabletsPerServer = new long[stats.getTServerInfoSize()];
+      Arrays.fill(tabletsPerServer, 0l);
+      for (int i = 0; i < stats.getTServerInfoSize(); i++) {
+        for (Map.Entry<String, TableInfo> entry : stats.getTServerInfo().get(i).getTableMap().entrySet()) {
+          tabletsPerServer[i] += entry.getValue().getTablets();
+        }
+      }
+
+      if (tabletsPerServer[0] <= 10) {
+        log.debug("We should have > 10 tablets. sleeping for " + currentWait + "ms");
+        continue;
+      }
+      if ((NumberUtils.min(tabletsPerServer) / ((double)NumberUtils.max(tabletsPerServer))) < 0.5) {
+        log.debug("ratio of min to max tablets per server should be roughly even. sleeping for " + currentWait + "ms");
+        continue;
+      }
+      balancingWorked = true;
+    }
+
+    Assert.assertTrue("did not properly balance", balancingWorked);
+  }
+
+}


[3/9] git commit: ACCUMULO-2694 Fix handling of tablet migrations for offline tables.

Posted by bu...@apache.org.
ACCUMULO-2694 Fix handling of tablet migrations for offline tables.

* Adds a funtional test that fails due to not rebalancing
* Fix master to clear migrations when it learns that a table has gone offline
* Update master to periodically clean up migrations for offline tables
* Fix balancers to make sure they log if they can't balance.


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

Branch: refs/heads/master
Commit: 6bbe12165d81067651cc2d8c1c545fb31580d1a3
Parents: f848178
Author: Sean Busbey <bu...@cloudera.com>
Authored: Fri Apr 18 16:44:54 2014 -0500
Committer: Sean Busbey <bu...@cloudera.com>
Committed: Fri Aug 1 02:46:55 2014 -0500

----------------------------------------------------------------------
 server/pom.xml                                  |  4 +
 .../apache/accumulo/server/master/Master.java   | 34 +++++--
 .../master/balancer/ChaoticLoadBalancer.java    | 10 +-
 .../master/balancer/DefaultLoadBalancer.java    | 12 ++-
 .../server/master/balancer/TabletBalancer.java  | 91 ++++++++++++++++++
 test/system/auto/stress/migrations.py           | 98 ++++++++++++++++++++
 6 files changed, 240 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/6bbe1216/server/pom.xml
----------------------------------------------------------------------
diff --git a/server/pom.xml b/server/pom.xml
index bd61fe6..fb526af 100644
--- a/server/pom.xml
+++ b/server/pom.xml
@@ -34,6 +34,10 @@
       <artifactId>gson</artifactId>
     </dependency>
     <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+    <dependency>
       <groupId>jline</groupId>
       <artifactId>jline</artifactId>
     </dependency>

http://git-wip-us.apache.org/repos/asf/accumulo/blob/6bbe1216/server/src/main/java/org/apache/accumulo/server/master/Master.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/master/Master.java b/server/src/main/java/org/apache/accumulo/server/master/Master.java
index a2ad2e6..12f8fed 100644
--- a/server/src/main/java/org/apache/accumulo/server/master/Master.java
+++ b/server/src/main/java/org/apache/accumulo/server/master/Master.java
@@ -1924,7 +1924,8 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
       while (stillMaster()) {
         if (!migrations.isEmpty()) {
           try {
-            cleanupMutations();
+            cleanupOfflineMigrations();
+            cleanupNonexistentMigrations(getConnector());
           } catch (Exception ex) {
             log.error("Error cleaning up migrations", ex);
           }
@@ -1933,12 +1934,13 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
       }
     }
     
-    // If a migrating tablet splits, and the tablet dies before sending the
-    // master a message, the migration will refer to a non-existing tablet,
-    // so it can never complete. Periodically scan the metadata table and
-    // remove any migrating tablets that no longer exist.
-    private void cleanupMutations() throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-      Connector connector = getConnector();
+    /**
+     * If a migrating tablet splits, and the tablet dies before sending the
+     * master a message, the migration will refer to a non-existing tablet,
+     * so it can never complete. Periodically scan the metadata table and
+     * remove any migrating tablets that no longer exist.
+     */
+    private void cleanupNonexistentMigrations(final Connector connector) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
       Scanner scanner = connector.createScanner(Constants.METADATA_TABLE_NAME, Constants.NO_AUTHS);
       Constants.METADATA_PREV_ROW_COLUMN.fetch(scanner);
       Set<KeyExtent> found = new HashSet<KeyExtent>();
@@ -1950,6 +1952,21 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
       }
       migrations.keySet().retainAll(found);
     }
+
+    /**
+     * If migrating a tablet for a table that is offline, the migration
+     * can never succeed because no tablet server will load the tablet.
+     * check for offline tables and remove their migrations.
+     */
+    private void cleanupOfflineMigrations() {
+      TableManager manager = TableManager.getInstance();
+      for (String tableId : Tables.getIdToNameMap(instance).keySet()) {
+        TableState state = manager.getTableState(tableId);
+        if (TableState.OFFLINE == state) {
+          clearMigrations(tableId);
+        }
+      }
+    }
   }
   
   private class StatusThread extends Daemon {
@@ -2418,6 +2435,9 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
   @Override
   public void stateChanged(String tableId, TableState state) {
     nextEvent.event("Table state in zookeeper changed for %s to %s", tableId, state);
+    if (TableState.OFFLINE == state) {
+      clearMigrations(tableId);
+    }
   }
   
   @Override

http://git-wip-us.apache.org/repos/asf/accumulo/blob/6bbe1216/server/src/main/java/org/apache/accumulo/server/master/balancer/ChaoticLoadBalancer.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/master/balancer/ChaoticLoadBalancer.java b/server/src/main/java/org/apache/accumulo/server/master/balancer/ChaoticLoadBalancer.java
index e14008a..92eca13 100644
--- a/server/src/main/java/org/apache/accumulo/server/master/balancer/ChaoticLoadBalancer.java
+++ b/server/src/main/java/org/apache/accumulo/server/master/balancer/ChaoticLoadBalancer.java
@@ -33,6 +33,7 @@ import org.apache.accumulo.core.tabletserver.thrift.TabletStats;
 import org.apache.accumulo.server.conf.ServerConfiguration;
 import org.apache.accumulo.server.master.state.TServerInstance;
 import org.apache.accumulo.server.master.state.TabletMigration;
+import org.apache.log4j.Logger;
 import org.apache.thrift.TException;
 
 /**
@@ -40,6 +41,7 @@ import org.apache.thrift.TException;
  * designed for performance, do not use on production systems. I'm calling it the LokiLoadBalancer.
  */
 public class ChaoticLoadBalancer extends TabletBalancer {
+  private static final Logger log = Logger.getLogger(ChaoticLoadBalancer.class);
   Random r = new Random();
   
   @Override
@@ -75,6 +77,8 @@ public class ChaoticLoadBalancer extends TabletBalancer {
     }
   }
   
+  protected final OutstandingMigrations outstandingMigrations = new OutstandingMigrations(log);
+
   /**
    * Will balance randomly, maintaining distribution
    */
@@ -83,8 +87,12 @@ public class ChaoticLoadBalancer extends TabletBalancer {
     Map<TServerInstance,Long> numTablets = new HashMap<TServerInstance,Long>();
     List<TServerInstance> underCapacityTServer = new ArrayList<TServerInstance>();
 
-    if (!migrations.isEmpty())
+    if (!migrations.isEmpty()) {
+      outstandingMigrations.migrations = migrations;
+      constraintNotMet(outstandingMigrations);
       return 100;
+    }
+    resetBalancerErrors();
 
     boolean moveMetadata = r.nextInt(4) == 0;
     long totalTablets = 0;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/6bbe1216/server/src/main/java/org/apache/accumulo/server/master/balancer/DefaultLoadBalancer.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/master/balancer/DefaultLoadBalancer.java b/server/src/main/java/org/apache/accumulo/server/master/balancer/DefaultLoadBalancer.java
index 1fcab46..9a970e7 100644
--- a/server/src/main/java/org/apache/accumulo/server/master/balancer/DefaultLoadBalancer.java
+++ b/server/src/main/java/org/apache/accumulo/server/master/balancer/DefaultLoadBalancer.java
@@ -302,16 +302,26 @@ public class DefaultLoadBalancer extends TabletBalancer {
       assignments.put(entry.getKey(), getAssignment(current, entry.getKey(), entry.getValue()));
     }
   }
-  
+
+  private static final NoTservers NO_SERVERS = new NoTservers(log);
+
+  protected final OutstandingMigrations outstandingMigrations = new OutstandingMigrations(log);
+
   @Override
   public long balance(SortedMap<TServerInstance,TabletServerStatus> current, Set<KeyExtent> migrations, List<TabletMigration> migrationsOut) {
     // do we have any servers?
     if (current.size() > 0) {
       // Don't migrate if we have migrations in progress
       if (migrations.size() == 0) {
+        resetBalancerErrors();
         if (getMigrations(current, migrationsOut))
           return 1 * 1000;
+      } else {
+        outstandingMigrations.migrations = migrations;
+        constraintNotMet(outstandingMigrations);
       }
+    } else {
+      constraintNotMet(NO_SERVERS);
     }
     return 5 * 1000;
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/6bbe1216/server/src/main/java/org/apache/accumulo/server/master/balancer/TabletBalancer.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/master/balancer/TabletBalancer.java b/server/src/main/java/org/apache/accumulo/server/master/balancer/TabletBalancer.java
index 69387d3..16c5dbc 100644
--- a/server/src/main/java/org/apache/accumulo/server/master/balancer/TabletBalancer.java
+++ b/server/src/main/java/org/apache/accumulo/server/master/balancer/TabletBalancer.java
@@ -17,11 +17,14 @@
 package org.apache.accumulo.server.master.balancer;
 
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.SortedMap;
 
+import com.google.common.collect.Iterables;
+
 import org.apache.accumulo.trace.instrument.Tracer;
 import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.data.KeyExtent;
@@ -67,6 +70,11 @@ public abstract class TabletBalancer {
   
   /**
    * Ask the balancer if any migrations are necessary.
+   *
+   * If the balancer is going to self-abort due to some environmental constraint (e.g. it requires some minimum number of tservers, or a maximum number
+   * of outstanding migrations), it should issue a log message to alert operators. The message should be at WARN normally and at ERROR if the balancer knows that the
+   * problem can not self correct. It should not issue these messages more than once a minute. Subclasses can use the convenience methods of {@link #constraintNotMet()} and
+   * {@link #balanceSuccessful()} to accomplish this logging.
    * 
    * @param current
    *          The current table-summary state of all the online tablet servers. Read-only.
@@ -79,6 +87,89 @@ public abstract class TabletBalancer {
    *         This method will not be called when there are unassigned tablets.
    */
   public abstract long balance(SortedMap<TServerInstance,TabletServerStatus> current, Set<KeyExtent> migrations, List<TabletMigration> migrationsOut);
+
+  private static final long ONE_SECOND = 1000l;
+  private boolean stuck = false;
+  private long stuckNotificationTime = -1l;
+
+  protected static final long TIME_BETWEEN_BALANCER_WARNINGS = 60 * ONE_SECOND;
+
+  /**
+   * A deferred call descendent TabletBalancers use to log why they can't continue.
+   * The call is deferred so that TabletBalancer can limit how often messages happen.
+   *
+   * Implementations should be reused as much as possible.
+   *
+   * Be sure to pass in a properly scoped Logger instance so that messages indicate
+   * what part of the system is having trouble.
+   */
+  protected static abstract class BalancerProblem implements Runnable {
+    protected final Logger balancerLog;
+    public BalancerProblem(Logger logger) {
+      balancerLog = logger;
+    }
+  }
+
+  /**
+   * If a TabletBalancer requires active tservers, it should use this problem to indicate when there are none.
+   * NoTservers is safe to share with anyone who uses the same Logger. TabletBalancers should have a single
+   * static instance.
+   */
+  protected static class NoTservers extends BalancerProblem {
+    public NoTservers(Logger logger) {
+      super(logger);
+    }
+
+    @Override
+    public void run() {
+      balancerLog.warn("Not balancing because we don't have any tservers");
+    }
+  }
+
+  /**
+   * If a TabletBalancer only balances when there are no outstanding migrations, it should use this problem
+   * to indicate when they exist.
+   *
+   * Iff a TabletBalancer makes use of the migrations member to provide samples, then OutstandingMigrations
+   * is not thread safe.
+   */
+  protected static class OutstandingMigrations extends BalancerProblem {
+    public Set<KeyExtent> migrations = Collections.<KeyExtent>emptySet();
+
+    public OutstandingMigrations(Logger logger) {
+      super(logger);
+    }
+
+    @Override
+    public void run() {
+      balancerLog.warn("Not balancing due to " + migrations.size() + " outstanding migrations.");
+      /* TODO ACCUMULO-2938 redact key extents in this output to avoid leaking protected information. */
+      balancerLog.debug("Sample up to 10 outstanding migrations: " + Iterables.limit(migrations, 10));
+    }
+  }
+
+  /**
+   * Warn that a Balancer can't work because of some external restriction.
+   * Will not call the provided logging handler  more often than TIME_BETWEEN_BALANCER_WARNINGS
+   */
+  protected void constraintNotMet(BalancerProblem cause) {
+    if (!stuck) {
+      stuck = true;
+      stuckNotificationTime = System.currentTimeMillis();
+    } else {
+      if ((System.currentTimeMillis() - stuckNotificationTime) > TIME_BETWEEN_BALANCER_WARNINGS) {
+        cause.run();
+        stuckNotificationTime = System.currentTimeMillis();
+      }
+    }
+  }
+
+  /**
+   * Resets logging about problems meeting an external constraint on balancing.
+   */
+  protected void resetBalancerErrors() {
+    stuck = false;
+  }
   
   /**
    * Fetch the tablets for the given table by asking the tablet server. Useful if your balance strategy needs details at the tablet level to decide what tablets

http://git-wip-us.apache.org/repos/asf/accumulo/blob/6bbe1216/test/system/auto/stress/migrations.py
----------------------------------------------------------------------
diff --git a/test/system/auto/stress/migrations.py b/test/system/auto/stress/migrations.py
index d07d7a8..9c94b88 100755
--- a/test/system/auto/stress/migrations.py
+++ b/test/system/auto/stress/migrations.py
@@ -76,7 +76,105 @@ class ChaoticBalancerIntegrity(SunnyDayTest):
         self.shell(self.masterHost(), 'flush -t test_ingest')
         self.waitForStop(self.verify(self.masterHost(), self.options.rows), 60)
 
+# Check for ACCUMULO-2694
+class BalanceInPresenceOfOfflineTable(SunnyDayTest):
+    """Start a new table, create many splits, and offline before they can rebalance. Then try to have a different table balance"""
+
+    order = 98
+
+    settings = TestUtilsMixin.settings.copy()
+    settings.update({
+        'tserver.memory.maps.max':'10K',
+        'tserver.compaction.major.delay': 0,
+        })
+    tableSettings = SunnyDayTest.tableSettings.copy()
+    tableSettings['test_ingest'] = {
+        'table.split.threshold': '10K',
+        }
+    def setUp(self):
+        # ensure we have two servers
+        if len(self.options.hosts) == 1:
+            self.options.hosts.append('localhost')
+        self.options.hosts = self.options.hosts[:2]
+
+        TestUtilsMixin.setUp(self);
+
+        # create a table with 200 splits
+        import tempfile
+        fileno, filename = tempfile.mkstemp()
+        fp = os.fdopen(fileno, "wb")
+        try:
+            for i in range(200):
+                fp.write("%08x\n" % (i * 1000))
+        finally:
+            fp.close()
+        self.createTable('unused', filename)
+        out,err,code = self.shell(self.masterHost(), 'offline -t unused\n')
+        self.processResult(out,err,code);
+
+        # create an empty table
+        self.createTable('test_ingest')
+
+    def runTest(self):
+
+        # start test ingestion
+        log.info("Starting Test Ingester")
+        self.ingester = self.ingest(self.masterHost(),
+                                    200000,
+                                    size=self.options.size)
+        self.waitForStop(self.ingester, self.timeout_factor * 120)
+        self.shell(self.masterHost(), 'flush -t test_ingest\n')
+        self.waitForStop(self.verify(self.masterHost(), self.options.rows), 60)
+
+        # let the server split tablets and move them around
+        # Keep retrying until the wait period for migration cleanup has passed
+        # which is hard coded to 5 minutes. :/
+        startTime = time.clock()
+        currentWait = 10
+        balancingWorked = False
+        while ((time.clock() - startTime) < 5*60+15):
+            self.sleep(currentWait)
+            # If we end up needing to sleep again, back off.
+            currentWait *= 2
+
+            # fetch the list of tablets from each server
+            h = self.runOn(self.masterHost(),
+                           [self.accumulo_sh(),
+                            'org.apache.accumulo.test.GetMasterStats'])
+            out, err = h.communicate()
+            servers = {}
+            server = None
+            # if balanced based on ingest, the table that split due to ingest
+            # will be split evenly on both servers, not just one
+            table = ''
+            tableId = self.getTableId('test_ingest');
+            for line in out.split('\n'):
+                if line.find(' Name: ') == 0:
+                    server = line[7:]
+                    servers.setdefault(server, 0)
+                if line.find('Table: ') >= 0:
+                    table = line.split(' ')[-1]
+                if line.find('    Tablets: ') == 0:
+                    if table == tableId:
+                       servers[server] += int(line.split()[-1])
+            log.info("Tablet counts " + repr(servers))
+
+            # we have two servers
+            if len(servers.values()) == 2:
+              servers = servers.values()
+              # a server has more than 10 splits
+              if servers[0] > 10:
+                # the ratio is roughly even
+                ratio = min(servers) / float(max(servers))
+                if ratio > 0.5:
+                  balancingWorked = True
+                  break
+            log.debug("tablets not balanced, sleeping for %d seconds" % currentWait)
+
+        self.assert_(balancingWorked)
+
 def suite():
     result = unittest.TestSuite()
     result.addTest(ChaoticBalancerIntegrity())
+    result.addTest(BalanceInPresenceOfOfflineTable())
     return result


[5/9] git commit: Merge branch '1.5.2-SNAPSHOT' into 1.6.1-SNAPSHOT

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

Conflicts:
	server/base/src/main/java/org/apache/accumulo/server/master/balancer/ChaoticLoadBalancer.java
	server/base/src/main/java/org/apache/accumulo/server/master/balancer/TabletBalancer.java
	server/src/main/java/org/apache/accumulo/server/master/Master.java
	server/tserver/pom.xml
	test/system/auto/stress/migrations.py


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

Branch: refs/heads/master
Commit: dedc9cddaaa46558f542b4d9d6d357316815378f
Parents: 39f405a 6bbe121
Author: Sean Busbey <bu...@cloudera.com>
Authored: Fri Aug 1 19:31:51 2014 -0500
Committer: Sean Busbey <bu...@cloudera.com>
Committed: Fri Aug 1 19:31:51 2014 -0500

----------------------------------------------------------------------
 .../master/balancer/ChaoticLoadBalancer.java    | 14 ++-
 .../master/balancer/DefaultLoadBalancer.java    | 12 ++-
 .../server/master/balancer/TabletBalancer.java  | 91 ++++++++++++++++++++
 3 files changed, 113 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/dedc9cdd/server/base/src/main/java/org/apache/accumulo/server/master/balancer/ChaoticLoadBalancer.java
----------------------------------------------------------------------
diff --cc server/base/src/main/java/org/apache/accumulo/server/master/balancer/ChaoticLoadBalancer.java
index ec3371c,0000000..5767934
mode 100644,000000..100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/ChaoticLoadBalancer.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/balancer/ChaoticLoadBalancer.java
@@@ -1,143 -1,0 +1,151 @@@
 +/*
 + * 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.master.balancer;
 +
 +import java.util.ArrayList;
 +import java.util.HashMap;
 +import java.util.List;
 +import java.util.Map;
 +import java.util.Map.Entry;
 +import java.util.Random;
 +import java.util.Set;
 +import java.util.SortedMap;
 +
 +import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
 +import org.apache.accumulo.core.data.KeyExtent;
 +import org.apache.accumulo.core.master.thrift.TableInfo;
 +import org.apache.accumulo.core.master.thrift.TabletServerStatus;
 +import org.apache.accumulo.core.metadata.MetadataTable;
 +import org.apache.accumulo.core.tabletserver.thrift.TabletStats;
 +import org.apache.accumulo.server.conf.ServerConfiguration;
 +import org.apache.accumulo.server.master.state.TServerInstance;
 +import org.apache.accumulo.server.master.state.TabletMigration;
++import org.apache.log4j.Logger;
 +import org.apache.thrift.TException;
 +
 +/**
 + * A chaotic load balancer used for testing. It constantly shuffles tablets, preventing them from resting in a single location for very long. This is not
 + * designed for performance, do not use on production systems. I'm calling it the LokiLoadBalancer.
 + */
 +public class ChaoticLoadBalancer extends TabletBalancer {
++  private static final Logger log = Logger.getLogger(ChaoticLoadBalancer.class);
 +  Random r = new Random();
 +  
 +  @Override
 +  public void getAssignments(SortedMap<TServerInstance,TabletServerStatus> current, Map<KeyExtent,TServerInstance> unassigned,
 +      Map<KeyExtent,TServerInstance> assignments) {
 +    long total = assignments.size() + unassigned.size();
 +    long avg = (long) Math.ceil(((double) total) / current.size());
 +    Map<TServerInstance,Long> toAssign = new HashMap<TServerInstance,Long>();
 +    List<TServerInstance> tServerArray = new ArrayList<TServerInstance>();
 +    for (Entry<TServerInstance,TabletServerStatus> e : current.entrySet()) {
 +      long numTablets = 0;
 +      for (TableInfo ti : e.getValue().getTableMap().values()) {
 +        numTablets += ti.tablets;
 +      }
 +      if (numTablets < avg) {
 +        tServerArray.add(e.getKey());
 +        toAssign.put(e.getKey(), avg - numTablets);
 +      }
 +    }
 +    
 +    for (KeyExtent ke : unassigned.keySet()) {
 +      int index = r.nextInt(tServerArray.size());
 +      TServerInstance dest = tServerArray.get(index);
 +      assignments.put(ke, dest);
 +      long remaining = toAssign.get(dest).longValue() - 1;
 +      if (remaining == 0) {
 +        tServerArray.remove(index);
 +        toAssign.remove(dest);
 +      } else {
 +        toAssign.put(dest, remaining);
 +      }
 +    }
 +  }
 +  
++  protected final OutstandingMigrations outstandingMigrations = new OutstandingMigrations(log);
++
 +  /**
 +   * Will balance randomly, maintaining distribution
 +   */
 +  @Override
 +  public long balance(SortedMap<TServerInstance,TabletServerStatus> current, Set<KeyExtent> migrations, List<TabletMigration> migrationsOut) {
 +    Map<TServerInstance,Long> numTablets = new HashMap<TServerInstance,Long>();
 +    List<TServerInstance> underCapacityTServer = new ArrayList<TServerInstance>();
-     
-     if (!migrations.isEmpty())
++
++    if (!migrations.isEmpty()) {
++      outstandingMigrations.migrations = migrations;
++      constraintNotMet(outstandingMigrations);
 +      return 100;
-     
++    }
++    resetBalancerErrors();
++
 +    boolean moveMetadata = r.nextInt(4) == 0;
 +    long totalTablets = 0;
 +    for (Entry<TServerInstance,TabletServerStatus> e : current.entrySet()) {
 +      long tabletCount = 0;
 +      for (TableInfo ti : e.getValue().getTableMap().values()) {
 +        tabletCount += ti.tablets;
 +      }
 +      numTablets.put(e.getKey(), tabletCount);
 +      underCapacityTServer.add(e.getKey());
 +      totalTablets += tabletCount;
 +    }
 +    // totalTablets is fuzzy due to asynchronicity of the stats
 +    // *1.2 to handle fuzziness, and prevent locking for 'perfect' balancing scenarios
 +    long avg = (long) Math.ceil(((double) totalTablets) / current.size() * 1.2);
 +    
 +    for (Entry<TServerInstance,TabletServerStatus> e : current.entrySet()) {
 +      for (String table : e.getValue().getTableMap().keySet()) {
 +        if (!moveMetadata && MetadataTable.NAME.equals(table))
 +          continue;
 +        try {
 +          for (TabletStats ts : getOnlineTabletsForTable(e.getKey(), table)) {
 +            KeyExtent ke = new KeyExtent(ts.extent);
 +            int index = r.nextInt(underCapacityTServer.size());
 +            TServerInstance dest = underCapacityTServer.get(index);
 +            if (dest.equals(e.getKey()))
 +              continue;
 +            migrationsOut.add(new TabletMigration(ke, e.getKey(), dest));
 +            if (numTablets.put(dest, numTablets.get(dest) + 1) > avg)
 +              underCapacityTServer.remove(index);
 +            if (numTablets.put(e.getKey(), numTablets.get(e.getKey()) - 1) <= avg && !underCapacityTServer.contains(e.getKey()))
 +              underCapacityTServer.add(e.getKey());
 +            
 +            // We can get some craziness with only 1 tserver, so lets make sure there's always an option!
 +            if (underCapacityTServer.isEmpty())
 +              underCapacityTServer.addAll(numTablets.keySet());
 +          }
 +        } catch (ThriftSecurityException e1) {
 +          // Shouldn't happen, but carry on if it does
 +          e1.printStackTrace();
 +        } catch (TException e1) {
 +          // Shouldn't happen, but carry on if it does
 +          e1.printStackTrace();
 +        }
 +      }
 +    }
 +    
 +    return 100;
 +  }
 +  
 +  @Override
 +  public void init(ServerConfiguration conf) {
 +    super.init(conf);
 +  }
 +  
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/dedc9cdd/server/base/src/main/java/org/apache/accumulo/server/master/balancer/DefaultLoadBalancer.java
----------------------------------------------------------------------
diff --cc server/base/src/main/java/org/apache/accumulo/server/master/balancer/DefaultLoadBalancer.java
index 1fcab46,0000000..9a970e7
mode 100644,000000..100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/DefaultLoadBalancer.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/balancer/DefaultLoadBalancer.java
@@@ -1,319 -1,0 +1,329 @@@
 +/*
 + * 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.master.balancer;
 +
 +import java.util.ArrayList;
 +import java.util.Collections;
 +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;
 +import java.util.SortedMap;
 +
 +import org.apache.accumulo.core.data.KeyExtent;
 +import org.apache.accumulo.core.master.thrift.TableInfo;
 +import org.apache.accumulo.core.master.thrift.TabletServerStatus;
 +import org.apache.accumulo.core.tabletserver.thrift.TabletStats;
 +import org.apache.accumulo.server.master.state.TServerInstance;
 +import org.apache.accumulo.server.master.state.TabletMigration;
 +import org.apache.log4j.Logger;
 +
 +public class DefaultLoadBalancer extends TabletBalancer {
 +  
 +  private static final Logger log = Logger.getLogger(DefaultLoadBalancer.class);
 +  
 +  Iterator<TServerInstance> assignments;
 +  // if tableToBalance is set, then only balance the given table
 +  String tableToBalance = null;
 +  
 +  public DefaultLoadBalancer() {
 +    
 +  }
 +  
 +  public DefaultLoadBalancer(String table) {
 +    tableToBalance = table;
 +  }
 +  
 +  List<TServerInstance> randomize(Set<TServerInstance> locations) {
 +    List<TServerInstance> result = new ArrayList<TServerInstance>(locations);
 +    Collections.shuffle(result);
 +    return result;
 +  }
 +  
 +  public TServerInstance getAssignment(SortedMap<TServerInstance,TabletServerStatus> locations, KeyExtent extent, TServerInstance last) {
 +    if (locations.size() == 0)
 +      return null;
 +    
 +    if (last != null) {
 +      // Maintain locality
 +      String fakeSessionID = " ";
 +      TServerInstance simple = new TServerInstance(last.getLocation(), fakeSessionID);
 +      Iterator<TServerInstance> find = locations.tailMap(simple).keySet().iterator();
 +      if (find.hasNext()) {
 +        TServerInstance current = find.next();
 +        if (current.host().equals(last.host()))
 +          return current;
 +      }
 +    }
 +    
 +    // The strategy here is to walk through the locations and hand them back, one at a time
 +    // Grab an iterator off of the set of options; use a new iterator if it hands back something not in the current list.
 +    if (assignments == null || !assignments.hasNext())
 +      assignments = randomize(locations.keySet()).iterator();
 +    TServerInstance result = assignments.next();
 +    if (!locations.containsKey(result)) {
 +      assignments = null;
 +      return randomize(locations.keySet()).iterator().next();
 +    }
 +    return result;
 +  }
 +  
 +  static class ServerCounts implements Comparable<ServerCounts> {
 +    public final TServerInstance server;
 +    public final int count;
 +    public final TabletServerStatus status;
 +    
 +    ServerCounts(int count, TServerInstance server, TabletServerStatus status) {
 +      this.count = count;
 +      this.server = server;
 +      this.status = status;
 +    }
 +    
 +    public int compareTo(ServerCounts obj) {
 +      int result = count - obj.count;
 +      if (result == 0)
 +        return server.compareTo(obj.server);
 +      return result;
 +    }
 +  }
 +  
 +  public boolean getMigrations(Map<TServerInstance,TabletServerStatus> current, List<TabletMigration> result) {
 +    boolean moreBalancingNeeded = false;
 +    try {
 +      // no moves possible
 +      if (current.size() < 2) {
 +        return false;
 +      }
 +      
 +      // Sort by total number of online tablets, per server
 +      int total = 0;
 +      ArrayList<ServerCounts> totals = new ArrayList<ServerCounts>();
 +      for (Entry<TServerInstance,TabletServerStatus> entry : current.entrySet()) {
 +        int serverTotal = 0;
 +        if (entry.getValue() != null && entry.getValue().tableMap != null) {
 +          for (Entry<String,TableInfo> e : entry.getValue().tableMap.entrySet()) {
 +            /**
 +             * The check below was on entry.getKey(), but that resolves to a tabletserver not a tablename. Believe it should be e.getKey() which is a tablename
 +             */
 +            if (tableToBalance == null || tableToBalance.equals(e.getKey()))
 +              serverTotal += e.getValue().onlineTablets;
 +          }
 +        }
 +        totals.add(new ServerCounts(serverTotal, entry.getKey(), entry.getValue()));
 +        total += serverTotal;
 +      }
 +      
 +      // order from low to high
 +      Collections.sort(totals);
 +      Collections.reverse(totals);
 +      int even = total / totals.size();
 +      int numServersOverEven = total % totals.size();
 +      
 +      // Move tablets from the servers with too many to the servers with
 +      // the fewest but only nominate tablets to move once. This allows us
 +      // to fill new servers with tablets from a mostly balanced server
 +      // very quickly. However, it may take several balancing passes to move
 +      // tablets from one hugely overloaded server to many slightly
 +      // under-loaded servers.
 +      int end = totals.size() - 1;
 +      int movedAlready = 0;
 +      for (int tooManyIndex = 0; tooManyIndex < totals.size(); tooManyIndex++) {
 +        ServerCounts tooMany = totals.get(tooManyIndex);
 +        int goal = even;
 +        if (tooManyIndex < numServersOverEven) {
 +          goal++;
 +        }
 +        int needToUnload = tooMany.count - goal;
 +        ServerCounts tooLittle = totals.get(end);
 +        int needToLoad = goal - tooLittle.count - movedAlready;
 +        if (needToUnload < 1 && needToLoad < 1) {
 +          break;
 +        }
 +        if (needToUnload >= needToLoad) {
 +          result.addAll(move(tooMany, tooLittle, needToLoad));
 +          end--;
 +          movedAlready = 0;
 +        } else {
 +          result.addAll(move(tooMany, tooLittle, needToUnload));
 +          movedAlready += needToUnload;
 +        }
 +        if (needToUnload > needToLoad)
 +          moreBalancingNeeded = true;
 +      }
 +      
 +    } finally {
 +      log.debug("balance ended with " + result.size() + " migrations");
 +    }
 +    return moreBalancingNeeded;
 +  }
 +  
 +  static class TableDiff {
 +    int diff;
 +    String table;
 +    
 +    public TableDiff(int diff, String table) {
 +      this.diff = diff;
 +      this.table = table;
 +    }
 +  };
 +  
 +  /**
 +   * Select a tablet based on differences between table loads; if the loads are even, use the busiest table
 +   */
 +  List<TabletMigration> move(ServerCounts tooMuch, ServerCounts tooLittle, int count) {
 +    
 +    List<TabletMigration> result = new ArrayList<TabletMigration>();
 +    if (count == 0)
 +      return result;
 +    
 +    Map<String,Map<KeyExtent,TabletStats>> onlineTablets = new HashMap<String,Map<KeyExtent,TabletStats>>();
 +    // Copy counts so we can update them as we propose migrations
 +    Map<String,Integer> tooMuchMap = tabletCountsPerTable(tooMuch.status);
 +    Map<String,Integer> tooLittleMap = tabletCountsPerTable(tooLittle.status);
 +    
 +    for (int i = 0; i < count; i++) {
 +      String table;
 +      Integer tooLittleCount;
 +      if (tableToBalance == null) {
 +        // find a table to migrate
 +        // look for an uneven table count
 +        int biggestDifference = 0;
 +        String biggestDifferenceTable = null;
 +        for (Entry<String,Integer> tableEntry : tooMuchMap.entrySet()) {
 +          String tableID = tableEntry.getKey();
 +          if (tooLittleMap.get(tableID) == null)
 +            tooLittleMap.put(tableID, 0);
 +          int diff = tableEntry.getValue() - tooLittleMap.get(tableID);
 +          if (diff > biggestDifference) {
 +            biggestDifference = diff;
 +            biggestDifferenceTable = tableID;
 +          }
 +        }
 +        if (biggestDifference < 2) {
 +          table = busiest(tooMuch.status.tableMap);
 +        } else {
 +          table = biggestDifferenceTable;
 +        }
 +      } else {
 +        // just balance the given table
 +        table = tableToBalance;
 +      }
 +      Map<KeyExtent,TabletStats> onlineTabletsForTable = onlineTablets.get(table);
 +      try {
 +        if (onlineTabletsForTable == null) {
 +          onlineTabletsForTable = new HashMap<KeyExtent,TabletStats>();
 +          for (TabletStats stat : getOnlineTabletsForTable(tooMuch.server, table))
 +            onlineTabletsForTable.put(new KeyExtent(stat.extent), stat);
 +          onlineTablets.put(table, onlineTabletsForTable);
 +        }
 +      } catch (Exception ex) {
 +        log.error("Unable to select a tablet to move", ex);
 +        return result;
 +      }
 +      KeyExtent extent = selectTablet(tooMuch.server, onlineTabletsForTable);
 +      onlineTabletsForTable.remove(extent);
 +      if (extent == null)
 +        return result;
 +      tooMuchMap.put(table, tooMuchMap.get(table) - 1);
 +      /**
 +       * If a table grows from 1 tablet then tooLittleMap.get(table) can return a null, since there is only one tabletserver that holds all of the tablets. Here
 +       * we check to see if in fact that is the case and if so set the value to 0.
 +       */
 +      tooLittleCount = tooLittleMap.get(table);
 +      if (tooLittleCount == null) {
 +        tooLittleCount = 0;
 +      }
 +      tooLittleMap.put(table, tooLittleCount + 1);
 +      
 +      result.add(new TabletMigration(extent, tooMuch.server, tooLittle.server));
 +    }
 +    return result;
 +  }
 +  
 +  static Map<String,Integer> tabletCountsPerTable(TabletServerStatus status) {
 +    Map<String,Integer> result = new HashMap<String,Integer>();
 +    if (status != null && status.tableMap != null) {
 +      Map<String,TableInfo> tableMap = status.tableMap;
 +      for (Entry<String,TableInfo> entry : tableMap.entrySet()) {
 +        result.put(entry.getKey(), entry.getValue().onlineTablets);
 +      }
 +    }
 +    return result;
 +  }
 +  
 +  static KeyExtent selectTablet(TServerInstance tserver, Map<KeyExtent,TabletStats> extents) {
 +    if (extents.size() == 0)
 +      return null;
 +    KeyExtent mostRecentlySplit = null;
 +    long splitTime = 0;
 +    for (Entry<KeyExtent,TabletStats> entry : extents.entrySet())
 +      if (entry.getValue().splitCreationTime >= splitTime) {
 +        splitTime = entry.getValue().splitCreationTime;
 +        mostRecentlySplit = entry.getKey();
 +      }
 +    return mostRecentlySplit;
 +  }
 +  
 +  // define what it means for a tablet to be busy
 +  private static String busiest(Map<String,TableInfo> tables) {
 +    String result = null;
 +    double busiest = Double.NEGATIVE_INFINITY;
 +    for (Entry<String,TableInfo> entry : tables.entrySet()) {
 +      TableInfo info = entry.getValue();
 +      double busy = info.ingestRate + info.queryRate;
 +      if (busy > busiest) {
 +        busiest = busy;
 +        result = entry.getKey();
 +      }
 +    }
 +    return result;
 +  }
 +  
 +  @Override
 +  public void getAssignments(SortedMap<TServerInstance,TabletServerStatus> current, Map<KeyExtent,TServerInstance> unassigned,
 +      Map<KeyExtent,TServerInstance> assignments) {
 +    for (Entry<KeyExtent,TServerInstance> entry : unassigned.entrySet()) {
 +      assignments.put(entry.getKey(), getAssignment(current, entry.getKey(), entry.getValue()));
 +    }
 +  }
-   
++
++  private static final NoTservers NO_SERVERS = new NoTservers(log);
++
++  protected final OutstandingMigrations outstandingMigrations = new OutstandingMigrations(log);
++
 +  @Override
 +  public long balance(SortedMap<TServerInstance,TabletServerStatus> current, Set<KeyExtent> migrations, List<TabletMigration> migrationsOut) {
 +    // do we have any servers?
 +    if (current.size() > 0) {
 +      // Don't migrate if we have migrations in progress
 +      if (migrations.size() == 0) {
++        resetBalancerErrors();
 +        if (getMigrations(current, migrationsOut))
 +          return 1 * 1000;
++      } else {
++        outstandingMigrations.migrations = migrations;
++        constraintNotMet(outstandingMigrations);
 +      }
++    } else {
++      constraintNotMet(NO_SERVERS);
 +    }
 +    return 5 * 1000;
 +  }
 +  
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/dedc9cdd/server/base/src/main/java/org/apache/accumulo/server/master/balancer/TabletBalancer.java
----------------------------------------------------------------------
diff --cc server/base/src/main/java/org/apache/accumulo/server/master/balancer/TabletBalancer.java
index 5bd1632,0000000..fb97628
mode 100644,000000..100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/TabletBalancer.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/balancer/TabletBalancer.java
@@@ -1,149 -1,0 +1,240 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one or more
 + * contributor license agreements.  See the NOTICE file distributed with
 + * this work for additional information regarding copyright ownership.
 + * The ASF licenses this file to You under the Apache License, Version 2.0
 + * (the "License"); you may not use this file except in compliance with
 + * the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.accumulo.server.master.balancer;
 +
 +import java.util.ArrayList;
++import java.util.Collections;
 +import java.util.List;
 +import java.util.Map;
 +import java.util.Set;
 +import java.util.SortedMap;
 +
++import com.google.common.collect.Iterables;
++
 +import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
 +import org.apache.accumulo.core.data.KeyExtent;
 +import org.apache.accumulo.core.master.thrift.TabletServerStatus;
 +import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
 +import org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Client;
 +import org.apache.accumulo.core.tabletserver.thrift.TabletStats;
 +import org.apache.accumulo.core.util.ThriftUtil;
 +import org.apache.accumulo.server.conf.ServerConfiguration;
 +import org.apache.accumulo.server.master.state.TServerInstance;
 +import org.apache.accumulo.server.master.state.TabletMigration;
 +import org.apache.accumulo.server.security.SystemCredentials;
 +import org.apache.accumulo.trace.instrument.Tracer;
 +import org.apache.log4j.Logger;
 +import org.apache.thrift.TException;
 +import org.apache.thrift.transport.TTransportException;
 +
 +public abstract class TabletBalancer {
 +  
 +  private static final Logger log = Logger.getLogger(TabletBalancer.class);
 +  
 +  protected ServerConfiguration configuration;
 +  
 +  /**
 +   * Initialize the TabletBalancer. This gives the balancer the opportunity to read the configuration.
 +   */
 +  public void init(ServerConfiguration conf) {
 +    configuration = conf;
 +  }
 +  
 +  /**
 +   * Assign tablets to tablet servers. This method is called whenever the master finds tablets that are unassigned.
 +   * 
 +   * @param current
 +   *          The current table-summary state of all the online tablet servers. Read-only. The TabletServerStatus for each server may be null if the tablet
 +   *          server has not yet responded to a recent request for status.
 +   * @param unassigned
 +   *          A map from unassigned tablet to the last known tablet server. Read-only.
 +   * @param assignments
 +   *          A map from tablet to assigned server. Write-only.
 +   */
 +  abstract public void getAssignments(SortedMap<TServerInstance,TabletServerStatus> current, Map<KeyExtent,TServerInstance> unassigned,
 +      Map<KeyExtent,TServerInstance> assignments);
 +  
 +  /**
 +   * Ask the balancer if any migrations are necessary.
++   *
++   * If the balancer is going to self-abort due to some environmental constraint (e.g. it requires some minimum number of tservers, or a maximum number
++   * of outstanding migrations), it should issue a log message to alert operators. The message should be at WARN normally and at ERROR if the balancer knows that the
++   * problem can not self correct. It should not issue these messages more than once a minute. Subclasses can use the convenience methods of {@link #constraintNotMet()} and
++   * {@link #balanceSuccessful()} to accomplish this logging.
 +   * 
 +   * @param current
 +   *          The current table-summary state of all the online tablet servers. Read-only.
 +   * @param migrations
 +   *          the current set of migrations. Read-only.
 +   * @param migrationsOut
 +   *          new migrations to perform; should not contain tablets in the current set of migrations. Write-only.
 +   * @return the time, in milliseconds, to wait before re-balancing.
 +   * 
 +   *         This method will not be called when there are unassigned tablets.
 +   */
 +  public abstract long balance(SortedMap<TServerInstance,TabletServerStatus> current, Set<KeyExtent> migrations, List<TabletMigration> migrationsOut);
++
++  private static final long ONE_SECOND = 1000l;
++  private boolean stuck = false;
++  private long stuckNotificationTime = -1l;
++
++  protected static final long TIME_BETWEEN_BALANCER_WARNINGS = 60 * ONE_SECOND;
++
++  /**
++   * A deferred call descendent TabletBalancers use to log why they can't continue.
++   * The call is deferred so that TabletBalancer can limit how often messages happen.
++   *
++   * Implementations should be reused as much as possible.
++   *
++   * Be sure to pass in a properly scoped Logger instance so that messages indicate
++   * what part of the system is having trouble.
++   */
++  protected static abstract class BalancerProblem implements Runnable {
++    protected final Logger balancerLog;
++    public BalancerProblem(Logger logger) {
++      balancerLog = logger;
++    }
++  }
++
++  /**
++   * If a TabletBalancer requires active tservers, it should use this problem to indicate when there are none.
++   * NoTservers is safe to share with anyone who uses the same Logger. TabletBalancers should have a single
++   * static instance.
++   */
++  protected static class NoTservers extends BalancerProblem {
++    public NoTservers(Logger logger) {
++      super(logger);
++    }
++
++    @Override
++    public void run() {
++      balancerLog.warn("Not balancing because we don't have any tservers");
++    }
++  }
++
++  /**
++   * If a TabletBalancer only balances when there are no outstanding migrations, it should use this problem
++   * to indicate when they exist.
++   *
++   * Iff a TabletBalancer makes use of the migrations member to provide samples, then OutstandingMigrations
++   * is not thread safe.
++   */
++  protected static class OutstandingMigrations extends BalancerProblem {
++    public Set<KeyExtent> migrations = Collections.<KeyExtent>emptySet();
++
++    public OutstandingMigrations(Logger logger) {
++      super(logger);
++    }
++
++    @Override
++    public void run() {
++      balancerLog.warn("Not balancing due to " + migrations.size() + " outstanding migrations.");
++      /* TODO ACCUMULO-2938 redact key extents in this output to avoid leaking protected information. */
++      balancerLog.debug("Sample up to 10 outstanding migrations: " + Iterables.limit(migrations, 10));
++    }
++  }
++
++  /**
++   * Warn that a Balancer can't work because of some external restriction.
++   * Will not call the provided logging handler  more often than TIME_BETWEEN_BALANCER_WARNINGS
++   */
++  protected void constraintNotMet(BalancerProblem cause) {
++    if (!stuck) {
++      stuck = true;
++      stuckNotificationTime = System.currentTimeMillis();
++    } else {
++      if ((System.currentTimeMillis() - stuckNotificationTime) > TIME_BETWEEN_BALANCER_WARNINGS) {
++        cause.run();
++        stuckNotificationTime = System.currentTimeMillis();
++      }
++    }
++  }
++
++  /**
++   * Resets logging about problems meeting an external constraint on balancing.
++   */
++  protected void resetBalancerErrors() {
++    stuck = false;
++  }
 +  
 +  /**
 +   * Fetch the tablets for the given table by asking the tablet server. Useful if your balance strategy needs details at the tablet level to decide what tablets
 +   * to move.
 +   * 
 +   * @param tserver
 +   *          The tablet server to ask.
 +   * @param tableId
 +   *          The table id
 +   * @return a list of tablet statistics
 +   * @throws ThriftSecurityException
 +   *           tablet server disapproves of your internal System password.
 +   * @throws TException
 +   *           any other problem
 +   */
 +  public List<TabletStats> getOnlineTabletsForTable(TServerInstance tserver, String tableId) throws ThriftSecurityException, TException {
 +    log.debug("Scanning tablet server " + tserver + " for table " + tableId);
 +    Client client = ThriftUtil.getClient(new TabletClientService.Client.Factory(), tserver.getLocation(), configuration.getConfiguration());
 +    try {
 +      List<TabletStats> onlineTabletsForTable = client.getTabletStats(Tracer.traceInfo(), SystemCredentials.get().toThrift(configuration.getInstance()),
 +          tableId);
 +      return onlineTabletsForTable;
 +    } catch (TTransportException e) {
 +      log.error("Unable to connect to " + tserver + ": " + e);
 +    } finally {
 +      ThriftUtil.returnClient(client);
 +    }
 +    return null;
 +  }
 +  
 +  /**
 +   * Utility to ensure that the migrations from balance() are consistent:
 +   * <ul>
 +   * <li>Tablet objects are not null
 +   * <li>Source and destination tablet servers are not null and current
 +   * </ul>
 +   * 
 +   * @return A list of TabletMigration object that passed sanity checks.
 +   */
 +  public static List<TabletMigration> checkMigrationSanity(Set<TServerInstance> current, List<TabletMigration> migrations) {
 +    List<TabletMigration> result = new ArrayList<TabletMigration>(migrations.size());
 +    for (TabletMigration m : migrations) {
 +      if (m.tablet == null) {
 +        log.warn("Balancer gave back a null tablet " + m);
 +        continue;
 +      }
 +      if (m.newServer == null) {
 +        log.warn("Balancer did not set the destination " + m);
 +        continue;
 +      }
 +      if (m.oldServer == null) {
 +        log.warn("Balancer did not set the source " + m);
 +        continue;
 +      }
 +      if (!current.contains(m.oldServer)) {
 +        log.warn("Balancer wants to move a tablet from a server that is not current: " + m);
 +        continue;
 +      }
 +      if (!current.contains(m.newServer)) {
 +        log.warn("Balancer wants to move a tablet to a server that is not current: " + m);
 +        continue;
 +      }
 +      result.add(m);
 +    }
 +    return result;
 +  }
 +  
 +}


[2/9] git commit: ACCUMULO-2694 Fix handling of tablet migrations for offline tables.

Posted by bu...@apache.org.
ACCUMULO-2694 Fix handling of tablet migrations for offline tables.

* Adds a funtional test that fails due to not rebalancing
* Fix master to clear migrations when it learns that a table has gone offline
* Update master to periodically clean up migrations for offline tables
* Fix balancers to make sure they log if they can't balance.


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

Branch: refs/heads/1.6.1-SNAPSHOT
Commit: 6bbe12165d81067651cc2d8c1c545fb31580d1a3
Parents: f848178
Author: Sean Busbey <bu...@cloudera.com>
Authored: Fri Apr 18 16:44:54 2014 -0500
Committer: Sean Busbey <bu...@cloudera.com>
Committed: Fri Aug 1 02:46:55 2014 -0500

----------------------------------------------------------------------
 server/pom.xml                                  |  4 +
 .../apache/accumulo/server/master/Master.java   | 34 +++++--
 .../master/balancer/ChaoticLoadBalancer.java    | 10 +-
 .../master/balancer/DefaultLoadBalancer.java    | 12 ++-
 .../server/master/balancer/TabletBalancer.java  | 91 ++++++++++++++++++
 test/system/auto/stress/migrations.py           | 98 ++++++++++++++++++++
 6 files changed, 240 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/6bbe1216/server/pom.xml
----------------------------------------------------------------------
diff --git a/server/pom.xml b/server/pom.xml
index bd61fe6..fb526af 100644
--- a/server/pom.xml
+++ b/server/pom.xml
@@ -34,6 +34,10 @@
       <artifactId>gson</artifactId>
     </dependency>
     <dependency>
+      <groupId>com.google.guava</groupId>
+      <artifactId>guava</artifactId>
+    </dependency>
+    <dependency>
       <groupId>jline</groupId>
       <artifactId>jline</artifactId>
     </dependency>

http://git-wip-us.apache.org/repos/asf/accumulo/blob/6bbe1216/server/src/main/java/org/apache/accumulo/server/master/Master.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/master/Master.java b/server/src/main/java/org/apache/accumulo/server/master/Master.java
index a2ad2e6..12f8fed 100644
--- a/server/src/main/java/org/apache/accumulo/server/master/Master.java
+++ b/server/src/main/java/org/apache/accumulo/server/master/Master.java
@@ -1924,7 +1924,8 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
       while (stillMaster()) {
         if (!migrations.isEmpty()) {
           try {
-            cleanupMutations();
+            cleanupOfflineMigrations();
+            cleanupNonexistentMigrations(getConnector());
           } catch (Exception ex) {
             log.error("Error cleaning up migrations", ex);
           }
@@ -1933,12 +1934,13 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
       }
     }
     
-    // If a migrating tablet splits, and the tablet dies before sending the
-    // master a message, the migration will refer to a non-existing tablet,
-    // so it can never complete. Periodically scan the metadata table and
-    // remove any migrating tablets that no longer exist.
-    private void cleanupMutations() throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-      Connector connector = getConnector();
+    /**
+     * If a migrating tablet splits, and the tablet dies before sending the
+     * master a message, the migration will refer to a non-existing tablet,
+     * so it can never complete. Periodically scan the metadata table and
+     * remove any migrating tablets that no longer exist.
+     */
+    private void cleanupNonexistentMigrations(final Connector connector) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
       Scanner scanner = connector.createScanner(Constants.METADATA_TABLE_NAME, Constants.NO_AUTHS);
       Constants.METADATA_PREV_ROW_COLUMN.fetch(scanner);
       Set<KeyExtent> found = new HashSet<KeyExtent>();
@@ -1950,6 +1952,21 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
       }
       migrations.keySet().retainAll(found);
     }
+
+    /**
+     * If migrating a tablet for a table that is offline, the migration
+     * can never succeed because no tablet server will load the tablet.
+     * check for offline tables and remove their migrations.
+     */
+    private void cleanupOfflineMigrations() {
+      TableManager manager = TableManager.getInstance();
+      for (String tableId : Tables.getIdToNameMap(instance).keySet()) {
+        TableState state = manager.getTableState(tableId);
+        if (TableState.OFFLINE == state) {
+          clearMigrations(tableId);
+        }
+      }
+    }
   }
   
   private class StatusThread extends Daemon {
@@ -2418,6 +2435,9 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
   @Override
   public void stateChanged(String tableId, TableState state) {
     nextEvent.event("Table state in zookeeper changed for %s to %s", tableId, state);
+    if (TableState.OFFLINE == state) {
+      clearMigrations(tableId);
+    }
   }
   
   @Override

http://git-wip-us.apache.org/repos/asf/accumulo/blob/6bbe1216/server/src/main/java/org/apache/accumulo/server/master/balancer/ChaoticLoadBalancer.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/master/balancer/ChaoticLoadBalancer.java b/server/src/main/java/org/apache/accumulo/server/master/balancer/ChaoticLoadBalancer.java
index e14008a..92eca13 100644
--- a/server/src/main/java/org/apache/accumulo/server/master/balancer/ChaoticLoadBalancer.java
+++ b/server/src/main/java/org/apache/accumulo/server/master/balancer/ChaoticLoadBalancer.java
@@ -33,6 +33,7 @@ import org.apache.accumulo.core.tabletserver.thrift.TabletStats;
 import org.apache.accumulo.server.conf.ServerConfiguration;
 import org.apache.accumulo.server.master.state.TServerInstance;
 import org.apache.accumulo.server.master.state.TabletMigration;
+import org.apache.log4j.Logger;
 import org.apache.thrift.TException;
 
 /**
@@ -40,6 +41,7 @@ import org.apache.thrift.TException;
  * designed for performance, do not use on production systems. I'm calling it the LokiLoadBalancer.
  */
 public class ChaoticLoadBalancer extends TabletBalancer {
+  private static final Logger log = Logger.getLogger(ChaoticLoadBalancer.class);
   Random r = new Random();
   
   @Override
@@ -75,6 +77,8 @@ public class ChaoticLoadBalancer extends TabletBalancer {
     }
   }
   
+  protected final OutstandingMigrations outstandingMigrations = new OutstandingMigrations(log);
+
   /**
    * Will balance randomly, maintaining distribution
    */
@@ -83,8 +87,12 @@ public class ChaoticLoadBalancer extends TabletBalancer {
     Map<TServerInstance,Long> numTablets = new HashMap<TServerInstance,Long>();
     List<TServerInstance> underCapacityTServer = new ArrayList<TServerInstance>();
 
-    if (!migrations.isEmpty())
+    if (!migrations.isEmpty()) {
+      outstandingMigrations.migrations = migrations;
+      constraintNotMet(outstandingMigrations);
       return 100;
+    }
+    resetBalancerErrors();
 
     boolean moveMetadata = r.nextInt(4) == 0;
     long totalTablets = 0;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/6bbe1216/server/src/main/java/org/apache/accumulo/server/master/balancer/DefaultLoadBalancer.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/master/balancer/DefaultLoadBalancer.java b/server/src/main/java/org/apache/accumulo/server/master/balancer/DefaultLoadBalancer.java
index 1fcab46..9a970e7 100644
--- a/server/src/main/java/org/apache/accumulo/server/master/balancer/DefaultLoadBalancer.java
+++ b/server/src/main/java/org/apache/accumulo/server/master/balancer/DefaultLoadBalancer.java
@@ -302,16 +302,26 @@ public class DefaultLoadBalancer extends TabletBalancer {
       assignments.put(entry.getKey(), getAssignment(current, entry.getKey(), entry.getValue()));
     }
   }
-  
+
+  private static final NoTservers NO_SERVERS = new NoTservers(log);
+
+  protected final OutstandingMigrations outstandingMigrations = new OutstandingMigrations(log);
+
   @Override
   public long balance(SortedMap<TServerInstance,TabletServerStatus> current, Set<KeyExtent> migrations, List<TabletMigration> migrationsOut) {
     // do we have any servers?
     if (current.size() > 0) {
       // Don't migrate if we have migrations in progress
       if (migrations.size() == 0) {
+        resetBalancerErrors();
         if (getMigrations(current, migrationsOut))
           return 1 * 1000;
+      } else {
+        outstandingMigrations.migrations = migrations;
+        constraintNotMet(outstandingMigrations);
       }
+    } else {
+      constraintNotMet(NO_SERVERS);
     }
     return 5 * 1000;
   }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/6bbe1216/server/src/main/java/org/apache/accumulo/server/master/balancer/TabletBalancer.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/master/balancer/TabletBalancer.java b/server/src/main/java/org/apache/accumulo/server/master/balancer/TabletBalancer.java
index 69387d3..16c5dbc 100644
--- a/server/src/main/java/org/apache/accumulo/server/master/balancer/TabletBalancer.java
+++ b/server/src/main/java/org/apache/accumulo/server/master/balancer/TabletBalancer.java
@@ -17,11 +17,14 @@
 package org.apache.accumulo.server.master.balancer;
 
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.SortedMap;
 
+import com.google.common.collect.Iterables;
+
 import org.apache.accumulo.trace.instrument.Tracer;
 import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.data.KeyExtent;
@@ -67,6 +70,11 @@ public abstract class TabletBalancer {
   
   /**
    * Ask the balancer if any migrations are necessary.
+   *
+   * If the balancer is going to self-abort due to some environmental constraint (e.g. it requires some minimum number of tservers, or a maximum number
+   * of outstanding migrations), it should issue a log message to alert operators. The message should be at WARN normally and at ERROR if the balancer knows that the
+   * problem can not self correct. It should not issue these messages more than once a minute. Subclasses can use the convenience methods of {@link #constraintNotMet()} and
+   * {@link #balanceSuccessful()} to accomplish this logging.
    * 
    * @param current
    *          The current table-summary state of all the online tablet servers. Read-only.
@@ -79,6 +87,89 @@ public abstract class TabletBalancer {
    *         This method will not be called when there are unassigned tablets.
    */
   public abstract long balance(SortedMap<TServerInstance,TabletServerStatus> current, Set<KeyExtent> migrations, List<TabletMigration> migrationsOut);
+
+  private static final long ONE_SECOND = 1000l;
+  private boolean stuck = false;
+  private long stuckNotificationTime = -1l;
+
+  protected static final long TIME_BETWEEN_BALANCER_WARNINGS = 60 * ONE_SECOND;
+
+  /**
+   * A deferred call descendent TabletBalancers use to log why they can't continue.
+   * The call is deferred so that TabletBalancer can limit how often messages happen.
+   *
+   * Implementations should be reused as much as possible.
+   *
+   * Be sure to pass in a properly scoped Logger instance so that messages indicate
+   * what part of the system is having trouble.
+   */
+  protected static abstract class BalancerProblem implements Runnable {
+    protected final Logger balancerLog;
+    public BalancerProblem(Logger logger) {
+      balancerLog = logger;
+    }
+  }
+
+  /**
+   * If a TabletBalancer requires active tservers, it should use this problem to indicate when there are none.
+   * NoTservers is safe to share with anyone who uses the same Logger. TabletBalancers should have a single
+   * static instance.
+   */
+  protected static class NoTservers extends BalancerProblem {
+    public NoTservers(Logger logger) {
+      super(logger);
+    }
+
+    @Override
+    public void run() {
+      balancerLog.warn("Not balancing because we don't have any tservers");
+    }
+  }
+
+  /**
+   * If a TabletBalancer only balances when there are no outstanding migrations, it should use this problem
+   * to indicate when they exist.
+   *
+   * Iff a TabletBalancer makes use of the migrations member to provide samples, then OutstandingMigrations
+   * is not thread safe.
+   */
+  protected static class OutstandingMigrations extends BalancerProblem {
+    public Set<KeyExtent> migrations = Collections.<KeyExtent>emptySet();
+
+    public OutstandingMigrations(Logger logger) {
+      super(logger);
+    }
+
+    @Override
+    public void run() {
+      balancerLog.warn("Not balancing due to " + migrations.size() + " outstanding migrations.");
+      /* TODO ACCUMULO-2938 redact key extents in this output to avoid leaking protected information. */
+      balancerLog.debug("Sample up to 10 outstanding migrations: " + Iterables.limit(migrations, 10));
+    }
+  }
+
+  /**
+   * Warn that a Balancer can't work because of some external restriction.
+   * Will not call the provided logging handler  more often than TIME_BETWEEN_BALANCER_WARNINGS
+   */
+  protected void constraintNotMet(BalancerProblem cause) {
+    if (!stuck) {
+      stuck = true;
+      stuckNotificationTime = System.currentTimeMillis();
+    } else {
+      if ((System.currentTimeMillis() - stuckNotificationTime) > TIME_BETWEEN_BALANCER_WARNINGS) {
+        cause.run();
+        stuckNotificationTime = System.currentTimeMillis();
+      }
+    }
+  }
+
+  /**
+   * Resets logging about problems meeting an external constraint on balancing.
+   */
+  protected void resetBalancerErrors() {
+    stuck = false;
+  }
   
   /**
    * Fetch the tablets for the given table by asking the tablet server. Useful if your balance strategy needs details at the tablet level to decide what tablets

http://git-wip-us.apache.org/repos/asf/accumulo/blob/6bbe1216/test/system/auto/stress/migrations.py
----------------------------------------------------------------------
diff --git a/test/system/auto/stress/migrations.py b/test/system/auto/stress/migrations.py
index d07d7a8..9c94b88 100755
--- a/test/system/auto/stress/migrations.py
+++ b/test/system/auto/stress/migrations.py
@@ -76,7 +76,105 @@ class ChaoticBalancerIntegrity(SunnyDayTest):
         self.shell(self.masterHost(), 'flush -t test_ingest')
         self.waitForStop(self.verify(self.masterHost(), self.options.rows), 60)
 
+# Check for ACCUMULO-2694
+class BalanceInPresenceOfOfflineTable(SunnyDayTest):
+    """Start a new table, create many splits, and offline before they can rebalance. Then try to have a different table balance"""
+
+    order = 98
+
+    settings = TestUtilsMixin.settings.copy()
+    settings.update({
+        'tserver.memory.maps.max':'10K',
+        'tserver.compaction.major.delay': 0,
+        })
+    tableSettings = SunnyDayTest.tableSettings.copy()
+    tableSettings['test_ingest'] = {
+        'table.split.threshold': '10K',
+        }
+    def setUp(self):
+        # ensure we have two servers
+        if len(self.options.hosts) == 1:
+            self.options.hosts.append('localhost')
+        self.options.hosts = self.options.hosts[:2]
+
+        TestUtilsMixin.setUp(self);
+
+        # create a table with 200 splits
+        import tempfile
+        fileno, filename = tempfile.mkstemp()
+        fp = os.fdopen(fileno, "wb")
+        try:
+            for i in range(200):
+                fp.write("%08x\n" % (i * 1000))
+        finally:
+            fp.close()
+        self.createTable('unused', filename)
+        out,err,code = self.shell(self.masterHost(), 'offline -t unused\n')
+        self.processResult(out,err,code);
+
+        # create an empty table
+        self.createTable('test_ingest')
+
+    def runTest(self):
+
+        # start test ingestion
+        log.info("Starting Test Ingester")
+        self.ingester = self.ingest(self.masterHost(),
+                                    200000,
+                                    size=self.options.size)
+        self.waitForStop(self.ingester, self.timeout_factor * 120)
+        self.shell(self.masterHost(), 'flush -t test_ingest\n')
+        self.waitForStop(self.verify(self.masterHost(), self.options.rows), 60)
+
+        # let the server split tablets and move them around
+        # Keep retrying until the wait period for migration cleanup has passed
+        # which is hard coded to 5 minutes. :/
+        startTime = time.clock()
+        currentWait = 10
+        balancingWorked = False
+        while ((time.clock() - startTime) < 5*60+15):
+            self.sleep(currentWait)
+            # If we end up needing to sleep again, back off.
+            currentWait *= 2
+
+            # fetch the list of tablets from each server
+            h = self.runOn(self.masterHost(),
+                           [self.accumulo_sh(),
+                            'org.apache.accumulo.test.GetMasterStats'])
+            out, err = h.communicate()
+            servers = {}
+            server = None
+            # if balanced based on ingest, the table that split due to ingest
+            # will be split evenly on both servers, not just one
+            table = ''
+            tableId = self.getTableId('test_ingest');
+            for line in out.split('\n'):
+                if line.find(' Name: ') == 0:
+                    server = line[7:]
+                    servers.setdefault(server, 0)
+                if line.find('Table: ') >= 0:
+                    table = line.split(' ')[-1]
+                if line.find('    Tablets: ') == 0:
+                    if table == tableId:
+                       servers[server] += int(line.split()[-1])
+            log.info("Tablet counts " + repr(servers))
+
+            # we have two servers
+            if len(servers.values()) == 2:
+              servers = servers.values()
+              # a server has more than 10 splits
+              if servers[0] > 10:
+                # the ratio is roughly even
+                ratio = min(servers) / float(max(servers))
+                if ratio > 0.5:
+                  balancingWorked = True
+                  break
+            log.debug("tablets not balanced, sleeping for %d seconds" % currentWait)
+
+        self.assert_(balancingWorked)
+
 def suite():
     result = unittest.TestSuite()
     result.addTest(ChaoticBalancerIntegrity())
+    result.addTest(BalanceInPresenceOfOfflineTable())
     return result


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

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

Conflicts:
	server/base/src/main/java/org/apache/accumulo/server/master/balancer/ChaoticLoadBalancer.java


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

Branch: refs/heads/master
Commit: 7b385fed5ff15d1bbd8a5e9510913b20664c2f24
Parents: 3c11090 533983f
Author: Sean Busbey <bu...@cloudera.com>
Authored: Sun Aug 3 00:33:48 2014 -0500
Committer: Sean Busbey <bu...@cloudera.com>
Committed: Sun Aug 3 00:33:48 2014 -0500

----------------------------------------------------------------------
 .../core/client/AccumuloSecurityException.java  |   8 +
 .../impl/MiniAccumuloClusterImpl.java           |  32 ++++
 .../impl/MiniAccumuloClusterImplTest.java       |  37 +++++
 minicluster/src/test/resources/log4j.properties |  22 ++-
 .../master/balancer/ChaoticLoadBalancer.java    |  12 +-
 .../master/balancer/DefaultLoadBalancer.java    |  12 +-
 .../server/master/balancer/TabletBalancer.java  |  91 +++++++++++
 .../server/security/SystemCredentials.java      |  21 ++-
 .../java/org/apache/accumulo/master/Master.java |  34 ++++-
 test/pom.xml                                    |   5 +
 .../BalanceInPresenceOfOfflineTableIT.java      | 152 +++++++++++++++++++
 11 files changed, 411 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/7b385fed/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImpl.java
----------------------------------------------------------------------
diff --cc minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImpl.java
index a037859,deb04d9..392894c
--- a/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImpl.java
+++ b/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImpl.java
@@@ -59,8 -61,11 +61,10 @@@ import org.apache.accumulo.core.client.
  import org.apache.accumulo.core.client.security.tokens.PasswordToken;
  import org.apache.accumulo.core.conf.Property;
  import org.apache.accumulo.core.master.thrift.MasterGoalState;
+ import org.apache.accumulo.core.master.thrift.MasterClientService;
+ import org.apache.accumulo.core.master.thrift.MasterMonitorInfo;
  import org.apache.accumulo.core.util.Daemon;
  import org.apache.accumulo.core.util.Pair;
 -import org.apache.accumulo.core.util.StringUtil;
  import org.apache.accumulo.core.util.UtilWaitThread;
  import org.apache.accumulo.gc.SimpleGarbageCollector;
  import org.apache.accumulo.master.Master;
@@@ -81,9 -88,9 +87,10 @@@ import org.apache.hadoop.fs.FileSystem
  import org.apache.hadoop.hdfs.DFSConfigKeys;
  import org.apache.hadoop.hdfs.MiniDFSCluster;
  import org.apache.log4j.Logger;
+ import org.apache.thrift.TException;
  import org.apache.zookeeper.server.ZooKeeperServerMain;
  
 +import com.google.common.base.Joiner;
  import com.google.common.base.Predicate;
  import com.google.common.collect.Maps;
  

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7b385fed/server/base/src/main/java/org/apache/accumulo/server/master/balancer/ChaoticLoadBalancer.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7b385fed/server/base/src/main/java/org/apache/accumulo/server/master/balancer/DefaultLoadBalancer.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7b385fed/server/base/src/main/java/org/apache/accumulo/server/master/balancer/TabletBalancer.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7b385fed/server/base/src/main/java/org/apache/accumulo/server/security/SystemCredentials.java
----------------------------------------------------------------------

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

http://git-wip-us.apache.org/repos/asf/accumulo/blob/7b385fed/test/pom.xml
----------------------------------------------------------------------


[7/9] git commit: ACCUMULO-2694 update migration cleanup for 1.6 branch.

Posted by bu...@apache.org.
ACCUMULO-2694 update migration cleanup for 1.6 branch.

- manually redo changes to Master
- convert python functional test to an IT
-- Add programmatic access to MasterMonitorInfo to MiniAccumuloClusterImpl
-- Make SystemCredentials work with MiniAccumuloClusterImpl


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

Branch: refs/heads/1.6.1-SNAPSHOT
Commit: 533983f10136ce581a8b877379b19eb2302fa578
Parents: dedc9cd
Author: Sean Busbey <bu...@cloudera.com>
Authored: Thu Jun 26 11:11:39 2014 -0500
Committer: Sean Busbey <bu...@cloudera.com>
Committed: Fri Aug 1 19:32:25 2014 -0500

----------------------------------------------------------------------
 .../core/client/AccumuloSecurityException.java  |   8 +
 .../impl/MiniAccumuloClusterImpl.java           |  32 ++++
 .../impl/MiniAccumuloClusterImplTest.java       |  37 +++++
 minicluster/src/test/resources/log4j.properties |  22 ++-
 .../server/security/SystemCredentials.java      |  21 ++-
 .../java/org/apache/accumulo/master/Master.java |  34 ++++-
 test/pom.xml                                    |   5 +
 .../BalanceInPresenceOfOfflineTableIT.java      | 152 +++++++++++++++++++
 8 files changed, 300 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/533983f1/core/src/main/java/org/apache/accumulo/core/client/AccumuloSecurityException.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/AccumuloSecurityException.java b/core/src/main/java/org/apache/accumulo/core/client/AccumuloSecurityException.java
index 06b148d..35ea188 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/AccumuloSecurityException.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/AccumuloSecurityException.java
@@ -75,6 +75,14 @@ public class AccumuloSecurityException extends Exception {
   }
   
   /**
+   * Construct a user-facing exception from a serialized version.
+   * @param thrift a serialized version
+   */
+  public AccumuloSecurityException(final ThriftSecurityException thrift) {
+    this(thrift.getUser(), thrift.getCode(), thrift);
+  }
+
+  /**
    * @param user
    *          the relevant user for the security violation
    * @param errorcode

http://git-wip-us.apache.org/repos/asf/accumulo/blob/533983f1/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImpl.java
----------------------------------------------------------------------
diff --git a/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImpl.java b/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImpl.java
index 9a8efa6..deb04d9 100644
--- a/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImpl.java
+++ b/minicluster/src/main/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImpl.java
@@ -56,9 +56,13 @@ import org.apache.accumulo.core.client.ClientConfiguration;
 import org.apache.accumulo.core.client.Connector;
 import org.apache.accumulo.core.client.Instance;
 import org.apache.accumulo.core.client.ZooKeeperInstance;
+import org.apache.accumulo.core.client.impl.MasterClient;
+import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.client.security.tokens.PasswordToken;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.master.thrift.MasterGoalState;
+import org.apache.accumulo.core.master.thrift.MasterClientService;
+import org.apache.accumulo.core.master.thrift.MasterMonitorInfo;
 import org.apache.accumulo.core.util.Daemon;
 import org.apache.accumulo.core.util.Pair;
 import org.apache.accumulo.core.util.StringUtil;
@@ -68,10 +72,12 @@ import org.apache.accumulo.master.Master;
 import org.apache.accumulo.master.state.SetGoalState;
 import org.apache.accumulo.minicluster.ServerType;
 import org.apache.accumulo.server.init.Initialize;
+import org.apache.accumulo.server.security.SystemCredentials;
 import org.apache.accumulo.server.util.PortUtils;
 import org.apache.accumulo.server.util.time.SimpleTimer;
 import org.apache.accumulo.start.Main;
 import org.apache.accumulo.start.classloader.vfs.MiniDFSUtil;
+import org.apache.accumulo.trace.instrument.Tracer;
 import org.apache.accumulo.tserver.TabletServer;
 import org.apache.commons.configuration.MapConfiguration;
 import org.apache.commons.io.FileUtils;
@@ -82,6 +88,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.log4j.Logger;
+import org.apache.thrift.TException;
 import org.apache.zookeeper.server.ZooKeeperServerMain;
 
 import com.google.common.base.Predicate;
@@ -742,4 +749,29 @@ public class MiniAccumuloClusterImpl implements AccumuloCluster {
 
     return future.get(timeout, unit);
   }
+
+  /**
+   * Get programmatic interface to information available in a normal monitor.
+   * XXX the returned structure won't contain information about the metadata table until there is data in it.
+   * e.g. if you want to see the metadata table you should create a table.
+   * @since 1.6.1
+   */
+  public MasterMonitorInfo getMasterMonitorInfo() throws AccumuloException, AccumuloSecurityException {
+    MasterClientService.Iface client = null;
+    MasterMonitorInfo stats = null;
+    try {
+      Instance instance = new ZooKeeperInstance(getClientConfig());
+      client = MasterClient.getConnectionWithRetry(instance);
+      stats = client.getMasterStats(Tracer.traceInfo(), SystemCredentials.get(instance).toThrift(instance));
+    } catch (ThriftSecurityException exception) {
+      throw new AccumuloSecurityException(exception);
+    } catch (TException exception) {
+      throw new AccumuloException(exception);
+    } finally {
+      if (client != null) {
+        MasterClient.close(client);
+      }
+    }
+    return stats;
+  }
 }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/533983f1/minicluster/src/test/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImplTest.java
----------------------------------------------------------------------
diff --git a/minicluster/src/test/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImplTest.java b/minicluster/src/test/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImplTest.java
index 99f8d7d..32b20b0 100644
--- a/minicluster/src/test/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImplTest.java
+++ b/minicluster/src/test/java/org/apache/accumulo/minicluster/impl/MiniAccumuloClusterImplTest.java
@@ -17,9 +17,17 @@
 package org.apache.accumulo.minicluster.impl;
 
 import java.io.File;
+import java.util.Arrays;
 import java.util.Collection;
+import java.util.List;
 import java.util.Map;
 
+import org.apache.accumulo.core.client.admin.TableOperations;
+import org.apache.accumulo.core.master.thrift.MasterMonitorInfo;
+import org.apache.accumulo.core.master.thrift.MasterGoalState;
+import org.apache.accumulo.core.master.thrift.MasterState;
+import org.apache.accumulo.core.metadata.MetadataTable;
+import org.apache.accumulo.core.metadata.RootTable;
 import org.apache.accumulo.minicluster.ServerType;
 import org.apache.commons.io.FileUtils;
 import org.apache.log4j.Level;
@@ -31,10 +39,17 @@ import org.junit.Test;
 
 public class MiniAccumuloClusterImplTest {
 
+  private static final Logger log = Logger.getLogger(MiniAccumuloClusterImplTest.class);
+
   public static File testDir;
 
   private static MiniAccumuloClusterImpl accumulo;
 
+  private static final int NUM_TSERVERS = 2;
+
+  private static String TEST_TABLE = "test";
+  private static String testTableID;
+
   @BeforeClass
   public static void setupMiniCluster() throws Exception {
     Logger.getLogger("org.apache.zookeeper").setLevel(Level.ERROR);
@@ -46,8 +61,14 @@ public class MiniAccumuloClusterImplTest {
     testDir.mkdir();
 
     MiniAccumuloConfigImpl config = new MiniAccumuloConfigImpl(testDir, "superSecret").setJDWPEnabled(true);
+    // expressly set number of tservers since we assert it later, in case the default changes
+    config.setNumTservers(NUM_TSERVERS);
     accumulo = new MiniAccumuloClusterImpl(config);
     accumulo.start();
+    // create a table to ensure there are some entries in the !0 table
+    TableOperations tableops = accumulo.getConnector("root","superSecret").tableOperations();
+    tableops.create(TEST_TABLE);
+    testTableID = tableops.tableIdMap().get(TEST_TABLE);
   }
 
   @Test(timeout = 10000)
@@ -67,6 +88,22 @@ public class MiniAccumuloClusterImplTest {
     }
   }
 
+  @Test
+  public void saneMonitorInfo() throws Exception {
+    log.info("ensure monitor info includes some base information.");
+    MasterMonitorInfo stats = accumulo.getMasterMonitorInfo();
+    List<MasterState> validStates = Arrays.asList(MasterState.values());
+    List<MasterGoalState> validGoals = Arrays.asList(MasterGoalState.values());
+    Assert.assertTrue("master state should be valid.", validStates.contains(stats.state));
+    Assert.assertTrue("master goal state should be in " + validGoals + ". is " + stats.goalState, validGoals.contains(stats.goalState));
+    Assert.assertNotNull("should have a table map.", stats.tableMap);
+    Assert.assertTrue("root table should exist in " + stats.tableMap.keySet(), stats.tableMap.keySet().contains(RootTable.ID));
+    Assert.assertTrue("meta table should exist in " + stats.tableMap.keySet(), stats.tableMap.keySet().contains(MetadataTable.ID));
+    Assert.assertTrue("our test table should exist in " + stats.tableMap.keySet(), stats.tableMap.keySet().contains(testTableID));
+    Assert.assertNotNull("there should be tservers.", stats.tServerInfo);
+    Assert.assertEquals(NUM_TSERVERS, stats.tServerInfo.size());
+  }
+
   @AfterClass
   public static void tearDownMiniCluster() throws Exception {
     accumulo.stop();

http://git-wip-us.apache.org/repos/asf/accumulo/blob/533983f1/minicluster/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/minicluster/src/test/resources/log4j.properties b/minicluster/src/test/resources/log4j.properties
index b5efe8d..75dfd78 100644
--- a/minicluster/src/test/resources/log4j.properties
+++ b/minicluster/src/test/resources/log4j.properties
@@ -1 +1,21 @@
-# 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.

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} [%-8c{2}] %-5p: %m%n

log4j
 .logger.org.apache.accumulo.core.client.impl.ServerClient=ERROR
\ No newline at end of file
+# 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.
+
+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} [%-8c{2}] %-5p: %m%n
+
+log4j.logger.org.apache.accumulo.core.client.impl.ServerClient=ERROR

http://git-wip-us.apache.org/repos/asf/accumulo/blob/533983f1/server/base/src/main/java/org/apache/accumulo/server/security/SystemCredentials.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/security/SystemCredentials.java b/server/base/src/main/java/org/apache/accumulo/server/security/SystemCredentials.java
index 767ed25..19e7ff7 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/security/SystemCredentials.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/security/SystemCredentials.java
@@ -57,14 +57,29 @@ public final class SystemCredentials extends Credentials {
   }
 
   public static SystemCredentials get() {
+    check_permission();
+    if (SYSTEM_CREDS == null) {
+      SYSTEM_CREDS = new SystemCredentials(HdfsZooInstance.getInstance());
+    }
+    return SYSTEM_CREDS;
+  }
+
+  private static void check_permission() {
     SecurityManager sm = System.getSecurityManager();
     if (sm != null) {
       sm.checkPermission(SYSTEM_CREDENTIALS_PERMISSION);
     }
-    if (SYSTEM_CREDS == null) {
-      SYSTEM_CREDS = new SystemCredentials(HdfsZooInstance.getInstance());
+  }
+
+  public static SystemCredentials get(Instance instance) {
+    check_permission();
+    /* Special case to avoid duplicating SYSTEM_CREDS */
+    if (null != SYSTEM_CREDS) {
+      if (SYSTEM_CREDS.AS_THRIFT.getInstanceId().equals(instance.getInstanceID())) {
+        return SYSTEM_CREDS;
+      }
     }
-    return SYSTEM_CREDS;
+    return new SystemCredentials(instance);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/accumulo/blob/533983f1/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 30b1f2e..2f04922 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
@@ -715,7 +715,8 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
       while (stillMaster()) {
         if (!migrations.isEmpty()) {
           try {
-            cleanupMutations();
+            cleanupOfflineMigrations();
+            cleanupNonexistentMigrations(getConnector());
           } catch (Exception ex) {
             log.error("Error cleaning up migrations", ex);
           }
@@ -724,12 +725,13 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
       }
     }
 
-    // If a migrating tablet splits, and the tablet dies before sending the
-    // master a message, the migration will refer to a non-existing tablet,
-    // so it can never complete. Periodically scan the metadata table and
-    // remove any migrating tablets that no longer exist.
-    private void cleanupMutations() throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
-      Connector connector = getConnector();
+    /**
+     * If a migrating tablet splits, and the tablet dies before sending the
+     * master a message, the migration will refer to a non-existing tablet,
+     * so it can never complete. Periodically scan the metadata table and
+     * remove any migrating tablets that no longer exist.
+     */
+    private void cleanupNonexistentMigrations(final Connector connector) throws AccumuloException, AccumuloSecurityException, TableNotFoundException {
       Scanner scanner = connector.createScanner(MetadataTable.NAME, Authorizations.EMPTY);
       TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN.fetch(scanner);
       Set<KeyExtent> found = new HashSet<KeyExtent>();
@@ -741,6 +743,21 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
       }
       migrations.keySet().retainAll(found);
     }
+
+    /**
+     * If migrating a tablet for a table that is offline, the migration
+     * can never succeed because no tablet server will load the tablet.
+     * check for offline tables and remove their migrations.
+     */
+    private void cleanupOfflineMigrations() {
+      TableManager manager = TableManager.getInstance();
+      for (String tableId : Tables.getIdToNameMap(instance).keySet()) {
+        TableState state = manager.getTableState(tableId);
+        if (TableState.OFFLINE == state) {
+          clearMigrations(tableId);
+        }
+      }
+    }
   }
 
   private class StatusThread extends Daemon {
@@ -1206,6 +1223,9 @@ public class Master implements LiveTServerSet.Listener, TableObserver, CurrentSt
   @Override
   public void stateChanged(String tableId, TableState state) {
     nextEvent.event("Table state in zookeeper changed for %s to %s", tableId, state);
+    if (TableState.OFFLINE == state) {
+      clearMigrations(tableId);
+    }
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/accumulo/blob/533983f1/test/pom.xml
----------------------------------------------------------------------
diff --git a/test/pom.xml b/test/pom.xml
index f3cc927..17c8527 100644
--- a/test/pom.xml
+++ b/test/pom.xml
@@ -137,6 +137,11 @@
       <scope>test</scope>
     </dependency>
     <dependency>
+      <groupId>commons-lang</groupId>
+      <artifactId>commons-lang</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>
       <scope>test</scope>

http://git-wip-us.apache.org/repos/asf/accumulo/blob/533983f1/test/src/test/java/org/apache/accumulo/test/functional/BalanceInPresenceOfOfflineTableIT.java
----------------------------------------------------------------------
diff --git a/test/src/test/java/org/apache/accumulo/test/functional/BalanceInPresenceOfOfflineTableIT.java b/test/src/test/java/org/apache/accumulo/test/functional/BalanceInPresenceOfOfflineTableIT.java
new file mode 100644
index 0000000..37aa8a1
--- /dev/null
+++ b/test/src/test/java/org/apache/accumulo/test/functional/BalanceInPresenceOfOfflineTableIT.java
@@ -0,0 +1,152 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.accumulo.test.functional;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
+import org.apache.accumulo.core.client.AccumuloException;
+import org.apache.accumulo.core.client.AccumuloSecurityException;
+import org.apache.accumulo.core.client.TableExistsException;
+import org.apache.accumulo.core.client.TableNotFoundException;
+import org.apache.accumulo.core.client.Connector;
+import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.master.thrift.MasterMonitorInfo;
+import org.apache.accumulo.core.master.thrift.TableInfo;
+import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl;
+import org.apache.accumulo.test.TestIngest;
+import org.apache.accumulo.test.VerifyIngest;
+import org.apache.commons.lang.math.NumberUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Text;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Start a new table, create many splits, and offline before they can rebalance. Then try to have a different table balance
+ */
+public class BalanceInPresenceOfOfflineTableIT extends ConfigurableMacIT {
+
+  private static Logger log = LoggerFactory.getLogger(BalanceInPresenceOfOfflineTableIT.class);
+
+  @Override
+  public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) {
+    Map<String,String> siteConfig = new HashMap<String, String>();
+    siteConfig.put(Property.TSERV_MAXMEM.getKey(), "10K");
+    siteConfig.put(Property.TSERV_MAJC_DELAY.getKey(), "0");
+    cfg.setSiteConfig(siteConfig );
+    // ensure we have two tservers
+    if (cfg.getNumTservers() < 2) {
+      cfg.setNumTservers(2);
+    }
+  }
+
+  @Override
+  protected int defaultTimeoutSeconds() {
+    return 10 * 60;
+  }
+
+  private static final int NUM_SPLITS = 200;
+  private static final String UNUSED_TABLE = "unused";
+  private static final String TEST_TABLE = "test_ingest";
+
+  private Connector connector;
+
+  @Before
+  public void setupTables() throws AccumuloException, AccumuloSecurityException, TableExistsException, TableNotFoundException {
+    // set up splits
+    final SortedSet<Text> splits = new TreeSet<Text>();
+    for (int i = 0; i < NUM_SPLITS; i++) {
+      splits.add(new Text(String.format("%08x", i * 1000)));
+    }
+    // load into a table we won't use
+    connector = getConnector();
+    connector.tableOperations().create(UNUSED_TABLE);
+    connector.tableOperations().addSplits(UNUSED_TABLE, splits);
+    // mark the table offline before it can rebalance.
+    connector.tableOperations().offline(UNUSED_TABLE);
+
+    // actual test table
+    connector.tableOperations().create(TEST_TABLE);
+    connector.tableOperations().setProperty(TEST_TABLE, Property.TABLE_SPLIT_THRESHOLD.getKey(), "10K");
+  }
+
+  @Test
+  public void test() throws Exception {
+    log.info("Test that balancing is not stopped by an offline table with outstanding migrations.");
+
+    log.debug("starting test ingestion");
+
+    TestIngest.Opts opts = new TestIngest.Opts();
+    VerifyIngest.Opts vopts = new VerifyIngest.Opts();
+    vopts.rows = opts.rows = 200000;
+    TestIngest.ingest(connector, opts, BWOPTS);
+    connector.tableOperations().flush(TEST_TABLE, null, null, true);
+    VerifyIngest.verifyIngest(connector, vopts, SOPTS);
+
+    log.debug("waiting for balancing, up to ~5 minutes to allow for migration cleanup.");
+    final long startTime = System.currentTimeMillis();
+    long currentWait = 10*1000;
+    boolean balancingWorked = false;
+
+    while (!balancingWorked && (System.currentTimeMillis() - startTime) < ((5*60 + 15)*1000)) {
+      Thread.sleep(currentWait);
+      currentWait *= 2;
+
+      log.debug("fetch the list of tablets assigned to each tserver.");
+      MasterMonitorInfo stats = getCluster().getMasterMonitorInfo();
+
+      if (stats.getTServerInfoSize() < 2) {
+        log.debug("we need >= 2 servers. sleeping for " + currentWait + "ms");
+        continue;
+      }
+      if (stats.getUnassignedTablets() != 0) {
+        log.debug("We shouldn't have unassigned tablets. sleeping for " + currentWait + "ms");
+        continue;
+      }
+
+      int numTablets = 0;
+      long[] tabletsPerServer = new long[stats.getTServerInfoSize()];
+      Arrays.fill(tabletsPerServer, 0l);
+      for (int i = 0; i < stats.getTServerInfoSize(); i++) {
+        for (Map.Entry<String, TableInfo> entry : stats.getTServerInfo().get(i).getTableMap().entrySet()) {
+          tabletsPerServer[i] += entry.getValue().getTablets();
+        }
+      }
+
+      if (tabletsPerServer[0] <= 10) {
+        log.debug("We should have > 10 tablets. sleeping for " + currentWait + "ms");
+        continue;
+      }
+      if ((NumberUtils.min(tabletsPerServer) / ((double)NumberUtils.max(tabletsPerServer))) < 0.5) {
+        log.debug("ratio of min to max tablets per server should be roughly even. sleeping for " + currentWait + "ms");
+        continue;
+      }
+      balancingWorked = true;
+    }
+
+    Assert.assertTrue("did not properly balance", balancingWorked);
+  }
+
+}


[9/9] git commit: ACCUMULO-2694 fix dependency in test module for master branch.

Posted by bu...@apache.org.
ACCUMULO-2694 fix dependency in test module for master branch.

the test module code for replication made use of commons-lang without
declaring the dependency, relying on a transitive inclusion from
commons-configuration.

since the test for balancing in the face of an offline table added
commons-lang at test scope, the transitive inclusion was lost and
compilation fails.


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

Branch: refs/heads/master
Commit: f34f1d8925ef3dc22691f5b4878ebd1f15468b16
Parents: 7b385fe
Author: Sean Busbey <bu...@cloudera.com>
Authored: Sun Aug 3 01:26:07 2014 -0500
Committer: Sean Busbey <bu...@cloudera.com>
Committed: Sun Aug 3 01:26:07 2014 -0500

----------------------------------------------------------------------
 test/pom.xml | 9 ++++-----
 1 file changed, 4 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/f34f1d89/test/pom.xml
----------------------------------------------------------------------
diff --git a/test/pom.xml b/test/pom.xml
index 450e385..382381c 100644
--- a/test/pom.xml
+++ b/test/pom.xml
@@ -54,6 +54,10 @@
       <artifactId>commons-io</artifactId>
     </dependency>
     <dependency>
+      <groupId>commons-lang</groupId>
+      <artifactId>commons-lang</artifactId>
+    </dependency>
+    <dependency>
       <groupId>jline</groupId>
       <artifactId>jline</artifactId>
     </dependency>
@@ -141,11 +145,6 @@
       <scope>test</scope>
     </dependency>
     <dependency>
-      <groupId>commons-lang</groupId>
-      <artifactId>commons-lang</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
       <groupId>junit</groupId>
       <artifactId>junit</artifactId>
       <scope>test</scope>


[4/9] git commit: Merge branch '1.5.2-SNAPSHOT' into 1.6.1-SNAPSHOT

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

Conflicts:
	server/base/src/main/java/org/apache/accumulo/server/master/balancer/ChaoticLoadBalancer.java
	server/base/src/main/java/org/apache/accumulo/server/master/balancer/TabletBalancer.java
	server/src/main/java/org/apache/accumulo/server/master/Master.java
	server/tserver/pom.xml
	test/system/auto/stress/migrations.py


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

Branch: refs/heads/1.6.1-SNAPSHOT
Commit: dedc9cddaaa46558f542b4d9d6d357316815378f
Parents: 39f405a 6bbe121
Author: Sean Busbey <bu...@cloudera.com>
Authored: Fri Aug 1 19:31:51 2014 -0500
Committer: Sean Busbey <bu...@cloudera.com>
Committed: Fri Aug 1 19:31:51 2014 -0500

----------------------------------------------------------------------
 .../master/balancer/ChaoticLoadBalancer.java    | 14 ++-
 .../master/balancer/DefaultLoadBalancer.java    | 12 ++-
 .../server/master/balancer/TabletBalancer.java  | 91 ++++++++++++++++++++
 3 files changed, 113 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/dedc9cdd/server/base/src/main/java/org/apache/accumulo/server/master/balancer/ChaoticLoadBalancer.java
----------------------------------------------------------------------
diff --cc server/base/src/main/java/org/apache/accumulo/server/master/balancer/ChaoticLoadBalancer.java
index ec3371c,0000000..5767934
mode 100644,000000..100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/ChaoticLoadBalancer.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/balancer/ChaoticLoadBalancer.java
@@@ -1,143 -1,0 +1,151 @@@
 +/*
 + * 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.master.balancer;
 +
 +import java.util.ArrayList;
 +import java.util.HashMap;
 +import java.util.List;
 +import java.util.Map;
 +import java.util.Map.Entry;
 +import java.util.Random;
 +import java.util.Set;
 +import java.util.SortedMap;
 +
 +import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
 +import org.apache.accumulo.core.data.KeyExtent;
 +import org.apache.accumulo.core.master.thrift.TableInfo;
 +import org.apache.accumulo.core.master.thrift.TabletServerStatus;
 +import org.apache.accumulo.core.metadata.MetadataTable;
 +import org.apache.accumulo.core.tabletserver.thrift.TabletStats;
 +import org.apache.accumulo.server.conf.ServerConfiguration;
 +import org.apache.accumulo.server.master.state.TServerInstance;
 +import org.apache.accumulo.server.master.state.TabletMigration;
++import org.apache.log4j.Logger;
 +import org.apache.thrift.TException;
 +
 +/**
 + * A chaotic load balancer used for testing. It constantly shuffles tablets, preventing them from resting in a single location for very long. This is not
 + * designed for performance, do not use on production systems. I'm calling it the LokiLoadBalancer.
 + */
 +public class ChaoticLoadBalancer extends TabletBalancer {
++  private static final Logger log = Logger.getLogger(ChaoticLoadBalancer.class);
 +  Random r = new Random();
 +  
 +  @Override
 +  public void getAssignments(SortedMap<TServerInstance,TabletServerStatus> current, Map<KeyExtent,TServerInstance> unassigned,
 +      Map<KeyExtent,TServerInstance> assignments) {
 +    long total = assignments.size() + unassigned.size();
 +    long avg = (long) Math.ceil(((double) total) / current.size());
 +    Map<TServerInstance,Long> toAssign = new HashMap<TServerInstance,Long>();
 +    List<TServerInstance> tServerArray = new ArrayList<TServerInstance>();
 +    for (Entry<TServerInstance,TabletServerStatus> e : current.entrySet()) {
 +      long numTablets = 0;
 +      for (TableInfo ti : e.getValue().getTableMap().values()) {
 +        numTablets += ti.tablets;
 +      }
 +      if (numTablets < avg) {
 +        tServerArray.add(e.getKey());
 +        toAssign.put(e.getKey(), avg - numTablets);
 +      }
 +    }
 +    
 +    for (KeyExtent ke : unassigned.keySet()) {
 +      int index = r.nextInt(tServerArray.size());
 +      TServerInstance dest = tServerArray.get(index);
 +      assignments.put(ke, dest);
 +      long remaining = toAssign.get(dest).longValue() - 1;
 +      if (remaining == 0) {
 +        tServerArray.remove(index);
 +        toAssign.remove(dest);
 +      } else {
 +        toAssign.put(dest, remaining);
 +      }
 +    }
 +  }
 +  
++  protected final OutstandingMigrations outstandingMigrations = new OutstandingMigrations(log);
++
 +  /**
 +   * Will balance randomly, maintaining distribution
 +   */
 +  @Override
 +  public long balance(SortedMap<TServerInstance,TabletServerStatus> current, Set<KeyExtent> migrations, List<TabletMigration> migrationsOut) {
 +    Map<TServerInstance,Long> numTablets = new HashMap<TServerInstance,Long>();
 +    List<TServerInstance> underCapacityTServer = new ArrayList<TServerInstance>();
-     
-     if (!migrations.isEmpty())
++
++    if (!migrations.isEmpty()) {
++      outstandingMigrations.migrations = migrations;
++      constraintNotMet(outstandingMigrations);
 +      return 100;
-     
++    }
++    resetBalancerErrors();
++
 +    boolean moveMetadata = r.nextInt(4) == 0;
 +    long totalTablets = 0;
 +    for (Entry<TServerInstance,TabletServerStatus> e : current.entrySet()) {
 +      long tabletCount = 0;
 +      for (TableInfo ti : e.getValue().getTableMap().values()) {
 +        tabletCount += ti.tablets;
 +      }
 +      numTablets.put(e.getKey(), tabletCount);
 +      underCapacityTServer.add(e.getKey());
 +      totalTablets += tabletCount;
 +    }
 +    // totalTablets is fuzzy due to asynchronicity of the stats
 +    // *1.2 to handle fuzziness, and prevent locking for 'perfect' balancing scenarios
 +    long avg = (long) Math.ceil(((double) totalTablets) / current.size() * 1.2);
 +    
 +    for (Entry<TServerInstance,TabletServerStatus> e : current.entrySet()) {
 +      for (String table : e.getValue().getTableMap().keySet()) {
 +        if (!moveMetadata && MetadataTable.NAME.equals(table))
 +          continue;
 +        try {
 +          for (TabletStats ts : getOnlineTabletsForTable(e.getKey(), table)) {
 +            KeyExtent ke = new KeyExtent(ts.extent);
 +            int index = r.nextInt(underCapacityTServer.size());
 +            TServerInstance dest = underCapacityTServer.get(index);
 +            if (dest.equals(e.getKey()))
 +              continue;
 +            migrationsOut.add(new TabletMigration(ke, e.getKey(), dest));
 +            if (numTablets.put(dest, numTablets.get(dest) + 1) > avg)
 +              underCapacityTServer.remove(index);
 +            if (numTablets.put(e.getKey(), numTablets.get(e.getKey()) - 1) <= avg && !underCapacityTServer.contains(e.getKey()))
 +              underCapacityTServer.add(e.getKey());
 +            
 +            // We can get some craziness with only 1 tserver, so lets make sure there's always an option!
 +            if (underCapacityTServer.isEmpty())
 +              underCapacityTServer.addAll(numTablets.keySet());
 +          }
 +        } catch (ThriftSecurityException e1) {
 +          // Shouldn't happen, but carry on if it does
 +          e1.printStackTrace();
 +        } catch (TException e1) {
 +          // Shouldn't happen, but carry on if it does
 +          e1.printStackTrace();
 +        }
 +      }
 +    }
 +    
 +    return 100;
 +  }
 +  
 +  @Override
 +  public void init(ServerConfiguration conf) {
 +    super.init(conf);
 +  }
 +  
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/dedc9cdd/server/base/src/main/java/org/apache/accumulo/server/master/balancer/DefaultLoadBalancer.java
----------------------------------------------------------------------
diff --cc server/base/src/main/java/org/apache/accumulo/server/master/balancer/DefaultLoadBalancer.java
index 1fcab46,0000000..9a970e7
mode 100644,000000..100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/DefaultLoadBalancer.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/balancer/DefaultLoadBalancer.java
@@@ -1,319 -1,0 +1,329 @@@
 +/*
 + * 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.master.balancer;
 +
 +import java.util.ArrayList;
 +import java.util.Collections;
 +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;
 +import java.util.SortedMap;
 +
 +import org.apache.accumulo.core.data.KeyExtent;
 +import org.apache.accumulo.core.master.thrift.TableInfo;
 +import org.apache.accumulo.core.master.thrift.TabletServerStatus;
 +import org.apache.accumulo.core.tabletserver.thrift.TabletStats;
 +import org.apache.accumulo.server.master.state.TServerInstance;
 +import org.apache.accumulo.server.master.state.TabletMigration;
 +import org.apache.log4j.Logger;
 +
 +public class DefaultLoadBalancer extends TabletBalancer {
 +  
 +  private static final Logger log = Logger.getLogger(DefaultLoadBalancer.class);
 +  
 +  Iterator<TServerInstance> assignments;
 +  // if tableToBalance is set, then only balance the given table
 +  String tableToBalance = null;
 +  
 +  public DefaultLoadBalancer() {
 +    
 +  }
 +  
 +  public DefaultLoadBalancer(String table) {
 +    tableToBalance = table;
 +  }
 +  
 +  List<TServerInstance> randomize(Set<TServerInstance> locations) {
 +    List<TServerInstance> result = new ArrayList<TServerInstance>(locations);
 +    Collections.shuffle(result);
 +    return result;
 +  }
 +  
 +  public TServerInstance getAssignment(SortedMap<TServerInstance,TabletServerStatus> locations, KeyExtent extent, TServerInstance last) {
 +    if (locations.size() == 0)
 +      return null;
 +    
 +    if (last != null) {
 +      // Maintain locality
 +      String fakeSessionID = " ";
 +      TServerInstance simple = new TServerInstance(last.getLocation(), fakeSessionID);
 +      Iterator<TServerInstance> find = locations.tailMap(simple).keySet().iterator();
 +      if (find.hasNext()) {
 +        TServerInstance current = find.next();
 +        if (current.host().equals(last.host()))
 +          return current;
 +      }
 +    }
 +    
 +    // The strategy here is to walk through the locations and hand them back, one at a time
 +    // Grab an iterator off of the set of options; use a new iterator if it hands back something not in the current list.
 +    if (assignments == null || !assignments.hasNext())
 +      assignments = randomize(locations.keySet()).iterator();
 +    TServerInstance result = assignments.next();
 +    if (!locations.containsKey(result)) {
 +      assignments = null;
 +      return randomize(locations.keySet()).iterator().next();
 +    }
 +    return result;
 +  }
 +  
 +  static class ServerCounts implements Comparable<ServerCounts> {
 +    public final TServerInstance server;
 +    public final int count;
 +    public final TabletServerStatus status;
 +    
 +    ServerCounts(int count, TServerInstance server, TabletServerStatus status) {
 +      this.count = count;
 +      this.server = server;
 +      this.status = status;
 +    }
 +    
 +    public int compareTo(ServerCounts obj) {
 +      int result = count - obj.count;
 +      if (result == 0)
 +        return server.compareTo(obj.server);
 +      return result;
 +    }
 +  }
 +  
 +  public boolean getMigrations(Map<TServerInstance,TabletServerStatus> current, List<TabletMigration> result) {
 +    boolean moreBalancingNeeded = false;
 +    try {
 +      // no moves possible
 +      if (current.size() < 2) {
 +        return false;
 +      }
 +      
 +      // Sort by total number of online tablets, per server
 +      int total = 0;
 +      ArrayList<ServerCounts> totals = new ArrayList<ServerCounts>();
 +      for (Entry<TServerInstance,TabletServerStatus> entry : current.entrySet()) {
 +        int serverTotal = 0;
 +        if (entry.getValue() != null && entry.getValue().tableMap != null) {
 +          for (Entry<String,TableInfo> e : entry.getValue().tableMap.entrySet()) {
 +            /**
 +             * The check below was on entry.getKey(), but that resolves to a tabletserver not a tablename. Believe it should be e.getKey() which is a tablename
 +             */
 +            if (tableToBalance == null || tableToBalance.equals(e.getKey()))
 +              serverTotal += e.getValue().onlineTablets;
 +          }
 +        }
 +        totals.add(new ServerCounts(serverTotal, entry.getKey(), entry.getValue()));
 +        total += serverTotal;
 +      }
 +      
 +      // order from low to high
 +      Collections.sort(totals);
 +      Collections.reverse(totals);
 +      int even = total / totals.size();
 +      int numServersOverEven = total % totals.size();
 +      
 +      // Move tablets from the servers with too many to the servers with
 +      // the fewest but only nominate tablets to move once. This allows us
 +      // to fill new servers with tablets from a mostly balanced server
 +      // very quickly. However, it may take several balancing passes to move
 +      // tablets from one hugely overloaded server to many slightly
 +      // under-loaded servers.
 +      int end = totals.size() - 1;
 +      int movedAlready = 0;
 +      for (int tooManyIndex = 0; tooManyIndex < totals.size(); tooManyIndex++) {
 +        ServerCounts tooMany = totals.get(tooManyIndex);
 +        int goal = even;
 +        if (tooManyIndex < numServersOverEven) {
 +          goal++;
 +        }
 +        int needToUnload = tooMany.count - goal;
 +        ServerCounts tooLittle = totals.get(end);
 +        int needToLoad = goal - tooLittle.count - movedAlready;
 +        if (needToUnload < 1 && needToLoad < 1) {
 +          break;
 +        }
 +        if (needToUnload >= needToLoad) {
 +          result.addAll(move(tooMany, tooLittle, needToLoad));
 +          end--;
 +          movedAlready = 0;
 +        } else {
 +          result.addAll(move(tooMany, tooLittle, needToUnload));
 +          movedAlready += needToUnload;
 +        }
 +        if (needToUnload > needToLoad)
 +          moreBalancingNeeded = true;
 +      }
 +      
 +    } finally {
 +      log.debug("balance ended with " + result.size() + " migrations");
 +    }
 +    return moreBalancingNeeded;
 +  }
 +  
 +  static class TableDiff {
 +    int diff;
 +    String table;
 +    
 +    public TableDiff(int diff, String table) {
 +      this.diff = diff;
 +      this.table = table;
 +    }
 +  };
 +  
 +  /**
 +   * Select a tablet based on differences between table loads; if the loads are even, use the busiest table
 +   */
 +  List<TabletMigration> move(ServerCounts tooMuch, ServerCounts tooLittle, int count) {
 +    
 +    List<TabletMigration> result = new ArrayList<TabletMigration>();
 +    if (count == 0)
 +      return result;
 +    
 +    Map<String,Map<KeyExtent,TabletStats>> onlineTablets = new HashMap<String,Map<KeyExtent,TabletStats>>();
 +    // Copy counts so we can update them as we propose migrations
 +    Map<String,Integer> tooMuchMap = tabletCountsPerTable(tooMuch.status);
 +    Map<String,Integer> tooLittleMap = tabletCountsPerTable(tooLittle.status);
 +    
 +    for (int i = 0; i < count; i++) {
 +      String table;
 +      Integer tooLittleCount;
 +      if (tableToBalance == null) {
 +        // find a table to migrate
 +        // look for an uneven table count
 +        int biggestDifference = 0;
 +        String biggestDifferenceTable = null;
 +        for (Entry<String,Integer> tableEntry : tooMuchMap.entrySet()) {
 +          String tableID = tableEntry.getKey();
 +          if (tooLittleMap.get(tableID) == null)
 +            tooLittleMap.put(tableID, 0);
 +          int diff = tableEntry.getValue() - tooLittleMap.get(tableID);
 +          if (diff > biggestDifference) {
 +            biggestDifference = diff;
 +            biggestDifferenceTable = tableID;
 +          }
 +        }
 +        if (biggestDifference < 2) {
 +          table = busiest(tooMuch.status.tableMap);
 +        } else {
 +          table = biggestDifferenceTable;
 +        }
 +      } else {
 +        // just balance the given table
 +        table = tableToBalance;
 +      }
 +      Map<KeyExtent,TabletStats> onlineTabletsForTable = onlineTablets.get(table);
 +      try {
 +        if (onlineTabletsForTable == null) {
 +          onlineTabletsForTable = new HashMap<KeyExtent,TabletStats>();
 +          for (TabletStats stat : getOnlineTabletsForTable(tooMuch.server, table))
 +            onlineTabletsForTable.put(new KeyExtent(stat.extent), stat);
 +          onlineTablets.put(table, onlineTabletsForTable);
 +        }
 +      } catch (Exception ex) {
 +        log.error("Unable to select a tablet to move", ex);
 +        return result;
 +      }
 +      KeyExtent extent = selectTablet(tooMuch.server, onlineTabletsForTable);
 +      onlineTabletsForTable.remove(extent);
 +      if (extent == null)
 +        return result;
 +      tooMuchMap.put(table, tooMuchMap.get(table) - 1);
 +      /**
 +       * If a table grows from 1 tablet then tooLittleMap.get(table) can return a null, since there is only one tabletserver that holds all of the tablets. Here
 +       * we check to see if in fact that is the case and if so set the value to 0.
 +       */
 +      tooLittleCount = tooLittleMap.get(table);
 +      if (tooLittleCount == null) {
 +        tooLittleCount = 0;
 +      }
 +      tooLittleMap.put(table, tooLittleCount + 1);
 +      
 +      result.add(new TabletMigration(extent, tooMuch.server, tooLittle.server));
 +    }
 +    return result;
 +  }
 +  
 +  static Map<String,Integer> tabletCountsPerTable(TabletServerStatus status) {
 +    Map<String,Integer> result = new HashMap<String,Integer>();
 +    if (status != null && status.tableMap != null) {
 +      Map<String,TableInfo> tableMap = status.tableMap;
 +      for (Entry<String,TableInfo> entry : tableMap.entrySet()) {
 +        result.put(entry.getKey(), entry.getValue().onlineTablets);
 +      }
 +    }
 +    return result;
 +  }
 +  
 +  static KeyExtent selectTablet(TServerInstance tserver, Map<KeyExtent,TabletStats> extents) {
 +    if (extents.size() == 0)
 +      return null;
 +    KeyExtent mostRecentlySplit = null;
 +    long splitTime = 0;
 +    for (Entry<KeyExtent,TabletStats> entry : extents.entrySet())
 +      if (entry.getValue().splitCreationTime >= splitTime) {
 +        splitTime = entry.getValue().splitCreationTime;
 +        mostRecentlySplit = entry.getKey();
 +      }
 +    return mostRecentlySplit;
 +  }
 +  
 +  // define what it means for a tablet to be busy
 +  private static String busiest(Map<String,TableInfo> tables) {
 +    String result = null;
 +    double busiest = Double.NEGATIVE_INFINITY;
 +    for (Entry<String,TableInfo> entry : tables.entrySet()) {
 +      TableInfo info = entry.getValue();
 +      double busy = info.ingestRate + info.queryRate;
 +      if (busy > busiest) {
 +        busiest = busy;
 +        result = entry.getKey();
 +      }
 +    }
 +    return result;
 +  }
 +  
 +  @Override
 +  public void getAssignments(SortedMap<TServerInstance,TabletServerStatus> current, Map<KeyExtent,TServerInstance> unassigned,
 +      Map<KeyExtent,TServerInstance> assignments) {
 +    for (Entry<KeyExtent,TServerInstance> entry : unassigned.entrySet()) {
 +      assignments.put(entry.getKey(), getAssignment(current, entry.getKey(), entry.getValue()));
 +    }
 +  }
-   
++
++  private static final NoTservers NO_SERVERS = new NoTservers(log);
++
++  protected final OutstandingMigrations outstandingMigrations = new OutstandingMigrations(log);
++
 +  @Override
 +  public long balance(SortedMap<TServerInstance,TabletServerStatus> current, Set<KeyExtent> migrations, List<TabletMigration> migrationsOut) {
 +    // do we have any servers?
 +    if (current.size() > 0) {
 +      // Don't migrate if we have migrations in progress
 +      if (migrations.size() == 0) {
++        resetBalancerErrors();
 +        if (getMigrations(current, migrationsOut))
 +          return 1 * 1000;
++      } else {
++        outstandingMigrations.migrations = migrations;
++        constraintNotMet(outstandingMigrations);
 +      }
++    } else {
++      constraintNotMet(NO_SERVERS);
 +    }
 +    return 5 * 1000;
 +  }
 +  
 +}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/dedc9cdd/server/base/src/main/java/org/apache/accumulo/server/master/balancer/TabletBalancer.java
----------------------------------------------------------------------
diff --cc server/base/src/main/java/org/apache/accumulo/server/master/balancer/TabletBalancer.java
index 5bd1632,0000000..fb97628
mode 100644,000000..100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/TabletBalancer.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/balancer/TabletBalancer.java
@@@ -1,149 -1,0 +1,240 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one or more
 + * contributor license agreements.  See the NOTICE file distributed with
 + * this work for additional information regarding copyright ownership.
 + * The ASF licenses this file to You under the Apache License, Version 2.0
 + * (the "License"); you may not use this file except in compliance with
 + * the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.accumulo.server.master.balancer;
 +
 +import java.util.ArrayList;
++import java.util.Collections;
 +import java.util.List;
 +import java.util.Map;
 +import java.util.Set;
 +import java.util.SortedMap;
 +
++import com.google.common.collect.Iterables;
++
 +import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
 +import org.apache.accumulo.core.data.KeyExtent;
 +import org.apache.accumulo.core.master.thrift.TabletServerStatus;
 +import org.apache.accumulo.core.tabletserver.thrift.TabletClientService;
 +import org.apache.accumulo.core.tabletserver.thrift.TabletClientService.Client;
 +import org.apache.accumulo.core.tabletserver.thrift.TabletStats;
 +import org.apache.accumulo.core.util.ThriftUtil;
 +import org.apache.accumulo.server.conf.ServerConfiguration;
 +import org.apache.accumulo.server.master.state.TServerInstance;
 +import org.apache.accumulo.server.master.state.TabletMigration;
 +import org.apache.accumulo.server.security.SystemCredentials;
 +import org.apache.accumulo.trace.instrument.Tracer;
 +import org.apache.log4j.Logger;
 +import org.apache.thrift.TException;
 +import org.apache.thrift.transport.TTransportException;
 +
 +public abstract class TabletBalancer {
 +  
 +  private static final Logger log = Logger.getLogger(TabletBalancer.class);
 +  
 +  protected ServerConfiguration configuration;
 +  
 +  /**
 +   * Initialize the TabletBalancer. This gives the balancer the opportunity to read the configuration.
 +   */
 +  public void init(ServerConfiguration conf) {
 +    configuration = conf;
 +  }
 +  
 +  /**
 +   * Assign tablets to tablet servers. This method is called whenever the master finds tablets that are unassigned.
 +   * 
 +   * @param current
 +   *          The current table-summary state of all the online tablet servers. Read-only. The TabletServerStatus for each server may be null if the tablet
 +   *          server has not yet responded to a recent request for status.
 +   * @param unassigned
 +   *          A map from unassigned tablet to the last known tablet server. Read-only.
 +   * @param assignments
 +   *          A map from tablet to assigned server. Write-only.
 +   */
 +  abstract public void getAssignments(SortedMap<TServerInstance,TabletServerStatus> current, Map<KeyExtent,TServerInstance> unassigned,
 +      Map<KeyExtent,TServerInstance> assignments);
 +  
 +  /**
 +   * Ask the balancer if any migrations are necessary.
++   *
++   * If the balancer is going to self-abort due to some environmental constraint (e.g. it requires some minimum number of tservers, or a maximum number
++   * of outstanding migrations), it should issue a log message to alert operators. The message should be at WARN normally and at ERROR if the balancer knows that the
++   * problem can not self correct. It should not issue these messages more than once a minute. Subclasses can use the convenience methods of {@link #constraintNotMet()} and
++   * {@link #balanceSuccessful()} to accomplish this logging.
 +   * 
 +   * @param current
 +   *          The current table-summary state of all the online tablet servers. Read-only.
 +   * @param migrations
 +   *          the current set of migrations. Read-only.
 +   * @param migrationsOut
 +   *          new migrations to perform; should not contain tablets in the current set of migrations. Write-only.
 +   * @return the time, in milliseconds, to wait before re-balancing.
 +   * 
 +   *         This method will not be called when there are unassigned tablets.
 +   */
 +  public abstract long balance(SortedMap<TServerInstance,TabletServerStatus> current, Set<KeyExtent> migrations, List<TabletMigration> migrationsOut);
++
++  private static final long ONE_SECOND = 1000l;
++  private boolean stuck = false;
++  private long stuckNotificationTime = -1l;
++
++  protected static final long TIME_BETWEEN_BALANCER_WARNINGS = 60 * ONE_SECOND;
++
++  /**
++   * A deferred call descendent TabletBalancers use to log why they can't continue.
++   * The call is deferred so that TabletBalancer can limit how often messages happen.
++   *
++   * Implementations should be reused as much as possible.
++   *
++   * Be sure to pass in a properly scoped Logger instance so that messages indicate
++   * what part of the system is having trouble.
++   */
++  protected static abstract class BalancerProblem implements Runnable {
++    protected final Logger balancerLog;
++    public BalancerProblem(Logger logger) {
++      balancerLog = logger;
++    }
++  }
++
++  /**
++   * If a TabletBalancer requires active tservers, it should use this problem to indicate when there are none.
++   * NoTservers is safe to share with anyone who uses the same Logger. TabletBalancers should have a single
++   * static instance.
++   */
++  protected static class NoTservers extends BalancerProblem {
++    public NoTservers(Logger logger) {
++      super(logger);
++    }
++
++    @Override
++    public void run() {
++      balancerLog.warn("Not balancing because we don't have any tservers");
++    }
++  }
++
++  /**
++   * If a TabletBalancer only balances when there are no outstanding migrations, it should use this problem
++   * to indicate when they exist.
++   *
++   * Iff a TabletBalancer makes use of the migrations member to provide samples, then OutstandingMigrations
++   * is not thread safe.
++   */
++  protected static class OutstandingMigrations extends BalancerProblem {
++    public Set<KeyExtent> migrations = Collections.<KeyExtent>emptySet();
++
++    public OutstandingMigrations(Logger logger) {
++      super(logger);
++    }
++
++    @Override
++    public void run() {
++      balancerLog.warn("Not balancing due to " + migrations.size() + " outstanding migrations.");
++      /* TODO ACCUMULO-2938 redact key extents in this output to avoid leaking protected information. */
++      balancerLog.debug("Sample up to 10 outstanding migrations: " + Iterables.limit(migrations, 10));
++    }
++  }
++
++  /**
++   * Warn that a Balancer can't work because of some external restriction.
++   * Will not call the provided logging handler  more often than TIME_BETWEEN_BALANCER_WARNINGS
++   */
++  protected void constraintNotMet(BalancerProblem cause) {
++    if (!stuck) {
++      stuck = true;
++      stuckNotificationTime = System.currentTimeMillis();
++    } else {
++      if ((System.currentTimeMillis() - stuckNotificationTime) > TIME_BETWEEN_BALANCER_WARNINGS) {
++        cause.run();
++        stuckNotificationTime = System.currentTimeMillis();
++      }
++    }
++  }
++
++  /**
++   * Resets logging about problems meeting an external constraint on balancing.
++   */
++  protected void resetBalancerErrors() {
++    stuck = false;
++  }
 +  
 +  /**
 +   * Fetch the tablets for the given table by asking the tablet server. Useful if your balance strategy needs details at the tablet level to decide what tablets
 +   * to move.
 +   * 
 +   * @param tserver
 +   *          The tablet server to ask.
 +   * @param tableId
 +   *          The table id
 +   * @return a list of tablet statistics
 +   * @throws ThriftSecurityException
 +   *           tablet server disapproves of your internal System password.
 +   * @throws TException
 +   *           any other problem
 +   */
 +  public List<TabletStats> getOnlineTabletsForTable(TServerInstance tserver, String tableId) throws ThriftSecurityException, TException {
 +    log.debug("Scanning tablet server " + tserver + " for table " + tableId);
 +    Client client = ThriftUtil.getClient(new TabletClientService.Client.Factory(), tserver.getLocation(), configuration.getConfiguration());
 +    try {
 +      List<TabletStats> onlineTabletsForTable = client.getTabletStats(Tracer.traceInfo(), SystemCredentials.get().toThrift(configuration.getInstance()),
 +          tableId);
 +      return onlineTabletsForTable;
 +    } catch (TTransportException e) {
 +      log.error("Unable to connect to " + tserver + ": " + e);
 +    } finally {
 +      ThriftUtil.returnClient(client);
 +    }
 +    return null;
 +  }
 +  
 +  /**
 +   * Utility to ensure that the migrations from balance() are consistent:
 +   * <ul>
 +   * <li>Tablet objects are not null
 +   * <li>Source and destination tablet servers are not null and current
 +   * </ul>
 +   * 
 +   * @return A list of TabletMigration object that passed sanity checks.
 +   */
 +  public static List<TabletMigration> checkMigrationSanity(Set<TServerInstance> current, List<TabletMigration> migrations) {
 +    List<TabletMigration> result = new ArrayList<TabletMigration>(migrations.size());
 +    for (TabletMigration m : migrations) {
 +      if (m.tablet == null) {
 +        log.warn("Balancer gave back a null tablet " + m);
 +        continue;
 +      }
 +      if (m.newServer == null) {
 +        log.warn("Balancer did not set the destination " + m);
 +        continue;
 +      }
 +      if (m.oldServer == null) {
 +        log.warn("Balancer did not set the source " + m);
 +        continue;
 +      }
 +      if (!current.contains(m.oldServer)) {
 +        log.warn("Balancer wants to move a tablet from a server that is not current: " + m);
 +        continue;
 +      }
 +      if (!current.contains(m.newServer)) {
 +        log.warn("Balancer wants to move a tablet to a server that is not current: " + m);
 +        continue;
 +      }
 +      result.add(m);
 +    }
 +    return result;
 +  }
 +  
 +}