You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by dl...@apache.org on 2017/01/25 21:44:32 UTC

[1/3] accumulo git commit: ACCUMULO-4576: Always use the most current tablet server information to make assignment / balancing decisions

Repository: accumulo
Updated Branches:
  refs/heads/master 6302a854e -> 1efc828f3


ACCUMULO-4576: Always use the most current tablet server information to make assignment / balancing decisions


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

Branch: refs/heads/master
Commit: ff134d2c63b47a4d496bb6dfe51b96cbbcadd55f
Parents: df400c5
Author: Dave Marion <dl...@apache.org>
Authored: Wed Jan 25 15:41:01 2017 -0500
Committer: Dave Marion <dl...@apache.org>
Committed: Wed Jan 25 15:41:01 2017 -0500

----------------------------------------------------------------------
 .../balancer/HostRegexTableLoadBalancer.java    | 86 ++++++++++----------
 .../BaseHostRegexTableLoadBalancerTest.java     | 14 +++-
 ...gexTableLoadBalancerReconfigurationTest.java |  5 +-
 .../HostRegexTableLoadBalancerTest.java         |  7 +-
 4 files changed, 60 insertions(+), 52 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/ff134d2c/server/base/src/main/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancer.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancer.java b/server/base/src/main/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancer.java
index 6b26dc4..051f306 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancer.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancer.java
@@ -57,9 +57,9 @@ import org.slf4j.LoggerFactory;
  * If this occurs then the offending tablets will be reassigned. This would cover the case where the configuration is changed and the master is restarted while
  * the tablet servers are up. To change the out of bounds check time period, set the following property:<br>
  * <b>table.custom.balancer.host.regex.oob.period=5m</b><br>
- * Periodically (default 1m) this balancer will regroup the set of current tablet servers into pools based on regexes applied to the tserver host names. This
- * would cover the case of tservers dying or coming online. To change the host pool check time period, set the following property: <br>
- * <b>table.custom.balancer.host.regex.pool.check=5m</b><br>
+ * (Deprecated) Periodically (default 1m) this balancer will regroup the set of current tablet servers into pools based on regexes applied to the tserver host
+ * names. This would cover the case of tservers dying or coming online. To change the host pool check time period, set the following property: <br>
+ * <b>table.custom.balancer.host.regex.pool.check=5m (Deprecated)</b><br>
  * Regex matching can be based on either the host name (default) or host ip address. To set this balancer to match the regular expressions to the tablet server
  * IP address, then set the following property:<br>
  * <b>table.custom.balancer.host.regex.is.ip=true</b><br>
@@ -74,7 +74,9 @@ public class HostRegexTableLoadBalancer extends TableLoadBalancer implements Con
   public static final String HOST_BALANCER_PREFIX = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.";
   public static final String HOST_BALANCER_OOB_CHECK_KEY = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.oob.period";
   private static final String HOST_BALANCER_OOB_DEFAULT = "5m";
+  @Deprecated
   public static final String HOST_BALANCER_POOL_RECHECK_KEY = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.pool.check";
+  @Deprecated
   private static final String HOST_BALANCER_POOL_RECHECK_DEFAULT = "1m";
   public static final String HOST_BALANCER_REGEX_USING_IPS_KEY = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.is.ip";
   public static final String HOST_BALANCER_REGEX_MAX_MIGRATIONS_KEY = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey()
@@ -83,12 +85,10 @@ public class HostRegexTableLoadBalancer extends TableLoadBalancer implements Con
   protected static final String DEFAULT_POOL = "HostTableLoadBalancer.ALL";
 
   protected long oobCheckMillis = AccumuloConfiguration.getTimeInMillis(HOST_BALANCER_OOB_DEFAULT);
-  protected long poolRecheckMillis = AccumuloConfiguration.getTimeInMillis(HOST_BALANCER_POOL_RECHECK_DEFAULT);
 
   private Map<String,String> tableIdToTableName = null;
   private Map<String,Pattern> poolNameToRegexPattern = null;
   private volatile long lastOOBCheck = System.currentTimeMillis();
-  private volatile long lastPoolRecheck = 0;
   private boolean isIpBasedRegex = false;
   private Map<String,SortedMap<TServerInstance,TabletServerStatus>> pools = new HashMap<>();
   private int maxTServerMigrations = HOST_BALANCER_REGEX_MAX_MIGRATIONS_DEFAULT;
@@ -102,22 +102,25 @@ public class HostRegexTableLoadBalancer extends TableLoadBalancer implements Con
    * @return current servers grouped by pool name, if not a match it is put into a default pool.
    */
   protected synchronized Map<String,SortedMap<TServerInstance,TabletServerStatus>> splitCurrentByRegex(SortedMap<TServerInstance,TabletServerStatus> current) {
-    if ((System.currentTimeMillis() - lastPoolRecheck) > poolRecheckMillis) {
-      LOG.debug("Performing pool recheck - regrouping tablet servers based on regular expressions");
-      Map<String,SortedMap<TServerInstance,TabletServerStatus>> newPools = new HashMap<>();
-      for (Entry<TServerInstance,TabletServerStatus> e : current.entrySet()) {
-        List<String> poolNames = getPoolNamesForHost(e.getKey().host());
-        for (String pool : poolNames) {
-          SortedMap<TServerInstance,TabletServerStatus> np = newPools.get(pool);
-          if (null == np) {
-            np = new TreeMap<>(current.comparator());
-            newPools.put(pool, np);
-          }
-          np.put(e.getKey(), e.getValue());
+    LOG.debug("Performing pool recheck - regrouping tablet servers based on regular expressions");
+    Map<String,SortedMap<TServerInstance,TabletServerStatus>> newPools = new HashMap<>();
+    for (Entry<TServerInstance,TabletServerStatus> e : current.entrySet()) {
+      List<String> poolNames = getPoolNamesForHost(e.getKey().host());
+      for (String pool : poolNames) {
+        SortedMap<TServerInstance,TabletServerStatus> np = newPools.get(pool);
+        if (null == np) {
+          np = new TreeMap<>(current.comparator());
+          newPools.put(pool, np);
         }
+        np.put(e.getKey(), e.getValue());
+      }
+    }
+    pools = newPools;
+    LOG.trace("Pool to TabletServer mapping:");
+    if (LOG.isTraceEnabled()) {
+      for (Entry<String,SortedMap<TServerInstance,TabletServerStatus>> e : pools.entrySet()) {
+        LOG.trace("\tpool: {} -> tservers: {}", e.getKey(), e.getValue().keySet());
       }
-      pools = newPools;
-      this.lastPoolRecheck = System.currentTimeMillis();
     }
     return pools;
   }
@@ -189,8 +192,8 @@ public class HostRegexTableLoadBalancer extends TableLoadBalancer implements Con
       if (null != customProps && customProps.size() > 0) {
         for (Entry<String,String> customProp : customProps.entrySet()) {
           if (customProp.getKey().startsWith(HOST_BALANCER_PREFIX)) {
-            if (customProp.getKey().equals(HOST_BALANCER_OOB_CHECK_KEY) || customProp.getKey().equals(HOST_BALANCER_POOL_RECHECK_KEY)
-                || customProp.getKey().equals(HOST_BALANCER_REGEX_USING_IPS_KEY) || customProp.getKey().equals(HOST_BALANCER_REGEX_MAX_MIGRATIONS_KEY)) {
+            if (customProp.getKey().equals(HOST_BALANCER_OOB_CHECK_KEY) || customProp.getKey().equals(HOST_BALANCER_REGEX_USING_IPS_KEY)
+                || customProp.getKey().equals(HOST_BALANCER_REGEX_MAX_MIGRATIONS_KEY)) {
               continue;
             }
             String tableName = customProp.getKey().substring(HOST_BALANCER_PREFIX.length());
@@ -204,10 +207,6 @@ public class HostRegexTableLoadBalancer extends TableLoadBalancer implements Con
     if (null != oobProperty) {
       oobCheckMillis = AccumuloConfiguration.getTimeInMillis(oobProperty);
     }
-    String poolRecheckProperty = conf.getConfiguration().get(HOST_BALANCER_POOL_RECHECK_KEY);
-    if (null != poolRecheckProperty) {
-      poolRecheckMillis = AccumuloConfiguration.getTimeInMillis(poolRecheckProperty);
-    }
     String ipBased = conf.getConfiguration().get(HOST_BALANCER_REGEX_USING_IPS_KEY);
     if (null != ipBased) {
       isIpBasedRegex = Boolean.parseBoolean(ipBased);
@@ -222,11 +221,10 @@ public class HostRegexTableLoadBalancer extends TableLoadBalancer implements Con
   @Override
   public String toString() {
     ToStringBuilder buf = new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE);
-    buf.append("Pool Recheck Interval", this.poolRecheckMillis);
-    buf.append("Tablet Out Of Bounds Check Interval", this.oobCheckMillis);
-    buf.append("Max Tablet Server Migrations", this.maxTServerMigrations);
-    buf.append("Regular Expressions use IPs", this.isIpBasedRegex);
-    buf.append("Pools", this.poolNameToRegexPattern);
+    buf.append("\nTablet Out Of Bounds Check Interval", this.oobCheckMillis);
+    buf.append("\nMax Tablet Server Migrations", this.maxTServerMigrations);
+    buf.append("\nRegular Expressions use IPs", this.isIpBasedRegex);
+    buf.append("\nPools", this.poolNameToRegexPattern);
     return buf.toString();
   }
 
@@ -242,8 +240,9 @@ public class HostRegexTableLoadBalancer extends TableLoadBalancer implements Con
     return oobCheckMillis;
   }
 
+  @Deprecated
   public long getPoolRecheckMillis() {
-    return poolRecheckMillis;
+    return 0;
   }
 
   public boolean isIpBasedRegex() {
@@ -297,7 +296,7 @@ public class HostRegexTableLoadBalancer extends TableLoadBalancer implements Con
 
   @Override
   public long balance(SortedMap<TServerInstance,TabletServerStatus> current, Set<KeyExtent> migrations, List<TabletMigration> migrationsOut) {
-    long minBalanceTime = 5 * 1000;
+    long minBalanceTime = 20 * 1000;
     // Iterate over the tables and balance each of them
     TableOperations t = getTableOperations();
     if (t == null)
@@ -307,11 +306,14 @@ public class HostRegexTableLoadBalancer extends TableLoadBalancer implements Con
     if ((System.currentTimeMillis() - this.lastOOBCheck) > this.oobCheckMillis) {
       try {
         // Check to see if a tablet is assigned outside the bounds of the pool. If so, migrate it.
-        for (Entry<TServerInstance,TabletServerStatus> e : current.entrySet()) {
-          for (String table : poolNameToRegexPattern.keySet()) {
+        for (String table : t.list()) {
+          LOG.debug("Checking for out of bounds tablets for table {}", table);
+          String tablePoolName = getPoolNameForTable(table);
+          for (Entry<TServerInstance,TabletServerStatus> e : current.entrySet()) {
             // pool names are the same as table names, except in the DEFAULT case.
             // If this table is assigned to a pool for this host, then move on.
-            if (getPoolNamesForHost(e.getKey().host()).contains(table)) {
+            List<String> hostPools = getPoolNamesForHost(e.getKey().host());
+            if (hostPools.contains(tablePoolName)) {
               continue;
             }
             String tid = t.tableIdMap().get(table);
@@ -328,7 +330,7 @@ public class HostRegexTableLoadBalancer extends TableLoadBalancer implements Con
               for (TabletStats ts : outOfBoundsTablets) {
                 KeyExtent ke = new KeyExtent(ts.getExtent());
                 if (migrations.contains(ke)) {
-                  LOG.debug("Migration for  out of bounds tablet {} has already been requested", ke);
+                  LOG.debug("Migration for out of bounds tablet {} has already been requested", ke);
                   continue;
                 }
                 String poolName = getPoolNameForTable(table);
@@ -361,6 +363,7 @@ public class HostRegexTableLoadBalancer extends TableLoadBalancer implements Con
 
     if (migrationsOut.size() > 0) {
       LOG.warn("Not balancing tables due to moving {} out of bounds tablets", migrationsOut.size());
+      LOG.info("Migrating out of bounds tablets: {}", migrationsOut);
       return minBalanceTime;
     }
 
@@ -374,21 +377,18 @@ public class HostRegexTableLoadBalancer extends TableLoadBalancer implements Con
       String regexTableName = getPoolNameForTable(tableName);
       SortedMap<TServerInstance,TabletServerStatus> currentView = currentGrouped.get(regexTableName);
       if (null == currentView) {
-        LOG.warn("Skipping balance for table {} as no tablet servers are online, will recheck for online tservers at {} ms intervals", tableName,
-            this.poolRecheckMillis);
+        LOG.warn("Skipping balance for table {} as no tablet servers are online.", tableName);
         continue;
       }
       ArrayList<TabletMigration> newMigrations = new ArrayList<>();
-      long tableBalanceTime = getBalancerForTable(s).balance(currentView, migrations, newMigrations);
-      if (tableBalanceTime < minBalanceTime) {
-        minBalanceTime = tableBalanceTime;
-      }
+      getBalancerForTable(s).balance(currentView, migrations, newMigrations);
+
       migrationsOut.addAll(newMigrations);
       if (migrationsOut.size() > this.maxTServerMigrations) {
         break;
       }
     }
-
+    LOG.info("Migrating tablets for balance: {}", migrationsOut);
     return minBalanceTime;
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ff134d2c/server/base/src/test/java/org/apache/accumulo/server/master/balancer/BaseHostRegexTableLoadBalancerTest.java
----------------------------------------------------------------------
diff --git a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/BaseHostRegexTableLoadBalancerTest.java b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/BaseHostRegexTableLoadBalancerTest.java
index 144d1fc..d41099e 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/BaseHostRegexTableLoadBalancerTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/BaseHostRegexTableLoadBalancerTest.java
@@ -24,7 +24,9 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.SortedMap;
+import java.util.SortedSet;
 import java.util.TreeMap;
+import java.util.TreeSet;
 
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -136,8 +138,7 @@ public abstract class BaseHostRegexTableLoadBalancerTest extends HostRegexTableL
 
   protected static final HashMap<String,String> DEFAULT_TABLE_PROPERTIES = new HashMap<>();
   {
-    DEFAULT_TABLE_PROPERTIES.put(HostRegexTableLoadBalancer.HOST_BALANCER_OOB_CHECK_KEY, "10s");
-    DEFAULT_TABLE_PROPERTIES.put(HostRegexTableLoadBalancer.HOST_BALANCER_POOL_RECHECK_KEY, "30s");
+    DEFAULT_TABLE_PROPERTIES.put(HostRegexTableLoadBalancer.HOST_BALANCER_OOB_CHECK_KEY, "2s");
     DEFAULT_TABLE_PROPERTIES.put(HostRegexTableLoadBalancer.HOST_BALANCER_PREFIX + FOO.getTableName(), "r01.*");
     DEFAULT_TABLE_PROPERTIES.put(HostRegexTableLoadBalancer.HOST_BALANCER_PREFIX + BAR.getTableName(), "r02.*");
   }
@@ -265,6 +266,15 @@ public abstract class BaseHostRegexTableLoadBalancerTest extends HostRegexTableL
         tables.put(BAZ.getTableName(), BAZ.getId());
         return tables;
       }
+
+      @Override
+      public SortedSet<String> list() {
+        TreeSet<String> tables = new TreeSet<>();
+        tables.add(BAR.getTableName());
+        tables.add(BAZ.getTableName());
+        tables.add(FOO.getTableName());
+        return tables;
+      }
     };
   }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ff134d2c/server/base/src/test/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancerReconfigurationTest.java
----------------------------------------------------------------------
diff --git a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancerReconfigurationTest.java b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancerReconfigurationTest.java
index 05755b7..14f6c3e 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancerReconfigurationTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancerReconfigurationTest.java
@@ -41,7 +41,6 @@ public class HostRegexTableLoadBalancerReconfigurationTest extends BaseHostRegex
 
   @Test
   public void testConfigurationChanges() {
-    DEFAULT_TABLE_PROPERTIES.put(HostRegexTableLoadBalancer.HOST_BALANCER_POOL_RECHECK_KEY, "10s");
 
     init(factory);
     Map<KeyExtent,TServerInstance> unassigned = new HashMap<>();
@@ -72,14 +71,14 @@ public class HostRegexTableLoadBalancerReconfigurationTest extends BaseHostRegex
     Set<KeyExtent> migrations = new HashSet<>();
     List<TabletMigration> migrationsOut = new ArrayList<>();
     // Wait to trigger the out of bounds check which will call our version of getOnlineTabletsForTable
-    UtilWaitThread.sleep(11000);
+    UtilWaitThread.sleep(3000);
     this.balance(Collections.unmodifiableSortedMap(allTabletServers), migrations, migrationsOut);
     Assert.assertEquals(0, migrationsOut.size());
     // Change property, simulate call by TableConfWatcher
     DEFAULT_TABLE_PROPERTIES.put(HostRegexTableLoadBalancer.HOST_BALANCER_PREFIX + BAR.getTableName(), "r01.*");
     this.propertiesChanged();
     // Wait to trigger the out of bounds check and the repool check
-    UtilWaitThread.sleep(11000);
+    UtilWaitThread.sleep(3000);
     this.balance(Collections.unmodifiableSortedMap(allTabletServers), migrations, migrationsOut);
     Assert.assertEquals(5, migrationsOut.size());
     for (TabletMigration migration : migrationsOut) {

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ff134d2c/server/base/src/test/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancerTest.java
----------------------------------------------------------------------
diff --git a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancerTest.java b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancerTest.java
index ed17508..6ac4096 100644
--- a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancerTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancerTest.java
@@ -50,8 +50,8 @@ public class HostRegexTableLoadBalancerTest extends BaseHostRegexTableLoadBalanc
   @Test
   public void testInit() {
     init(factory);
-    Assert.assertEquals("OOB check interval value is incorrect", 10000, this.getOobCheckMillis());
-    Assert.assertEquals("Pool check interval value is incorrect", 30000, this.getPoolRecheckMillis());
+    Assert.assertEquals("OOB check interval value is incorrect", 2000, this.getOobCheckMillis());
+    Assert.assertEquals("Pool check interval value is incorrect", 0, this.getPoolRecheckMillis());
     Assert.assertFalse(isIpBasedRegex());
     Map<String,Pattern> patterns = this.getPoolNameToRegexPattern();
     Assert.assertEquals(2, patterns.size());
@@ -75,7 +75,7 @@ public class HostRegexTableLoadBalancerTest extends BaseHostRegexTableLoadBalanc
     List<TabletMigration> migrations = new ArrayList<>();
     init(factory);
     long wait = this.balance(Collections.unmodifiableSortedMap(createCurrent(2)), Collections.singleton(new KeyExtent()), migrations);
-    Assert.assertEquals(5000, wait);
+    Assert.assertEquals(20000, wait);
     Assert.assertEquals(0, migrations.size());
   }
 
@@ -180,7 +180,6 @@ public class HostRegexTableLoadBalancerTest extends BaseHostRegexTableLoadBalanc
       public synchronized AccumuloConfiguration getConfiguration() {
         HashMap<String,String> props = new HashMap<>();
         props.put(HostRegexTableLoadBalancer.HOST_BALANCER_OOB_CHECK_KEY, "30s");
-        props.put(HostRegexTableLoadBalancer.HOST_BALANCER_POOL_RECHECK_KEY, "30s");
         props.put(HostRegexTableLoadBalancer.HOST_BALANCER_REGEX_USING_IPS_KEY, "true");
         return new ConfigurationCopy(props);
       }


[2/3] accumulo git commit: Merge branch '1.7' into 1.8

Posted by dl...@apache.org.
Merge branch '1.7' into 1.8


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

Branch: refs/heads/master
Commit: 8668c7f381832128d6bf76e8b6097778a42b71c4
Parents: b903766 ff134d2
Author: Dave Marion <dl...@apache.org>
Authored: Wed Jan 25 16:11:20 2017 -0500
Committer: Dave Marion <dl...@apache.org>
Committed: Wed Jan 25 16:11:20 2017 -0500

----------------------------------------------------------------------
 .../balancer/HostRegexTableLoadBalancer.java    | 86 ++++++++++----------
 .../BaseHostRegexTableLoadBalancerTest.java     | 14 +++-
 ...gexTableLoadBalancerReconfigurationTest.java |  5 +-
 .../HostRegexTableLoadBalancerTest.java         |  7 +-
 4 files changed, 60 insertions(+), 52 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/8668c7f3/server/base/src/main/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancer.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/accumulo/blob/8668c7f3/server/base/src/test/java/org/apache/accumulo/server/master/balancer/BaseHostRegexTableLoadBalancerTest.java
----------------------------------------------------------------------


[3/3] accumulo git commit: Merge branch '1.8'

Posted by dl...@apache.org.
Merge branch '1.8'


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

Branch: refs/heads/master
Commit: 1efc828f32c4a4546b5130cd5bf8e2d4455e2243
Parents: 6302a85 8668c7f
Author: Dave Marion <dl...@apache.org>
Authored: Wed Jan 25 16:42:26 2017 -0500
Committer: Dave Marion <dl...@apache.org>
Committed: Wed Jan 25 16:42:26 2017 -0500

----------------------------------------------------------------------
 .../balancer/HostRegexTableLoadBalancer.java    | 84 +++++++++-----------
 .../BaseHostRegexTableLoadBalancerTest.java     | 14 +++-
 ...gexTableLoadBalancerReconfigurationTest.java |  5 +-
 .../HostRegexTableLoadBalancerTest.java         |  6 +-
 4 files changed, 52 insertions(+), 57 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/1efc828f/server/base/src/main/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancer.java
----------------------------------------------------------------------
diff --cc server/base/src/main/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancer.java
index a7e2a93,33494a7..af7a5da
--- a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancer.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancer.java
@@@ -57,9 -57,9 +57,6 @@@ import org.slf4j.LoggerFactory
   * If this occurs then the offending tablets will be reassigned. This would cover the case where the configuration is changed and the master is restarted while
   * the tablet servers are up. To change the out of bounds check time period, set the following property:<br>
   * <b>table.custom.balancer.host.regex.oob.period=5m</b><br>
-  * Periodically (default 1m) this balancer will regroup the set of current tablet servers into pools based on regexes applied to the tserver host names. This
-  * would cover the case of tservers dying or coming online. To change the host pool check time period, set the following property: <br>
-  * <b>table.custom.balancer.host.regex.pool.check=5m</b><br>
 - * (Deprecated) Periodically (default 1m) this balancer will regroup the set of current tablet servers into pools based on regexes applied to the tserver host
 - * names. This would cover the case of tservers dying or coming online. To change the host pool check time period, set the following property: <br>
 - * <b>table.custom.balancer.host.regex.pool.check=5m (Deprecated)</b><br>
   * Regex matching can be based on either the host name (default) or host ip address. To set this balancer to match the regular expressions to the tablet server
   * IP address, then set the following property:<br>
   * <b>table.custom.balancer.host.regex.is.ip=true</b><br>
@@@ -74,8 -74,10 +71,6 @@@ public class HostRegexTableLoadBalance
    public static final String HOST_BALANCER_PREFIX = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.";
    public static final String HOST_BALANCER_OOB_CHECK_KEY = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.oob.period";
    private static final String HOST_BALANCER_OOB_DEFAULT = "5m";
 -  @Deprecated
--  public static final String HOST_BALANCER_POOL_RECHECK_KEY = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.pool.check";
 -  @Deprecated
--  private static final String HOST_BALANCER_POOL_RECHECK_DEFAULT = "1m";
    public static final String HOST_BALANCER_REGEX_USING_IPS_KEY = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.is.ip";
    public static final String HOST_BALANCER_REGEX_MAX_MIGRATIONS_KEY = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey()
        + "balancer.host.regex.concurrent.migrations";
@@@ -242,10 -240,11 +233,6 @@@
      return oobCheckMillis;
    }
  
 -  @Deprecated
--  public long getPoolRecheckMillis() {
-     return poolRecheckMillis;
 -    return 0;
--  }
--
    public boolean isIpBasedRegex() {
      return isIpBasedRegex;
    }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/1efc828f/server/base/src/test/java/org/apache/accumulo/server/master/balancer/BaseHostRegexTableLoadBalancerTest.java
----------------------------------------------------------------------
diff --cc server/base/src/test/java/org/apache/accumulo/server/master/balancer/BaseHostRegexTableLoadBalancerTest.java
index 9360c82,a1730cb..2c0d20e
--- a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/BaseHostRegexTableLoadBalancerTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/BaseHostRegexTableLoadBalancerTest.java
@@@ -24,8 -24,9 +24,10 @@@ import java.util.List
  import java.util.Map;
  import java.util.Map.Entry;
  import java.util.SortedMap;
+ import java.util.SortedSet;
  import java.util.TreeMap;
+ import java.util.TreeSet;
 +import java.util.function.Predicate;
  
  import org.apache.accumulo.core.client.AccumuloException;
  import org.apache.accumulo.core.client.AccumuloSecurityException;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/1efc828f/server/base/src/test/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancerTest.java
----------------------------------------------------------------------
diff --cc server/base/src/test/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancerTest.java
index 79561c4,6ac4096..fb5fc1a
--- a/server/base/src/test/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancerTest.java
+++ b/server/base/src/test/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancerTest.java
@@@ -49,8 -50,8 +49,7 @@@ public class HostRegexTableLoadBalancer
    @Test
    public void testInit() {
      init(factory);
-     Assert.assertEquals("OOB check interval value is incorrect", 10000, this.getOobCheckMillis());
-     Assert.assertEquals("Pool check interval value is incorrect", 30000, this.getPoolRecheckMillis());
+     Assert.assertEquals("OOB check interval value is incorrect", 2000, this.getOobCheckMillis());
 -    Assert.assertEquals("Pool check interval value is incorrect", 0, this.getPoolRecheckMillis());
      Assert.assertFalse(isIpBasedRegex());
      Map<String,Pattern> patterns = this.getPoolNameToRegexPattern();
      Assert.assertEquals(2, patterns.size());