You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@accumulo.apache.org by dlmarion <gi...@git.apache.org> on 2016/03/29 16:40:44 UTC

[GitHub] accumulo pull request: ACCUMULO-4173: Host Regex Table Load Balanc...

GitHub user dlmarion opened a pull request:

    https://github.com/apache/accumulo/pull/83

    ACCUMULO-4173: Host Regex Table Load Balancer

    A table load balancer that passes a subset of the tserver hosts to the configured tablet balancer for a table. Tservers that do not match any regex are part of a default pool. Tables with configured regex (or those that are in error) are balanced within the default pool of tservers.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/apache/accumulo ACCUMULO-4173

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/accumulo/pull/83.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #83
    
----
commit 594ecc3c6a876922b1a47301a619551e32c31e7a
Author: Dave Marion <dl...@apache.org>
Date:   2016-03-29T14:33:41Z

    Squashed commit of the following:
    
    commit 2afa820763ece3a1299ac9e8889532fa4966acff
    Author: Dave Marion <dl...@apache.org>
    Date:   Fri Mar 25 14:36:18 2016 -0400
    
        Added tests where tablets were either all assigned or partially assigned
    
    commit 9c08258b7907ad3195227d21a782ffe7545fa696
    Author: Dave Marion <dl...@apache.org>
    Date:   Fri Mar 25 14:26:57 2016 -0400
    
        Added test for out of bounds tablets
    
    commit be07ba3d20e378d88c71f77366e1b3f2dd325c73
    Author: Dave Marion <dl...@apache.org>
    Date:   Fri Mar 25 13:38:35 2016 -0400
    
        Added more tests
    
    commit dc842305d4fda77d2271ae8a111031bfb1eda9f7
    Author: Dave Marion <dl...@apache.org>
    Date:   Fri Mar 25 11:31:20 2016 -0400
    
        Initial work on Host based regex master balancer

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request: ACCUMULO-4173: Host Regex Table Load Balanc...

Posted by ivakegg <gi...@git.apache.org>.
Github user ivakegg commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/83#discussion_r57936464
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancer.java ---
    @@ -0,0 +1,294 @@
    +/*
    + * 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.net.InetAddress;
    +import java.net.UnknownHostException;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Set;
    +import java.util.SortedMap;
    +import java.util.TreeMap;
    +import java.util.regex.Pattern;
    +
    +import org.apache.accumulo.core.client.admin.TableOperations;
    +import org.apache.accumulo.core.conf.AccumuloConfiguration;
    +import org.apache.accumulo.core.conf.Property;
    +import org.apache.accumulo.core.data.impl.KeyExtent;
    +import org.apache.accumulo.core.master.thrift.TabletServerStatus;
    +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.thrift.TException;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * This balancer will create pools of tablet servers by grouping tablet servers that match a regex into the same pool and calling the balancer set on the table
    + * to balance within the set of matching tablet servers. <br>
    + * Regex properties for this balancer are specified as:<br>
    + * <b>table.custom.balancer.host.regex.&lt;tablename&gt;=&lt;regex&gt;</b><br>
    + * Periodically (default 5m) this balancer will check to see if a tablet server is hosting tablets that it should not be according to the regex configuration.
    + * 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 5m) 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>
    + * 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>
    + *
    + */
    +public class HostRegexTableLoadBalancer extends TableLoadBalancer {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(HostRegexTableLoadBalancer.class);
    +  public static final String HOST_BALANCER_PREFIX = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.";
    +  public static final String HOST_BALANCER_OOB_CHECK = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.oob.period";
    +  private static final String HOST_BALANCER_OOB_DEFAULT = "5m";
    +  public static final String HOST_BALANCER_POOL_RECHECK_KEY = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.pool.check";
    +  private static final String HOST_BALANCER_POOL_RECHECK_DEFAULT = "5m";
    +  public static final String HOST_BALANCER_REGEX_USING_IPS = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.is.ip";
    +  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 long lastOOBCheck = System.currentTimeMillis();
    +  private long lastPoolRecheck = 0;
    +  private boolean isIpBasedRegex = false;
    +  private Map<String,SortedMap<TServerInstance,TabletServerStatus>> pools = new HashMap<String,SortedMap<TServerInstance,TabletServerStatus>>();
    +
    +  /**
    +   * Group the set of current tservers by pool name. Tservers that don't match a regex are put into a default ppol.
    +   *
    +   * @param current
    +   *          map of current tservers
    +   * @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) {
    +      Map<String,SortedMap<TServerInstance,TabletServerStatus>> newPools = new HashMap<String,SortedMap<TServerInstance,TabletServerStatus>>();
    +      for (Entry<TServerInstance,TabletServerStatus> e : current.entrySet()) {
    +        List<String> poolNames = getPoolNamesForHost(e.getKey().host());
    +        for (String pool : poolNames) {
    +          if (!newPools.containsKey(pool)) {
    --- End diff --
    
    more efficient to do newPools.get(pool) and check result for null.  Avoids additional get down below when adding to the sorted map


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request: ACCUMULO-4173: Host Regex Table Load Balanc...

Posted by dlmarion <gi...@git.apache.org>.
Github user dlmarion commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/83#discussion_r57936092
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancer.java ---
    @@ -0,0 +1,314 @@
    +/*
    + * 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.net.InetAddress;
    +import java.net.UnknownHostException;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Set;
    +import java.util.SortedMap;
    +import java.util.TreeMap;
    +import java.util.regex.Pattern;
    +
    +import org.apache.accumulo.core.client.admin.TableOperations;
    +import org.apache.accumulo.core.conf.AccumuloConfiguration;
    +import org.apache.accumulo.core.conf.Property;
    +import org.apache.accumulo.core.data.impl.KeyExtent;
    +import org.apache.accumulo.core.master.thrift.TabletServerStatus;
    +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.thrift.TException;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * This balancer will create pools of tablet servers by grouping tablet servers that match a regex into the same pool and calling the balancer set on the table
    + * to balance within the set of matching tablet servers. <br>
    + * Regex properties for this balancer are specified as:<br>
    + * <b>table.custom.balancer.host.regex.&lt;tablename&gt;=&lt;regex&gt;</b><br>
    + * Periodically (default 5m) this balancer will check to see if a tablet server is hosting tablets that it should not be according to the regex configuration.
    + * 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 5m) 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>
    + * 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>
    + *
    + */
    +public class HostRegexTableLoadBalancer extends TableLoadBalancer {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(HostRegexTableLoadBalancer.class);
    +  public static final String HOST_BALANCER_PREFIX = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.";
    +  public static final String HOST_BALANCER_OOB_CHECK = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.oob.period";
    +  private static final String HOST_BALANCER_OOB_DEFAULT = "5m";
    +  public static final String HOST_BALANCER_POOL_RECHECK_KEY = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.pool.check";
    +  private static final String HOST_BALANCER_POOL_RECHECK_DEFAULT = "5m";
    +  public static final String HOST_BALANCER_REGEX_USING_IPS = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.is.ip";
    +  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 long lastOOBCheck = System.currentTimeMillis();
    +  private long lastPoolRecheck = 0;
    +  private boolean isIpBasedRegex = false;
    +  private Map<String,SortedMap<TServerInstance,TabletServerStatus>> pools = new HashMap<String,SortedMap<TServerInstance,TabletServerStatus>>();
    +
    +  /**
    +   * Group the set of current tservers by pool name. Tservers that don't match a regex are put into a default ppol.
    +   *
    +   * @param current
    +   *          map of current tservers
    +   * @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) {
    +      Map<String,SortedMap<TServerInstance,TabletServerStatus>> newPools = new HashMap<String,SortedMap<TServerInstance,TabletServerStatus>>();
    +      for (Entry<TServerInstance,TabletServerStatus> e : current.entrySet()) {
    +        String tableName = getPoolNameForHost(e.getKey().host());
    +        if (!newPools.containsKey(tableName)) {
    +          newPools.put(tableName, new TreeMap<TServerInstance,TabletServerStatus>());
    +        }
    +        newPools.get(tableName).put(e.getKey(), e.getValue());
    +      }
    +      // Ensure that no host is in more than one pool
    +      // TODO: I'm not sure that I need to check for disjoint as the call to getPoolNameForHost checks for more than one match
    +      boolean error = false;
    +      for (SortedMap<TServerInstance,TabletServerStatus> s1 : newPools.values()) {
    +        for (SortedMap<TServerInstance,TabletServerStatus> s2 : newPools.values()) {
    +          if (s1 == s2) {
    +            continue;
    +          }
    +          if (!Collections.disjoint(s1.keySet(), s2.keySet())) {
    +            LOG.error("Pools are not disjoint: {}, there is a problem with your regular expressions. Putting all servers in default pool", newPools);
    +            error = true;
    +          }
    +        }
    +      }
    +      if (error) {
    +        // Put all servers into the default pool
    +        newPools.clear();
    +        newPools.put(DEFAULT_POOL, new TreeMap<TServerInstance,TabletServerStatus>());
    +        for (Entry<TServerInstance,TabletServerStatus> e : current.entrySet()) {
    +          newPools.get(DEFAULT_POOL).put(e.getKey(), e.getValue());
    +        }
    +      }
    +      pools = newPools;
    +      this.lastPoolRecheck = System.currentTimeMillis();
    +    }
    +    return pools;
    +  }
    +
    +  /**
    +   * Matches host against the regexes and returns the matching pool name
    +   *
    +   * @param host
    +   *          tablet server host
    +   * @return name of pool. will return default pool if host matches more than one regex
    +   */
    +  protected String getPoolNameForHost(String host) {
    +    String test = host;
    +    String table = DEFAULT_POOL;
    +    if (!isIpBasedRegex) {
    +      try {
    +        test = getNameFromIp(host);
    +      } catch (UnknownHostException e1) {
    +        LOG.error("Unable to determine host name for IP: " + host + ", setting to default pool", e1);
    +        return table;
    +      }
    +    }
    +    for (Entry<String,Pattern> e : poolNameToRegexPattern.entrySet()) {
    +      if (e.getValue().matcher(test).matches()) {
    +        if (!table.equals(DEFAULT_POOL)) {
    +          LOG.warn("host {} matches more than one regex, assigning to default pool", host);
    +          return DEFAULT_POOL;
    +        }
    +        table = e.getKey();
    +      }
    +    }
    +    return table;
    +  }
    +
    +  protected String getNameFromIp(String hostIp) throws UnknownHostException {
    +    return InetAddress.getByName(hostIp).getHostName();
    +  }
    +
    +  /**
    +   * Matches table name against pool names, returns matching pool name or DEFAULT_POOL.
    +   *
    +   * @param tableName
    +   *          name of table
    +   * @return tablet server pool name (table name or DEFAULT_POOL)
    +   */
    +  protected String getPoolNameForTable(String tableName) {
    +    if (null == tableName) {
    +      return DEFAULT_POOL;
    +    }
    +    return poolNameToRegexPattern.containsKey(tableName) ? tableName : DEFAULT_POOL;
    +  }
    +
    +  /**
    +   * Parse configuration and extract properties
    +   *
    +   * @param conf
    +   *          server configuration
    +   */
    +  protected void parseConfiguration(ServerConfiguration conf) {
    +    tableIdToTableName = new HashMap<>();
    +    poolNameToRegexPattern = new HashMap<>();
    +    for (Entry<String,String> table : getTableOperations().tableIdMap().entrySet()) {
    +      tableIdToTableName.put(table.getValue(), table.getKey());
    +      Map<String,String> customProps = conf.getTableConfiguration(table.getValue()).getAllPropertiesWithPrefix(Property.TABLE_ARBITRARY_PROP_PREFIX);
    +      if (null != customProps && customProps.size() > 0) {
    +        for (Entry<String,String> customProp : customProps.entrySet()) {
    +          String tableName = customProp.getKey().substring(HOST_BALANCER_PREFIX.length());
    +          String regex = customProp.getValue();
    +          poolNameToRegexPattern.put(tableName, Pattern.compile(regex));
    +        }
    +      }
    +    }
    +    String oobProperty = conf.getConfiguration().get(HOST_BALANCER_OOB_CHECK);
    +    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);
    +    if (null != ipBased) {
    +      isIpBasedRegex = Boolean.parseBoolean(ipBased);
    +    }
    +  }
    +
    +  public Map<String,String> getTableIdToTableName() {
    +    return tableIdToTableName;
    +  }
    +
    +  public Map<String,Pattern> getPoolNameToRegexPattern() {
    +    return poolNameToRegexPattern;
    +  }
    +
    +  public long getOobCheckMillis() {
    +    return oobCheckMillis;
    +  }
    +
    +  public long getPoolRecheckMillis() {
    +    return poolRecheckMillis;
    +  }
    +
    +  public boolean isIpBasedRegex() {
    +    return isIpBasedRegex;
    +  }
    +
    +  @Override
    +  public void init(ServerConfiguration conf) {
    +    super.init(conf);
    +    parseConfiguration(conf);
    +  }
    +
    +  @Override
    +  public void getAssignments(SortedMap<TServerInstance,TabletServerStatus> current, Map<KeyExtent,TServerInstance> unassigned,
    +      Map<KeyExtent,TServerInstance> assignments) {
    +
    +    if ((System.currentTimeMillis() - this.lastOOBCheck) > this.oobCheckMillis) {
    +      // 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()) {
    +        String assignedPool = getPoolNameForHost(e.getKey().host());
    +        for (String pool : poolNameToRegexPattern.keySet()) {
    +          if (assignedPool.equals(pool) || pool.equals(DEFAULT_POOL)) {
    +            continue;
    +          }
    +          String tid = getTableOperations().tableIdMap().get(pool);
    +          try {
    +            List<TabletStats> outOfBoundsTablets = getOnlineTabletsForTable(e.getKey(), tid);
    +            for (TabletStats ts : outOfBoundsTablets) {
    +              LOG.info("Tablet {} is currently outside the bounds of the regex, reassigning", ts.toString());
    +              unassigned.put(new KeyExtent(ts.getExtent()), e.getKey());
    +            }
    +          } catch (TException e1) {
    +            LOG.error("Error in OOB check getting tablets for table {} from server {}", tid, e.getKey().host(), e);
    +          }
    +        }
    +      }
    +      this.oobCheckMillis = System.currentTimeMillis();
    +    }
    +
    +    Map<String,SortedMap<TServerInstance,TabletServerStatus>> pools = splitCurrentByRegex(current);
    +    // separate the unassigned into tables
    +    Map<String,Map<KeyExtent,TServerInstance>> groupedUnassigned = new HashMap<String,Map<KeyExtent,TServerInstance>>();
    +    for (Entry<KeyExtent,TServerInstance> e : unassigned.entrySet()) {
    +      Map<KeyExtent,TServerInstance> tableUnassigned = groupedUnassigned.get(e.getKey().getTableId());
    +      if (tableUnassigned == null) {
    +        tableUnassigned = new HashMap<KeyExtent,TServerInstance>();
    +        groupedUnassigned.put(e.getKey().getTableId(), tableUnassigned);
    +      }
    +      tableUnassigned.put(e.getKey(), e.getValue());
    +    }
    +    // Send a view of the current servers to the tables tablet balancer
    +    for (Entry<String,Map<KeyExtent,TServerInstance>> e : groupedUnassigned.entrySet()) {
    +      Map<KeyExtent,TServerInstance> newAssignments = new HashMap<KeyExtent,TServerInstance>();
    +      String tableName = tableIdToTableName.get(e.getKey());
    +      String poolName = getPoolNameForTable(tableName);
    +      SortedMap<TServerInstance,TabletServerStatus> currentView = pools.get(poolName);
    +      getBalancerForTable(e.getKey()).getAssignments(currentView, e.getValue(), newAssignments);
    +      assignments.putAll(newAssignments);
    +    }
    +  }
    +
    +  @Override
    +  public long balance(SortedMap<TServerInstance,TabletServerStatus> current, Set<KeyExtent> migrations, List<TabletMigration> migrationsOut) {
    +    long minBalanceTime = 5 * 1000;
    +    // Iterate over the tables and balance each of them
    +    TableOperations t = getTableOperations();
    +    if (t == null)
    --- End diff --
    
    Good catch. This is a copy/paste from TableLoadBalancer.balance(). It might be superfluous there.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request: ACCUMULO-4173: Host Regex Table Load Balanc...

Posted by dlmarion <gi...@git.apache.org>.
Github user dlmarion commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/83#discussion_r59393002
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancer.java ---
    @@ -0,0 +1,334 @@
    +/*
    + * 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.net.InetAddress;
    +import java.net.UnknownHostException;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Set;
    +import java.util.SortedMap;
    +import java.util.TreeMap;
    +import java.util.regex.Pattern;
    +
    +import org.apache.accumulo.core.client.admin.TableOperations;
    +import org.apache.accumulo.core.conf.AccumuloConfiguration;
    +import org.apache.accumulo.core.conf.Property;
    +import org.apache.accumulo.core.data.impl.KeyExtent;
    +import org.apache.accumulo.core.master.thrift.TabletServerStatus;
    +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.thrift.TException;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * This balancer will create pools of tablet servers by grouping tablet servers that match a regex into the same pool and calling the balancer set on the table
    + * to balance within the set of matching tablet servers. All tablet servers that do not match a regex are grouped into a default pool.<br>
    + * Regex properties for this balancer are specified as:<br>
    + * <b>table.custom.balancer.host.regex.&lt;tablename&gt;=&lt;regex&gt;</b><br>
    + * Periodically (default 5m) this balancer will check to see if a tablet server is hosting tablets that it should not be according to the regex configuration.
    + * 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 5m) 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>
    + * 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>
    + *
    + */
    +public class HostRegexTableLoadBalancer extends TableLoadBalancer {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(HostRegexTableLoadBalancer.class);
    +  public static final String HOST_BALANCER_PREFIX = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.";
    +  public static final String HOST_BALANCER_OOB_CHECK = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.oob.period";
    +  private static final String HOST_BALANCER_OOB_DEFAULT = "5m";
    +  public static final String HOST_BALANCER_POOL_RECHECK_KEY = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.pool.check";
    +  private static final String HOST_BALANCER_POOL_RECHECK_DEFAULT = "5m";
    +  public static final String HOST_BALANCER_REGEX_USING_IPS = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.is.ip";
    +  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<String,SortedMap<TServerInstance,TabletServerStatus>>();
    +
    +  /**
    +   * Group the set of current tservers by pool name. Tservers that don't match a regex are put into a default pool.
    +   *
    +   * @param current
    +   *          map of current tservers
    +   * @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 into pools based on regex");
    +      Map<String,SortedMap<TServerInstance,TabletServerStatus>> newPools = new HashMap<String,SortedMap<TServerInstance,TabletServerStatus>>();
    +      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<TServerInstance,TabletServerStatus>(current.comparator());
    +            newPools.put(pool, np);
    +          }
    +          np.put(e.getKey(), e.getValue());
    +        }
    +      }
    +      pools = newPools;
    +      this.lastPoolRecheck = System.currentTimeMillis();
    +    }
    +    return pools;
    +  }
    +
    +  /**
    +   * Matches host against the regexes and returns the matching pool names
    +   *
    +   * @param host
    +   *          tablet server host
    +   * @return pool names, will return default pool if host matches more no regex
    +   */
    +  protected List<String> getPoolNamesForHost(String host) {
    +    String test = host;
    +    if (!isIpBasedRegex) {
    +      try {
    +        test = getNameFromIp(host);
    +      } catch (UnknownHostException e1) {
    +        LOG.error("Unable to determine host name for IP: " + host + ", setting to default pool", e1);
    +        return Collections.singletonList(DEFAULT_POOL);
    +      }
    +    }
    +    List<String> pools = new ArrayList<>();
    +    for (Entry<String,Pattern> e : poolNameToRegexPattern.entrySet()) {
    +      if (e.getValue().matcher(test).matches()) {
    +        pools.add(e.getKey());
    +      }
    +    }
    +    if (pools.size() == 0) {
    +      pools.add(DEFAULT_POOL);
    +    }
    +    return pools;
    +  }
    +
    +  protected String getNameFromIp(String hostIp) throws UnknownHostException {
    +    return InetAddress.getByName(hostIp).getHostName();
    +  }
    +
    +  /**
    +   * Matches table name against pool names, returns matching pool name or DEFAULT_POOL.
    +   *
    +   * @param tableName
    +   *          name of table
    +   * @return tablet server pool name (table name or DEFAULT_POOL)
    +   */
    +  protected String getPoolNameForTable(String tableName) {
    +    if (null == tableName) {
    +      return DEFAULT_POOL;
    +    }
    +    return poolNameToRegexPattern.containsKey(tableName) ? tableName : DEFAULT_POOL;
    +  }
    +
    +  /**
    +   * Parse configuration and extract properties
    +   *
    +   * @param conf
    +   *          server configuration
    +   */
    +  protected void parseConfiguration(ServerConfiguration conf) {
    +    tableIdToTableName = new HashMap<>();
    +    poolNameToRegexPattern = new HashMap<>();
    +    for (Entry<String,String> table : getTableOperations().tableIdMap().entrySet()) {
    +      tableIdToTableName.put(table.getValue(), table.getKey());
    +      Map<String,String> customProps = conf.getTableConfiguration(table.getValue()).getAllPropertiesWithPrefix(Property.TABLE_ARBITRARY_PROP_PREFIX);
    +      if (null != customProps && customProps.size() > 0) {
    +        for (Entry<String,String> customProp : customProps.entrySet()) {
    +          if (customProp.getKey().startsWith(HOST_BALANCER_PREFIX)) {
    +            String tableName = customProp.getKey().substring(HOST_BALANCER_PREFIX.length());
    +            String regex = customProp.getValue();
    +            poolNameToRegexPattern.put(tableName, Pattern.compile(regex));
    +          }
    +        }
    +      }
    +    }
    +    String oobProperty = conf.getConfiguration().get(HOST_BALANCER_OOB_CHECK);
    +    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);
    +    if (null != ipBased) {
    +      isIpBasedRegex = Boolean.parseBoolean(ipBased);
    +    }
    +  }
    +
    +  public Map<String,String> getTableIdToTableName() {
    +    return tableIdToTableName;
    +  }
    +
    +  public Map<String,Pattern> getPoolNameToRegexPattern() {
    +    return poolNameToRegexPattern;
    +  }
    +
    +  public long getOobCheckMillis() {
    +    return oobCheckMillis;
    +  }
    +
    +  public long getPoolRecheckMillis() {
    +    return poolRecheckMillis;
    +  }
    +
    +  public boolean isIpBasedRegex() {
    +    return isIpBasedRegex;
    +  }
    +
    +  @Override
    +  public void init(ServerConfiguration conf) {
    +    super.init(conf);
    +    parseConfiguration(conf);
    +  }
    +
    +  @Override
    +  public void getAssignments(SortedMap<TServerInstance,TabletServerStatus> current, Map<KeyExtent,TServerInstance> unassigned,
    +      Map<KeyExtent,TServerInstance> assignments) {
    +
    +    Map<String,SortedMap<TServerInstance,TabletServerStatus>> pools = splitCurrentByRegex(current);
    --- End diff --
    
    Lowered to 1m


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request: ACCUMULO-4173: Host Regex Table Load Balanc...

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/83#discussion_r59381530
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancer.java ---
    @@ -0,0 +1,334 @@
    +/*
    + * 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.net.InetAddress;
    +import java.net.UnknownHostException;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Set;
    +import java.util.SortedMap;
    +import java.util.TreeMap;
    +import java.util.regex.Pattern;
    +
    +import org.apache.accumulo.core.client.admin.TableOperations;
    +import org.apache.accumulo.core.conf.AccumuloConfiguration;
    +import org.apache.accumulo.core.conf.Property;
    +import org.apache.accumulo.core.data.impl.KeyExtent;
    +import org.apache.accumulo.core.master.thrift.TabletServerStatus;
    +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.thrift.TException;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * This balancer will create pools of tablet servers by grouping tablet servers that match a regex into the same pool and calling the balancer set on the table
    --- End diff --
    
    Nit: this is more about how it is implemented than what it does. "This balancer creates groups of tabletservers, using user-provided regular expressions over the tabletservers' hostnames, and then delegates to another TableBalancer for each group of tabletserver" (or something).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request: ACCUMULO-4173: Host Regex Table Load Balanc...

Posted by brianloss <gi...@git.apache.org>.
Github user brianloss commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/83#discussion_r57933756
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancer.java ---
    @@ -0,0 +1,314 @@
    +/*
    + * 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.net.InetAddress;
    +import java.net.UnknownHostException;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Set;
    +import java.util.SortedMap;
    +import java.util.TreeMap;
    +import java.util.regex.Pattern;
    +
    +import org.apache.accumulo.core.client.admin.TableOperations;
    +import org.apache.accumulo.core.conf.AccumuloConfiguration;
    +import org.apache.accumulo.core.conf.Property;
    +import org.apache.accumulo.core.data.impl.KeyExtent;
    +import org.apache.accumulo.core.master.thrift.TabletServerStatus;
    +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.thrift.TException;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * This balancer will create pools of tablet servers by grouping tablet servers that match a regex into the same pool and calling the balancer set on the table
    + * to balance within the set of matching tablet servers. <br>
    + * Regex properties for this balancer are specified as:<br>
    + * <b>table.custom.balancer.host.regex.&lt;tablename&gt;=&lt;regex&gt;</b><br>
    + * Periodically (default 5m) this balancer will check to see if a tablet server is hosting tablets that it should not be according to the regex configuration.
    + * 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 5m) 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>
    + * 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>
    + *
    + */
    +public class HostRegexTableLoadBalancer extends TableLoadBalancer {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(HostRegexTableLoadBalancer.class);
    +  public static final String HOST_BALANCER_PREFIX = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.";
    +  public static final String HOST_BALANCER_OOB_CHECK = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.oob.period";
    +  private static final String HOST_BALANCER_OOB_DEFAULT = "5m";
    +  public static final String HOST_BALANCER_POOL_RECHECK_KEY = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.pool.check";
    +  private static final String HOST_BALANCER_POOL_RECHECK_DEFAULT = "5m";
    +  public static final String HOST_BALANCER_REGEX_USING_IPS = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.is.ip";
    +  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 long lastOOBCheck = System.currentTimeMillis();
    +  private long lastPoolRecheck = 0;
    +  private boolean isIpBasedRegex = false;
    +  private Map<String,SortedMap<TServerInstance,TabletServerStatus>> pools = new HashMap<String,SortedMap<TServerInstance,TabletServerStatus>>();
    +
    +  /**
    +   * Group the set of current tservers by pool name. Tservers that don't match a regex are put into a default ppol.
    +   *
    +   * @param current
    +   *          map of current tservers
    +   * @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) {
    +      Map<String,SortedMap<TServerInstance,TabletServerStatus>> newPools = new HashMap<String,SortedMap<TServerInstance,TabletServerStatus>>();
    +      for (Entry<TServerInstance,TabletServerStatus> e : current.entrySet()) {
    +        String tableName = getPoolNameForHost(e.getKey().host());
    +        if (!newPools.containsKey(tableName)) {
    +          newPools.put(tableName, new TreeMap<TServerInstance,TabletServerStatus>());
    +        }
    +        newPools.get(tableName).put(e.getKey(), e.getValue());
    +      }
    +      // Ensure that no host is in more than one pool
    +      // TODO: I'm not sure that I need to check for disjoint as the call to getPoolNameForHost checks for more than one match
    +      boolean error = false;
    +      for (SortedMap<TServerInstance,TabletServerStatus> s1 : newPools.values()) {
    +        for (SortedMap<TServerInstance,TabletServerStatus> s2 : newPools.values()) {
    +          if (s1 == s2) {
    +            continue;
    +          }
    +          if (!Collections.disjoint(s1.keySet(), s2.keySet())) {
    +            LOG.error("Pools are not disjoint: {}, there is a problem with your regular expressions. Putting all servers in default pool", newPools);
    +            error = true;
    +          }
    +        }
    +      }
    +      if (error) {
    +        // Put all servers into the default pool
    +        newPools.clear();
    +        newPools.put(DEFAULT_POOL, new TreeMap<TServerInstance,TabletServerStatus>());
    +        for (Entry<TServerInstance,TabletServerStatus> e : current.entrySet()) {
    +          newPools.get(DEFAULT_POOL).put(e.getKey(), e.getValue());
    +        }
    +      }
    +      pools = newPools;
    +      this.lastPoolRecheck = System.currentTimeMillis();
    +    }
    +    return pools;
    +  }
    +
    +  /**
    +   * Matches host against the regexes and returns the matching pool name
    +   *
    +   * @param host
    +   *          tablet server host
    +   * @return name of pool. will return default pool if host matches more than one regex
    +   */
    +  protected String getPoolNameForHost(String host) {
    +    String test = host;
    +    String table = DEFAULT_POOL;
    +    if (!isIpBasedRegex) {
    +      try {
    +        test = getNameFromIp(host);
    +      } catch (UnknownHostException e1) {
    +        LOG.error("Unable to determine host name for IP: " + host + ", setting to default pool", e1);
    +        return table;
    +      }
    +    }
    +    for (Entry<String,Pattern> e : poolNameToRegexPattern.entrySet()) {
    +      if (e.getValue().matcher(test).matches()) {
    +        if (!table.equals(DEFAULT_POOL)) {
    +          LOG.warn("host {} matches more than one regex, assigning to default pool", host);
    +          return DEFAULT_POOL;
    +        }
    +        table = e.getKey();
    +      }
    +    }
    +    return table;
    +  }
    +
    +  protected String getNameFromIp(String hostIp) throws UnknownHostException {
    +    return InetAddress.getByName(hostIp).getHostName();
    +  }
    +
    +  /**
    +   * Matches table name against pool names, returns matching pool name or DEFAULT_POOL.
    +   *
    +   * @param tableName
    +   *          name of table
    +   * @return tablet server pool name (table name or DEFAULT_POOL)
    +   */
    +  protected String getPoolNameForTable(String tableName) {
    +    if (null == tableName) {
    +      return DEFAULT_POOL;
    +    }
    +    return poolNameToRegexPattern.containsKey(tableName) ? tableName : DEFAULT_POOL;
    +  }
    +
    +  /**
    +   * Parse configuration and extract properties
    +   *
    +   * @param conf
    +   *          server configuration
    +   */
    +  protected void parseConfiguration(ServerConfiguration conf) {
    +    tableIdToTableName = new HashMap<>();
    +    poolNameToRegexPattern = new HashMap<>();
    +    for (Entry<String,String> table : getTableOperations().tableIdMap().entrySet()) {
    +      tableIdToTableName.put(table.getValue(), table.getKey());
    +      Map<String,String> customProps = conf.getTableConfiguration(table.getValue()).getAllPropertiesWithPrefix(Property.TABLE_ARBITRARY_PROP_PREFIX);
    +      if (null != customProps && customProps.size() > 0) {
    +        for (Entry<String,String> customProp : customProps.entrySet()) {
    +          String tableName = customProp.getKey().substring(HOST_BALANCER_PREFIX.length());
    +          String regex = customProp.getValue();
    +          poolNameToRegexPattern.put(tableName, Pattern.compile(regex));
    +        }
    +      }
    +    }
    +    String oobProperty = conf.getConfiguration().get(HOST_BALANCER_OOB_CHECK);
    +    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);
    +    if (null != ipBased) {
    +      isIpBasedRegex = Boolean.parseBoolean(ipBased);
    +    }
    +  }
    +
    +  public Map<String,String> getTableIdToTableName() {
    +    return tableIdToTableName;
    +  }
    +
    +  public Map<String,Pattern> getPoolNameToRegexPattern() {
    +    return poolNameToRegexPattern;
    +  }
    +
    +  public long getOobCheckMillis() {
    +    return oobCheckMillis;
    +  }
    +
    +  public long getPoolRecheckMillis() {
    +    return poolRecheckMillis;
    +  }
    +
    +  public boolean isIpBasedRegex() {
    +    return isIpBasedRegex;
    +  }
    +
    +  @Override
    +  public void init(ServerConfiguration conf) {
    +    super.init(conf);
    +    parseConfiguration(conf);
    +  }
    +
    +  @Override
    +  public void getAssignments(SortedMap<TServerInstance,TabletServerStatus> current, Map<KeyExtent,TServerInstance> unassigned,
    +      Map<KeyExtent,TServerInstance> assignments) {
    +
    +    if ((System.currentTimeMillis() - this.lastOOBCheck) > this.oobCheckMillis) {
    +      // 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()) {
    +        String assignedPool = getPoolNameForHost(e.getKey().host());
    +        for (String pool : poolNameToRegexPattern.keySet()) {
    +          if (assignedPool.equals(pool) || pool.equals(DEFAULT_POOL)) {
    +            continue;
    +          }
    +          String tid = getTableOperations().tableIdMap().get(pool);
    +          try {
    +            List<TabletStats> outOfBoundsTablets = getOnlineTabletsForTable(e.getKey(), tid);
    +            for (TabletStats ts : outOfBoundsTablets) {
    +              LOG.info("Tablet {} is currently outside the bounds of the regex, reassigning", ts.toString());
    +              unassigned.put(new KeyExtent(ts.getExtent()), e.getKey());
    --- End diff --
    
    getAssignments is for making assignments where there isn't one currently. Is it safe to take a tablet that is already assigned, and generate a new assignment here? It might be totally fine--but it's something to verify.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request: ACCUMULO-4173: Host Regex Table Load Balanc...

Posted by dlmarion <gi...@git.apache.org>.
Github user dlmarion commented on the pull request:

    https://github.com/apache/accumulo/pull/83#issuecomment-208920076
  
    Update PR based on changes from actually testing this on a test cluster. Any final comments?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request: ACCUMULO-4173: Host Regex Table Load Balanc...

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/83#discussion_r59383136
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancer.java ---
    @@ -0,0 +1,334 @@
    +/*
    + * 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.net.InetAddress;
    +import java.net.UnknownHostException;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Set;
    +import java.util.SortedMap;
    +import java.util.TreeMap;
    +import java.util.regex.Pattern;
    +
    +import org.apache.accumulo.core.client.admin.TableOperations;
    +import org.apache.accumulo.core.conf.AccumuloConfiguration;
    +import org.apache.accumulo.core.conf.Property;
    +import org.apache.accumulo.core.data.impl.KeyExtent;
    +import org.apache.accumulo.core.master.thrift.TabletServerStatus;
    +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.thrift.TException;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * This balancer will create pools of tablet servers by grouping tablet servers that match a regex into the same pool and calling the balancer set on the table
    + * to balance within the set of matching tablet servers. All tablet servers that do not match a regex are grouped into a default pool.<br>
    + * Regex properties for this balancer are specified as:<br>
    + * <b>table.custom.balancer.host.regex.&lt;tablename&gt;=&lt;regex&gt;</b><br>
    + * Periodically (default 5m) this balancer will check to see if a tablet server is hosting tablets that it should not be according to the regex configuration.
    + * 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 5m) 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>
    + * 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>
    + *
    + */
    +public class HostRegexTableLoadBalancer extends TableLoadBalancer {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(HostRegexTableLoadBalancer.class);
    +  public static final String HOST_BALANCER_PREFIX = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.";
    +  public static final String HOST_BALANCER_OOB_CHECK = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.oob.period";
    +  private static final String HOST_BALANCER_OOB_DEFAULT = "5m";
    +  public static final String HOST_BALANCER_POOL_RECHECK_KEY = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.pool.check";
    +  private static final String HOST_BALANCER_POOL_RECHECK_DEFAULT = "5m";
    +  public static final String HOST_BALANCER_REGEX_USING_IPS = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.is.ip";
    +  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<String,SortedMap<TServerInstance,TabletServerStatus>>();
    +
    +  /**
    +   * Group the set of current tservers by pool name. Tservers that don't match a regex are put into a default pool.
    +   *
    +   * @param current
    +   *          map of current tservers
    +   * @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 into pools based on regex");
    +      Map<String,SortedMap<TServerInstance,TabletServerStatus>> newPools = new HashMap<String,SortedMap<TServerInstance,TabletServerStatus>>();
    +      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<TServerInstance,TabletServerStatus>(current.comparator());
    +            newPools.put(pool, np);
    +          }
    +          np.put(e.getKey(), e.getValue());
    +        }
    +      }
    +      pools = newPools;
    +      this.lastPoolRecheck = System.currentTimeMillis();
    +    }
    +    return pools;
    +  }
    +
    +  /**
    +   * Matches host against the regexes and returns the matching pool names
    +   *
    +   * @param host
    +   *          tablet server host
    +   * @return pool names, will return default pool if host matches more no regex
    +   */
    +  protected List<String> getPoolNamesForHost(String host) {
    +    String test = host;
    +    if (!isIpBasedRegex) {
    +      try {
    +        test = getNameFromIp(host);
    +      } catch (UnknownHostException e1) {
    +        LOG.error("Unable to determine host name for IP: " + host + ", setting to default pool", e1);
    +        return Collections.singletonList(DEFAULT_POOL);
    +      }
    +    }
    +    List<String> pools = new ArrayList<>();
    +    for (Entry<String,Pattern> e : poolNameToRegexPattern.entrySet()) {
    +      if (e.getValue().matcher(test).matches()) {
    +        pools.add(e.getKey());
    +      }
    +    }
    +    if (pools.size() == 0) {
    +      pools.add(DEFAULT_POOL);
    +    }
    +    return pools;
    +  }
    +
    +  protected String getNameFromIp(String hostIp) throws UnknownHostException {
    +    return InetAddress.getByName(hostIp).getHostName();
    +  }
    +
    +  /**
    +   * Matches table name against pool names, returns matching pool name or DEFAULT_POOL.
    +   *
    +   * @param tableName
    +   *          name of table
    +   * @return tablet server pool name (table name or DEFAULT_POOL)
    +   */
    +  protected String getPoolNameForTable(String tableName) {
    +    if (null == tableName) {
    +      return DEFAULT_POOL;
    +    }
    +    return poolNameToRegexPattern.containsKey(tableName) ? tableName : DEFAULT_POOL;
    +  }
    +
    +  /**
    +   * Parse configuration and extract properties
    +   *
    +   * @param conf
    +   *          server configuration
    +   */
    +  protected void parseConfiguration(ServerConfiguration conf) {
    +    tableIdToTableName = new HashMap<>();
    +    poolNameToRegexPattern = new HashMap<>();
    +    for (Entry<String,String> table : getTableOperations().tableIdMap().entrySet()) {
    +      tableIdToTableName.put(table.getValue(), table.getKey());
    +      Map<String,String> customProps = conf.getTableConfiguration(table.getValue()).getAllPropertiesWithPrefix(Property.TABLE_ARBITRARY_PROP_PREFIX);
    +      if (null != customProps && customProps.size() > 0) {
    +        for (Entry<String,String> customProp : customProps.entrySet()) {
    +          if (customProp.getKey().startsWith(HOST_BALANCER_PREFIX)) {
    +            String tableName = customProp.getKey().substring(HOST_BALANCER_PREFIX.length());
    +            String regex = customProp.getValue();
    +            poolNameToRegexPattern.put(tableName, Pattern.compile(regex));
    +          }
    +        }
    +      }
    +    }
    +    String oobProperty = conf.getConfiguration().get(HOST_BALANCER_OOB_CHECK);
    +    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);
    +    if (null != ipBased) {
    +      isIpBasedRegex = Boolean.parseBoolean(ipBased);
    +    }
    +  }
    +
    +  public Map<String,String> getTableIdToTableName() {
    +    return tableIdToTableName;
    +  }
    +
    +  public Map<String,Pattern> getPoolNameToRegexPattern() {
    +    return poolNameToRegexPattern;
    +  }
    +
    +  public long getOobCheckMillis() {
    +    return oobCheckMillis;
    +  }
    +
    +  public long getPoolRecheckMillis() {
    +    return poolRecheckMillis;
    +  }
    +
    +  public boolean isIpBasedRegex() {
    +    return isIpBasedRegex;
    +  }
    +
    +  @Override
    +  public void init(ServerConfiguration conf) {
    +    super.init(conf);
    +    parseConfiguration(conf);
    +  }
    +
    +  @Override
    +  public void getAssignments(SortedMap<TServerInstance,TabletServerStatus> current, Map<KeyExtent,TServerInstance> unassigned,
    +      Map<KeyExtent,TServerInstance> assignments) {
    +
    +    Map<String,SortedMap<TServerInstance,TabletServerStatus>> pools = splitCurrentByRegex(current);
    --- End diff --
    
    This seems like it would be problematic. I would have to wait a default of 5minutes for tablets to be migrated to a new server in a pool (because the pools would not be recomputed). Did I miss something?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request: ACCUMULO-4173: Host Regex Table Load Balanc...

Posted by dlmarion <gi...@git.apache.org>.
Github user dlmarion commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/83#discussion_r57934586
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancer.java ---
    @@ -0,0 +1,314 @@
    +/*
    + * 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.net.InetAddress;
    +import java.net.UnknownHostException;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Set;
    +import java.util.SortedMap;
    +import java.util.TreeMap;
    +import java.util.regex.Pattern;
    +
    +import org.apache.accumulo.core.client.admin.TableOperations;
    +import org.apache.accumulo.core.conf.AccumuloConfiguration;
    +import org.apache.accumulo.core.conf.Property;
    +import org.apache.accumulo.core.data.impl.KeyExtent;
    +import org.apache.accumulo.core.master.thrift.TabletServerStatus;
    +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.thrift.TException;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * This balancer will create pools of tablet servers by grouping tablet servers that match a regex into the same pool and calling the balancer set on the table
    + * to balance within the set of matching tablet servers. <br>
    + * Regex properties for this balancer are specified as:<br>
    + * <b>table.custom.balancer.host.regex.&lt;tablename&gt;=&lt;regex&gt;</b><br>
    + * Periodically (default 5m) this balancer will check to see if a tablet server is hosting tablets that it should not be according to the regex configuration.
    + * 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 5m) 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>
    + * 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>
    + *
    + */
    +public class HostRegexTableLoadBalancer extends TableLoadBalancer {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(HostRegexTableLoadBalancer.class);
    +  public static final String HOST_BALANCER_PREFIX = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.";
    +  public static final String HOST_BALANCER_OOB_CHECK = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.oob.period";
    +  private static final String HOST_BALANCER_OOB_DEFAULT = "5m";
    +  public static final String HOST_BALANCER_POOL_RECHECK_KEY = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.pool.check";
    +  private static final String HOST_BALANCER_POOL_RECHECK_DEFAULT = "5m";
    +  public static final String HOST_BALANCER_REGEX_USING_IPS = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.is.ip";
    +  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 long lastOOBCheck = System.currentTimeMillis();
    +  private long lastPoolRecheck = 0;
    +  private boolean isIpBasedRegex = false;
    +  private Map<String,SortedMap<TServerInstance,TabletServerStatus>> pools = new HashMap<String,SortedMap<TServerInstance,TabletServerStatus>>();
    +
    +  /**
    +   * Group the set of current tservers by pool name. Tservers that don't match a regex are put into a default ppol.
    +   *
    +   * @param current
    +   *          map of current tservers
    +   * @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) {
    +      Map<String,SortedMap<TServerInstance,TabletServerStatus>> newPools = new HashMap<String,SortedMap<TServerInstance,TabletServerStatus>>();
    +      for (Entry<TServerInstance,TabletServerStatus> e : current.entrySet()) {
    +        String tableName = getPoolNameForHost(e.getKey().host());
    +        if (!newPools.containsKey(tableName)) {
    +          newPools.put(tableName, new TreeMap<TServerInstance,TabletServerStatus>());
    +        }
    +        newPools.get(tableName).put(e.getKey(), e.getValue());
    +      }
    +      // Ensure that no host is in more than one pool
    +      // TODO: I'm not sure that I need to check for disjoint as the call to getPoolNameForHost checks for more than one match
    +      boolean error = false;
    +      for (SortedMap<TServerInstance,TabletServerStatus> s1 : newPools.values()) {
    +        for (SortedMap<TServerInstance,TabletServerStatus> s2 : newPools.values()) {
    +          if (s1 == s2) {
    +            continue;
    +          }
    +          if (!Collections.disjoint(s1.keySet(), s2.keySet())) {
    +            LOG.error("Pools are not disjoint: {}, there is a problem with your regular expressions. Putting all servers in default pool", newPools);
    +            error = true;
    +          }
    +        }
    +      }
    +      if (error) {
    +        // Put all servers into the default pool
    +        newPools.clear();
    +        newPools.put(DEFAULT_POOL, new TreeMap<TServerInstance,TabletServerStatus>());
    +        for (Entry<TServerInstance,TabletServerStatus> e : current.entrySet()) {
    +          newPools.get(DEFAULT_POOL).put(e.getKey(), e.getValue());
    +        }
    +      }
    +      pools = newPools;
    +      this.lastPoolRecheck = System.currentTimeMillis();
    +    }
    +    return pools;
    +  }
    +
    +  /**
    +   * Matches host against the regexes and returns the matching pool name
    +   *
    +   * @param host
    +   *          tablet server host
    +   * @return name of pool. will return default pool if host matches more than one regex
    +   */
    +  protected String getPoolNameForHost(String host) {
    +    String test = host;
    +    String table = DEFAULT_POOL;
    +    if (!isIpBasedRegex) {
    +      try {
    +        test = getNameFromIp(host);
    +      } catch (UnknownHostException e1) {
    +        LOG.error("Unable to determine host name for IP: " + host + ", setting to default pool", e1);
    +        return table;
    +      }
    +    }
    +    for (Entry<String,Pattern> e : poolNameToRegexPattern.entrySet()) {
    +      if (e.getValue().matcher(test).matches()) {
    +        if (!table.equals(DEFAULT_POOL)) {
    +          LOG.warn("host {} matches more than one regex, assigning to default pool", host);
    +          return DEFAULT_POOL;
    +        }
    +        table = e.getKey();
    +      }
    +    }
    +    return table;
    +  }
    +
    +  protected String getNameFromIp(String hostIp) throws UnknownHostException {
    +    return InetAddress.getByName(hostIp).getHostName();
    +  }
    +
    +  /**
    +   * Matches table name against pool names, returns matching pool name or DEFAULT_POOL.
    +   *
    +   * @param tableName
    +   *          name of table
    +   * @return tablet server pool name (table name or DEFAULT_POOL)
    +   */
    +  protected String getPoolNameForTable(String tableName) {
    +    if (null == tableName) {
    +      return DEFAULT_POOL;
    +    }
    +    return poolNameToRegexPattern.containsKey(tableName) ? tableName : DEFAULT_POOL;
    +  }
    +
    +  /**
    +   * Parse configuration and extract properties
    +   *
    +   * @param conf
    +   *          server configuration
    +   */
    +  protected void parseConfiguration(ServerConfiguration conf) {
    +    tableIdToTableName = new HashMap<>();
    +    poolNameToRegexPattern = new HashMap<>();
    +    for (Entry<String,String> table : getTableOperations().tableIdMap().entrySet()) {
    --- End diff --
    
    I think you would need to restart the master


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request: ACCUMULO-4173: Host Regex Table Load Balanc...

Posted by brianloss <gi...@git.apache.org>.
Github user brianloss commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/83#discussion_r57934799
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancer.java ---
    @@ -0,0 +1,314 @@
    +/*
    + * 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.net.InetAddress;
    +import java.net.UnknownHostException;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Set;
    +import java.util.SortedMap;
    +import java.util.TreeMap;
    +import java.util.regex.Pattern;
    +
    +import org.apache.accumulo.core.client.admin.TableOperations;
    +import org.apache.accumulo.core.conf.AccumuloConfiguration;
    +import org.apache.accumulo.core.conf.Property;
    +import org.apache.accumulo.core.data.impl.KeyExtent;
    +import org.apache.accumulo.core.master.thrift.TabletServerStatus;
    +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.thrift.TException;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * This balancer will create pools of tablet servers by grouping tablet servers that match a regex into the same pool and calling the balancer set on the table
    + * to balance within the set of matching tablet servers. <br>
    + * Regex properties for this balancer are specified as:<br>
    + * <b>table.custom.balancer.host.regex.&lt;tablename&gt;=&lt;regex&gt;</b><br>
    + * Periodically (default 5m) this balancer will check to see if a tablet server is hosting tablets that it should not be according to the regex configuration.
    + * 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 5m) 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>
    + * 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>
    + *
    + */
    +public class HostRegexTableLoadBalancer extends TableLoadBalancer {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(HostRegexTableLoadBalancer.class);
    +  public static final String HOST_BALANCER_PREFIX = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.";
    +  public static final String HOST_BALANCER_OOB_CHECK = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.oob.period";
    +  private static final String HOST_BALANCER_OOB_DEFAULT = "5m";
    +  public static final String HOST_BALANCER_POOL_RECHECK_KEY = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.pool.check";
    +  private static final String HOST_BALANCER_POOL_RECHECK_DEFAULT = "5m";
    +  public static final String HOST_BALANCER_REGEX_USING_IPS = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.is.ip";
    +  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 long lastOOBCheck = System.currentTimeMillis();
    +  private long lastPoolRecheck = 0;
    +  private boolean isIpBasedRegex = false;
    +  private Map<String,SortedMap<TServerInstance,TabletServerStatus>> pools = new HashMap<String,SortedMap<TServerInstance,TabletServerStatus>>();
    +
    +  /**
    +   * Group the set of current tservers by pool name. Tservers that don't match a regex are put into a default ppol.
    +   *
    +   * @param current
    +   *          map of current tservers
    +   * @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) {
    +      Map<String,SortedMap<TServerInstance,TabletServerStatus>> newPools = new HashMap<String,SortedMap<TServerInstance,TabletServerStatus>>();
    +      for (Entry<TServerInstance,TabletServerStatus> e : current.entrySet()) {
    +        String tableName = getPoolNameForHost(e.getKey().host());
    +        if (!newPools.containsKey(tableName)) {
    +          newPools.put(tableName, new TreeMap<TServerInstance,TabletServerStatus>());
    +        }
    +        newPools.get(tableName).put(e.getKey(), e.getValue());
    +      }
    +      // Ensure that no host is in more than one pool
    +      // TODO: I'm not sure that I need to check for disjoint as the call to getPoolNameForHost checks for more than one match
    +      boolean error = false;
    +      for (SortedMap<TServerInstance,TabletServerStatus> s1 : newPools.values()) {
    +        for (SortedMap<TServerInstance,TabletServerStatus> s2 : newPools.values()) {
    +          if (s1 == s2) {
    +            continue;
    +          }
    +          if (!Collections.disjoint(s1.keySet(), s2.keySet())) {
    +            LOG.error("Pools are not disjoint: {}, there is a problem with your regular expressions. Putting all servers in default pool", newPools);
    +            error = true;
    +          }
    +        }
    +      }
    +      if (error) {
    +        // Put all servers into the default pool
    +        newPools.clear();
    +        newPools.put(DEFAULT_POOL, new TreeMap<TServerInstance,TabletServerStatus>());
    +        for (Entry<TServerInstance,TabletServerStatus> e : current.entrySet()) {
    +          newPools.get(DEFAULT_POOL).put(e.getKey(), e.getValue());
    +        }
    +      }
    +      pools = newPools;
    +      this.lastPoolRecheck = System.currentTimeMillis();
    +    }
    +    return pools;
    +  }
    +
    +  /**
    +   * Matches host against the regexes and returns the matching pool name
    +   *
    +   * @param host
    +   *          tablet server host
    +   * @return name of pool. will return default pool if host matches more than one regex
    +   */
    +  protected String getPoolNameForHost(String host) {
    +    String test = host;
    +    String table = DEFAULT_POOL;
    +    if (!isIpBasedRegex) {
    +      try {
    +        test = getNameFromIp(host);
    +      } catch (UnknownHostException e1) {
    +        LOG.error("Unable to determine host name for IP: " + host + ", setting to default pool", e1);
    +        return table;
    +      }
    +    }
    +    for (Entry<String,Pattern> e : poolNameToRegexPattern.entrySet()) {
    +      if (e.getValue().matcher(test).matches()) {
    +        if (!table.equals(DEFAULT_POOL)) {
    +          LOG.warn("host {} matches more than one regex, assigning to default pool", host);
    +          return DEFAULT_POOL;
    +        }
    +        table = e.getKey();
    +      }
    +    }
    +    return table;
    +  }
    +
    +  protected String getNameFromIp(String hostIp) throws UnknownHostException {
    +    return InetAddress.getByName(hostIp).getHostName();
    +  }
    +
    +  /**
    +   * Matches table name against pool names, returns matching pool name or DEFAULT_POOL.
    +   *
    +   * @param tableName
    +   *          name of table
    +   * @return tablet server pool name (table name or DEFAULT_POOL)
    +   */
    +  protected String getPoolNameForTable(String tableName) {
    +    if (null == tableName) {
    +      return DEFAULT_POOL;
    +    }
    +    return poolNameToRegexPattern.containsKey(tableName) ? tableName : DEFAULT_POOL;
    +  }
    +
    +  /**
    +   * Parse configuration and extract properties
    +   *
    +   * @param conf
    +   *          server configuration
    +   */
    +  protected void parseConfiguration(ServerConfiguration conf) {
    +    tableIdToTableName = new HashMap<>();
    +    poolNameToRegexPattern = new HashMap<>();
    +    for (Entry<String,String> table : getTableOperations().tableIdMap().entrySet()) {
    +      tableIdToTableName.put(table.getValue(), table.getKey());
    +      Map<String,String> customProps = conf.getTableConfiguration(table.getValue()).getAllPropertiesWithPrefix(Property.TABLE_ARBITRARY_PROP_PREFIX);
    +      if (null != customProps && customProps.size() > 0) {
    +        for (Entry<String,String> customProp : customProps.entrySet()) {
    +          String tableName = customProp.getKey().substring(HOST_BALANCER_PREFIX.length());
    +          String regex = customProp.getValue();
    +          poolNameToRegexPattern.put(tableName, Pattern.compile(regex));
    +        }
    +      }
    +    }
    +    String oobProperty = conf.getConfiguration().get(HOST_BALANCER_OOB_CHECK);
    +    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);
    +    if (null != ipBased) {
    +      isIpBasedRegex = Boolean.parseBoolean(ipBased);
    +    }
    +  }
    +
    +  public Map<String,String> getTableIdToTableName() {
    +    return tableIdToTableName;
    +  }
    +
    +  public Map<String,Pattern> getPoolNameToRegexPattern() {
    +    return poolNameToRegexPattern;
    +  }
    +
    +  public long getOobCheckMillis() {
    +    return oobCheckMillis;
    +  }
    +
    +  public long getPoolRecheckMillis() {
    +    return poolRecheckMillis;
    +  }
    +
    +  public boolean isIpBasedRegex() {
    +    return isIpBasedRegex;
    +  }
    +
    +  @Override
    +  public void init(ServerConfiguration conf) {
    +    super.init(conf);
    +    parseConfiguration(conf);
    +  }
    +
    +  @Override
    +  public void getAssignments(SortedMap<TServerInstance,TabletServerStatus> current, Map<KeyExtent,TServerInstance> unassigned,
    +      Map<KeyExtent,TServerInstance> assignments) {
    +
    +    if ((System.currentTimeMillis() - this.lastOOBCheck) > this.oobCheckMillis) {
    +      // 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()) {
    +        String assignedPool = getPoolNameForHost(e.getKey().host());
    +        for (String pool : poolNameToRegexPattern.keySet()) {
    +          if (assignedPool.equals(pool) || pool.equals(DEFAULT_POOL)) {
    +            continue;
    +          }
    +          String tid = getTableOperations().tableIdMap().get(pool);
    +          try {
    +            List<TabletStats> outOfBoundsTablets = getOnlineTabletsForTable(e.getKey(), tid);
    +            for (TabletStats ts : outOfBoundsTablets) {
    +              LOG.info("Tablet {} is currently outside the bounds of the regex, reassigning", ts.toString());
    +              unassigned.put(new KeyExtent(ts.getExtent()), e.getKey());
    --- End diff --
    
    Assuming this is safe for now, you are taking an entry from current, adding it to unassigned, but leaving it in current. Then the results get passed to the table-specific balancer below. Not sure if this will cause any problems for the table-specific balancers. It's probably worth removing the assignment from current.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request: ACCUMULO-4173: Host Regex Table Load Balanc...

Posted by dlmarion <gi...@git.apache.org>.
Github user dlmarion commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/83#discussion_r57950095
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancer.java ---
    @@ -0,0 +1,294 @@
    +/*
    + * 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.net.InetAddress;
    +import java.net.UnknownHostException;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Set;
    +import java.util.SortedMap;
    +import java.util.TreeMap;
    +import java.util.regex.Pattern;
    +
    +import org.apache.accumulo.core.client.admin.TableOperations;
    +import org.apache.accumulo.core.conf.AccumuloConfiguration;
    +import org.apache.accumulo.core.conf.Property;
    +import org.apache.accumulo.core.data.impl.KeyExtent;
    +import org.apache.accumulo.core.master.thrift.TabletServerStatus;
    +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.thrift.TException;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * This balancer will create pools of tablet servers by grouping tablet servers that match a regex into the same pool and calling the balancer set on the table
    + * to balance within the set of matching tablet servers. <br>
    + * Regex properties for this balancer are specified as:<br>
    + * <b>table.custom.balancer.host.regex.&lt;tablename&gt;=&lt;regex&gt;</b><br>
    + * Periodically (default 5m) this balancer will check to see if a tablet server is hosting tablets that it should not be according to the regex configuration.
    + * 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 5m) 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>
    + * 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>
    + *
    + */
    +public class HostRegexTableLoadBalancer extends TableLoadBalancer {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(HostRegexTableLoadBalancer.class);
    +  public static final String HOST_BALANCER_PREFIX = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.";
    +  public static final String HOST_BALANCER_OOB_CHECK = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.oob.period";
    +  private static final String HOST_BALANCER_OOB_DEFAULT = "5m";
    +  public static final String HOST_BALANCER_POOL_RECHECK_KEY = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.pool.check";
    +  private static final String HOST_BALANCER_POOL_RECHECK_DEFAULT = "5m";
    +  public static final String HOST_BALANCER_REGEX_USING_IPS = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.is.ip";
    +  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 long lastOOBCheck = System.currentTimeMillis();
    +  private long lastPoolRecheck = 0;
    +  private boolean isIpBasedRegex = false;
    +  private Map<String,SortedMap<TServerInstance,TabletServerStatus>> pools = new HashMap<String,SortedMap<TServerInstance,TabletServerStatus>>();
    +
    +  /**
    +   * Group the set of current tservers by pool name. Tservers that don't match a regex are put into a default ppol.
    +   *
    +   * @param current
    +   *          map of current tservers
    +   * @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) {
    +      Map<String,SortedMap<TServerInstance,TabletServerStatus>> newPools = new HashMap<String,SortedMap<TServerInstance,TabletServerStatus>>();
    +      for (Entry<TServerInstance,TabletServerStatus> e : current.entrySet()) {
    +        List<String> poolNames = getPoolNamesForHost(e.getKey().host());
    +        for (String pool : poolNames) {
    +          if (!newPools.containsKey(pool)) {
    +            newPools.put(pool, new TreeMap<TServerInstance,TabletServerStatus>(current.comparator()));
    +          }
    +          newPools.get(pool).put(e.getKey(), e.getValue());
    +        }
    +      }
    +      pools = newPools;
    +      this.lastPoolRecheck = System.currentTimeMillis();
    +    }
    +    return pools;
    +  }
    +
    +  /**
    +   * Matches host against the regexes and returns the matching pool names
    +   *
    +   * @param host
    +   *          tablet server host
    +   * @return pool names, will return default pool if host matches more no regex
    +   */
    +  protected List<String> getPoolNamesForHost(String host) {
    +    String test = host;
    +    if (!isIpBasedRegex) {
    +      try {
    +        test = getNameFromIp(host);
    +      } catch (UnknownHostException e1) {
    +        LOG.error("Unable to determine host name for IP: " + host + ", setting to default pool", e1);
    +        return Collections.singletonList(DEFAULT_POOL);
    +      }
    +    }
    +    List<String> pools = new ArrayList<>();
    +    for (Entry<String,Pattern> e : poolNameToRegexPattern.entrySet()) {
    +      if (e.getValue().matcher(test).matches()) {
    +        pools.add(e.getKey());
    +      }
    +    }
    +    if (pools.size() == 0) {
    +      pools.add(DEFAULT_POOL);
    +    }
    +    return pools;
    +  }
    +
    +  protected String getNameFromIp(String hostIp) throws UnknownHostException {
    +    return InetAddress.getByName(hostIp).getHostName();
    +  }
    +
    +  /**
    +   * Matches table name against pool names, returns matching pool name or DEFAULT_POOL.
    +   *
    +   * @param tableName
    +   *          name of table
    +   * @return tablet server pool name (table name or DEFAULT_POOL)
    +   */
    +  protected String getPoolNameForTable(String tableName) {
    +    if (null == tableName) {
    +      return DEFAULT_POOL;
    +    }
    +    return poolNameToRegexPattern.containsKey(tableName) ? tableName : DEFAULT_POOL;
    +  }
    +
    +  /**
    +   * Parse configuration and extract properties
    +   *
    +   * @param conf
    +   *          server configuration
    +   */
    +  protected void parseConfiguration(ServerConfiguration conf) {
    +    tableIdToTableName = new HashMap<>();
    +    poolNameToRegexPattern = new HashMap<>();
    +    for (Entry<String,String> table : getTableOperations().tableIdMap().entrySet()) {
    +      tableIdToTableName.put(table.getValue(), table.getKey());
    +      Map<String,String> customProps = conf.getTableConfiguration(table.getValue()).getAllPropertiesWithPrefix(Property.TABLE_ARBITRARY_PROP_PREFIX);
    +      if (null != customProps && customProps.size() > 0) {
    +        for (Entry<String,String> customProp : customProps.entrySet()) {
    +          String tableName = customProp.getKey().substring(HOST_BALANCER_PREFIX.length());
    +          String regex = customProp.getValue();
    +          poolNameToRegexPattern.put(tableName, Pattern.compile(regex));
    +        }
    +      }
    +    }
    +    String oobProperty = conf.getConfiguration().get(HOST_BALANCER_OOB_CHECK);
    +    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);
    +    if (null != ipBased) {
    +      isIpBasedRegex = Boolean.parseBoolean(ipBased);
    +    }
    +  }
    +
    +  public Map<String,String> getTableIdToTableName() {
    +    return tableIdToTableName;
    +  }
    +
    +  public Map<String,Pattern> getPoolNameToRegexPattern() {
    +    return poolNameToRegexPattern;
    +  }
    +
    +  public long getOobCheckMillis() {
    +    return oobCheckMillis;
    +  }
    +
    +  public long getPoolRecheckMillis() {
    +    return poolRecheckMillis;
    +  }
    +
    +  public boolean isIpBasedRegex() {
    +    return isIpBasedRegex;
    +  }
    +
    +  @Override
    +  public void init(ServerConfiguration conf) {
    +    super.init(conf);
    +    parseConfiguration(conf);
    +  }
    +
    +  @Override
    +  public void getAssignments(SortedMap<TServerInstance,TabletServerStatus> current, Map<KeyExtent,TServerInstance> unassigned,
    +      Map<KeyExtent,TServerInstance> assignments) {
    +
    +    if ((System.currentTimeMillis() - this.lastOOBCheck) > this.oobCheckMillis) {
    +      // 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 assignedPool : getPoolNamesForHost(e.getKey().host())) {
    +          for (String pool : poolNameToRegexPattern.keySet()) {
    +            if (assignedPool.equals(pool) || pool.equals(DEFAULT_POOL)) {
    +              continue;
    +            }
    +            String tid = getTableOperations().tableIdMap().get(pool);
    +            try {
    +              List<TabletStats> outOfBoundsTablets = getOnlineTabletsForTable(e.getKey(), tid);
    +              for (TabletStats ts : outOfBoundsTablets) {
    +                LOG.info("Tablet {} is currently outside the bounds of the regex, reassigning", ts.toString());
    +                unassigned.put(new KeyExtent(ts.getExtent()), e.getKey());
    +              }
    +            } catch (TException e1) {
    +              LOG.error("Error in OOB check getting tablets for table {} from server {}", tid, e.getKey().host(), e);
    +            }
    +          }
    +        }
    +      }
    +      this.oobCheckMillis = System.currentTimeMillis();
    +    }
    +
    +    Map<String,SortedMap<TServerInstance,TabletServerStatus>> pools = splitCurrentByRegex(current);
    +    // separate the unassigned into tables
    +    Map<String,Map<KeyExtent,TServerInstance>> groupedUnassigned = new HashMap<String,Map<KeyExtent,TServerInstance>>();
    +    for (Entry<KeyExtent,TServerInstance> e : unassigned.entrySet()) {
    +      Map<KeyExtent,TServerInstance> tableUnassigned = groupedUnassigned.get(e.getKey().getTableId());
    +      if (tableUnassigned == null) {
    +        tableUnassigned = new HashMap<KeyExtent,TServerInstance>();
    +        groupedUnassigned.put(e.getKey().getTableId(), tableUnassigned);
    +      }
    +      tableUnassigned.put(e.getKey(), e.getValue());
    +    }
    +    // Send a view of the current servers to the tables tablet balancer
    +    for (Entry<String,Map<KeyExtent,TServerInstance>> e : groupedUnassigned.entrySet()) {
    +      Map<KeyExtent,TServerInstance> newAssignments = new HashMap<KeyExtent,TServerInstance>();
    +      String tableName = tableIdToTableName.get(e.getKey());
    +      String poolName = getPoolNameForTable(tableName);
    +      SortedMap<TServerInstance,TabletServerStatus> currentView = pools.get(poolName);
    +      getBalancerForTable(e.getKey()).getAssignments(currentView, e.getValue(), newAssignments);
    --- End diff --
    
    I think I have addressed this now, see above lines 263-278


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request: ACCUMULO-4173: Host Regex Table Load Balanc...

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/83#discussion_r59391317
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancer.java ---
    @@ -0,0 +1,334 @@
    +/*
    + * 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.net.InetAddress;
    +import java.net.UnknownHostException;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Set;
    +import java.util.SortedMap;
    +import java.util.TreeMap;
    +import java.util.regex.Pattern;
    +
    +import org.apache.accumulo.core.client.admin.TableOperations;
    +import org.apache.accumulo.core.conf.AccumuloConfiguration;
    +import org.apache.accumulo.core.conf.Property;
    +import org.apache.accumulo.core.data.impl.KeyExtent;
    +import org.apache.accumulo.core.master.thrift.TabletServerStatus;
    +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.thrift.TException;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * This balancer will create pools of tablet servers by grouping tablet servers that match a regex into the same pool and calling the balancer set on the table
    + * to balance within the set of matching tablet servers. All tablet servers that do not match a regex are grouped into a default pool.<br>
    + * Regex properties for this balancer are specified as:<br>
    + * <b>table.custom.balancer.host.regex.&lt;tablename&gt;=&lt;regex&gt;</b><br>
    + * Periodically (default 5m) this balancer will check to see if a tablet server is hosting tablets that it should not be according to the regex configuration.
    + * 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 5m) 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>
    + * 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>
    + *
    + */
    +public class HostRegexTableLoadBalancer extends TableLoadBalancer {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(HostRegexTableLoadBalancer.class);
    +  public static final String HOST_BALANCER_PREFIX = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.";
    +  public static final String HOST_BALANCER_OOB_CHECK = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.oob.period";
    +  private static final String HOST_BALANCER_OOB_DEFAULT = "5m";
    +  public static final String HOST_BALANCER_POOL_RECHECK_KEY = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.pool.check";
    +  private static final String HOST_BALANCER_POOL_RECHECK_DEFAULT = "5m";
    +  public static final String HOST_BALANCER_REGEX_USING_IPS = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.is.ip";
    +  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<String,SortedMap<TServerInstance,TabletServerStatus>>();
    +
    +  /**
    +   * Group the set of current tservers by pool name. Tservers that don't match a regex are put into a default pool.
    +   *
    +   * @param current
    +   *          map of current tservers
    +   * @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 into pools based on regex");
    +      Map<String,SortedMap<TServerInstance,TabletServerStatus>> newPools = new HashMap<String,SortedMap<TServerInstance,TabletServerStatus>>();
    +      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<TServerInstance,TabletServerStatus>(current.comparator());
    +            newPools.put(pool, np);
    +          }
    +          np.put(e.getKey(), e.getValue());
    +        }
    +      }
    +      pools = newPools;
    +      this.lastPoolRecheck = System.currentTimeMillis();
    +    }
    +    return pools;
    +  }
    +
    +  /**
    +   * Matches host against the regexes and returns the matching pool names
    +   *
    +   * @param host
    +   *          tablet server host
    +   * @return pool names, will return default pool if host matches more no regex
    +   */
    +  protected List<String> getPoolNamesForHost(String host) {
    +    String test = host;
    +    if (!isIpBasedRegex) {
    +      try {
    +        test = getNameFromIp(host);
    +      } catch (UnknownHostException e1) {
    +        LOG.error("Unable to determine host name for IP: " + host + ", setting to default pool", e1);
    +        return Collections.singletonList(DEFAULT_POOL);
    +      }
    +    }
    +    List<String> pools = new ArrayList<>();
    +    for (Entry<String,Pattern> e : poolNameToRegexPattern.entrySet()) {
    +      if (e.getValue().matcher(test).matches()) {
    +        pools.add(e.getKey());
    +      }
    +    }
    +    if (pools.size() == 0) {
    +      pools.add(DEFAULT_POOL);
    +    }
    +    return pools;
    +  }
    +
    +  protected String getNameFromIp(String hostIp) throws UnknownHostException {
    +    return InetAddress.getByName(hostIp).getHostName();
    +  }
    +
    +  /**
    +   * Matches table name against pool names, returns matching pool name or DEFAULT_POOL.
    +   *
    +   * @param tableName
    +   *          name of table
    +   * @return tablet server pool name (table name or DEFAULT_POOL)
    +   */
    +  protected String getPoolNameForTable(String tableName) {
    +    if (null == tableName) {
    +      return DEFAULT_POOL;
    +    }
    +    return poolNameToRegexPattern.containsKey(tableName) ? tableName : DEFAULT_POOL;
    +  }
    +
    +  /**
    +   * Parse configuration and extract properties
    +   *
    +   * @param conf
    +   *          server configuration
    +   */
    +  protected void parseConfiguration(ServerConfiguration conf) {
    +    tableIdToTableName = new HashMap<>();
    +    poolNameToRegexPattern = new HashMap<>();
    +    for (Entry<String,String> table : getTableOperations().tableIdMap().entrySet()) {
    +      tableIdToTableName.put(table.getValue(), table.getKey());
    +      Map<String,String> customProps = conf.getTableConfiguration(table.getValue()).getAllPropertiesWithPrefix(Property.TABLE_ARBITRARY_PROP_PREFIX);
    +      if (null != customProps && customProps.size() > 0) {
    +        for (Entry<String,String> customProp : customProps.entrySet()) {
    +          if (customProp.getKey().startsWith(HOST_BALANCER_PREFIX)) {
    +            String tableName = customProp.getKey().substring(HOST_BALANCER_PREFIX.length());
    +            String regex = customProp.getValue();
    +            poolNameToRegexPattern.put(tableName, Pattern.compile(regex));
    +          }
    +        }
    +      }
    +    }
    +    String oobProperty = conf.getConfiguration().get(HOST_BALANCER_OOB_CHECK);
    +    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);
    +    if (null != ipBased) {
    +      isIpBasedRegex = Boolean.parseBoolean(ipBased);
    +    }
    +  }
    +
    +  public Map<String,String> getTableIdToTableName() {
    +    return tableIdToTableName;
    +  }
    +
    +  public Map<String,Pattern> getPoolNameToRegexPattern() {
    +    return poolNameToRegexPattern;
    +  }
    +
    +  public long getOobCheckMillis() {
    +    return oobCheckMillis;
    +  }
    +
    +  public long getPoolRecheckMillis() {
    +    return poolRecheckMillis;
    +  }
    +
    +  public boolean isIpBasedRegex() {
    +    return isIpBasedRegex;
    +  }
    +
    +  @Override
    +  public void init(ServerConfiguration conf) {
    +    super.init(conf);
    +    parseConfiguration(conf);
    +  }
    +
    +  @Override
    +  public void getAssignments(SortedMap<TServerInstance,TabletServerStatus> current, Map<KeyExtent,TServerInstance> unassigned,
    +      Map<KeyExtent,TServerInstance> assignments) {
    +
    +    Map<String,SortedMap<TServerInstance,TabletServerStatus>> pools = splitCurrentByRegex(current);
    --- End diff --
    
    5 minutes seems high as a default then. Considering the average person will have 10's of servers, *maybe* 100's, waiting 5 minutes by default seems extraordinary. Hypothetically, grouping 100 hostnames every minute seems fine to me (esp in the master). Maybe the scope of the feature itself isn't a worry for people with 1s-10s of servers? WDYT?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request: ACCUMULO-4173: Host Regex Table Load Balanc...

Posted by dlmarion <gi...@git.apache.org>.
Github user dlmarion commented on the pull request:

    https://github.com/apache/accumulo/pull/83#issuecomment-203039636
  
    Jenkins failure is not due to the patch.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request: ACCUMULO-4173: Host Regex Table Load Balanc...

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/83#discussion_r59392771
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancer.java ---
    @@ -0,0 +1,334 @@
    +/*
    + * 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.net.InetAddress;
    +import java.net.UnknownHostException;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Set;
    +import java.util.SortedMap;
    +import java.util.TreeMap;
    +import java.util.regex.Pattern;
    +
    +import org.apache.accumulo.core.client.admin.TableOperations;
    +import org.apache.accumulo.core.conf.AccumuloConfiguration;
    +import org.apache.accumulo.core.conf.Property;
    +import org.apache.accumulo.core.data.impl.KeyExtent;
    +import org.apache.accumulo.core.master.thrift.TabletServerStatus;
    +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.thrift.TException;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * This balancer will create pools of tablet servers by grouping tablet servers that match a regex into the same pool and calling the balancer set on the table
    + * to balance within the set of matching tablet servers. All tablet servers that do not match a regex are grouped into a default pool.<br>
    + * Regex properties for this balancer are specified as:<br>
    + * <b>table.custom.balancer.host.regex.&lt;tablename&gt;=&lt;regex&gt;</b><br>
    + * Periodically (default 5m) this balancer will check to see if a tablet server is hosting tablets that it should not be according to the regex configuration.
    + * 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 5m) 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>
    + * 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>
    + *
    + */
    +public class HostRegexTableLoadBalancer extends TableLoadBalancer {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(HostRegexTableLoadBalancer.class);
    +  public static final String HOST_BALANCER_PREFIX = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.";
    +  public static final String HOST_BALANCER_OOB_CHECK = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.oob.period";
    +  private static final String HOST_BALANCER_OOB_DEFAULT = "5m";
    +  public static final String HOST_BALANCER_POOL_RECHECK_KEY = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.pool.check";
    +  private static final String HOST_BALANCER_POOL_RECHECK_DEFAULT = "5m";
    +  public static final String HOST_BALANCER_REGEX_USING_IPS = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.is.ip";
    +  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<String,SortedMap<TServerInstance,TabletServerStatus>>();
    +
    +  /**
    +   * Group the set of current tservers by pool name. Tservers that don't match a regex are put into a default pool.
    +   *
    +   * @param current
    +   *          map of current tservers
    +   * @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 into pools based on regex");
    +      Map<String,SortedMap<TServerInstance,TabletServerStatus>> newPools = new HashMap<String,SortedMap<TServerInstance,TabletServerStatus>>();
    +      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<TServerInstance,TabletServerStatus>(current.comparator());
    +            newPools.put(pool, np);
    +          }
    +          np.put(e.getKey(), e.getValue());
    +        }
    +      }
    +      pools = newPools;
    +      this.lastPoolRecheck = System.currentTimeMillis();
    +    }
    +    return pools;
    +  }
    +
    +  /**
    +   * Matches host against the regexes and returns the matching pool names
    +   *
    +   * @param host
    +   *          tablet server host
    +   * @return pool names, will return default pool if host matches more no regex
    +   */
    +  protected List<String> getPoolNamesForHost(String host) {
    +    String test = host;
    +    if (!isIpBasedRegex) {
    +      try {
    +        test = getNameFromIp(host);
    +      } catch (UnknownHostException e1) {
    +        LOG.error("Unable to determine host name for IP: " + host + ", setting to default pool", e1);
    +        return Collections.singletonList(DEFAULT_POOL);
    +      }
    +    }
    +    List<String> pools = new ArrayList<>();
    +    for (Entry<String,Pattern> e : poolNameToRegexPattern.entrySet()) {
    +      if (e.getValue().matcher(test).matches()) {
    +        pools.add(e.getKey());
    +      }
    +    }
    +    if (pools.size() == 0) {
    +      pools.add(DEFAULT_POOL);
    +    }
    +    return pools;
    +  }
    +
    +  protected String getNameFromIp(String hostIp) throws UnknownHostException {
    +    return InetAddress.getByName(hostIp).getHostName();
    +  }
    +
    +  /**
    +   * Matches table name against pool names, returns matching pool name or DEFAULT_POOL.
    +   *
    +   * @param tableName
    +   *          name of table
    +   * @return tablet server pool name (table name or DEFAULT_POOL)
    +   */
    +  protected String getPoolNameForTable(String tableName) {
    +    if (null == tableName) {
    +      return DEFAULT_POOL;
    +    }
    +    return poolNameToRegexPattern.containsKey(tableName) ? tableName : DEFAULT_POOL;
    +  }
    +
    +  /**
    +   * Parse configuration and extract properties
    +   *
    +   * @param conf
    +   *          server configuration
    +   */
    +  protected void parseConfiguration(ServerConfiguration conf) {
    +    tableIdToTableName = new HashMap<>();
    +    poolNameToRegexPattern = new HashMap<>();
    +    for (Entry<String,String> table : getTableOperations().tableIdMap().entrySet()) {
    +      tableIdToTableName.put(table.getValue(), table.getKey());
    +      Map<String,String> customProps = conf.getTableConfiguration(table.getValue()).getAllPropertiesWithPrefix(Property.TABLE_ARBITRARY_PROP_PREFIX);
    +      if (null != customProps && customProps.size() > 0) {
    +        for (Entry<String,String> customProp : customProps.entrySet()) {
    +          if (customProp.getKey().startsWith(HOST_BALANCER_PREFIX)) {
    +            String tableName = customProp.getKey().substring(HOST_BALANCER_PREFIX.length());
    +            String regex = customProp.getValue();
    +            poolNameToRegexPattern.put(tableName, Pattern.compile(regex));
    +          }
    +        }
    +      }
    +    }
    +    String oobProperty = conf.getConfiguration().get(HOST_BALANCER_OOB_CHECK);
    +    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);
    +    if (null != ipBased) {
    +      isIpBasedRegex = Boolean.parseBoolean(ipBased);
    +    }
    +  }
    +
    +  public Map<String,String> getTableIdToTableName() {
    +    return tableIdToTableName;
    +  }
    +
    +  public Map<String,Pattern> getPoolNameToRegexPattern() {
    +    return poolNameToRegexPattern;
    +  }
    +
    +  public long getOobCheckMillis() {
    +    return oobCheckMillis;
    +  }
    +
    +  public long getPoolRecheckMillis() {
    +    return poolRecheckMillis;
    +  }
    +
    +  public boolean isIpBasedRegex() {
    +    return isIpBasedRegex;
    +  }
    +
    +  @Override
    +  public void init(ServerConfiguration conf) {
    +    super.init(conf);
    +    parseConfiguration(conf);
    +  }
    +
    +  @Override
    +  public void getAssignments(SortedMap<TServerInstance,TabletServerStatus> current, Map<KeyExtent,TServerInstance> unassigned,
    +      Map<KeyExtent,TServerInstance> assignments) {
    +
    +    Map<String,SortedMap<TServerInstance,TabletServerStatus>> pools = splitCurrentByRegex(current);
    --- End diff --
    
    Cool, thanks Dave.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request: ACCUMULO-4173: Host Regex Table Load Balanc...

Posted by brianloss <gi...@git.apache.org>.
Github user brianloss commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/83#discussion_r57931945
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancer.java ---
    @@ -0,0 +1,314 @@
    +/*
    + * 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.net.InetAddress;
    +import java.net.UnknownHostException;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Set;
    +import java.util.SortedMap;
    +import java.util.TreeMap;
    +import java.util.regex.Pattern;
    +
    +import org.apache.accumulo.core.client.admin.TableOperations;
    +import org.apache.accumulo.core.conf.AccumuloConfiguration;
    +import org.apache.accumulo.core.conf.Property;
    +import org.apache.accumulo.core.data.impl.KeyExtent;
    +import org.apache.accumulo.core.master.thrift.TabletServerStatus;
    +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.thrift.TException;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * This balancer will create pools of tablet servers by grouping tablet servers that match a regex into the same pool and calling the balancer set on the table
    + * to balance within the set of matching tablet servers. <br>
    + * Regex properties for this balancer are specified as:<br>
    + * <b>table.custom.balancer.host.regex.&lt;tablename&gt;=&lt;regex&gt;</b><br>
    + * Periodically (default 5m) this balancer will check to see if a tablet server is hosting tablets that it should not be according to the regex configuration.
    + * 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 5m) 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>
    + * 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>
    + *
    + */
    +public class HostRegexTableLoadBalancer extends TableLoadBalancer {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(HostRegexTableLoadBalancer.class);
    +  public static final String HOST_BALANCER_PREFIX = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.";
    +  public static final String HOST_BALANCER_OOB_CHECK = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.oob.period";
    +  private static final String HOST_BALANCER_OOB_DEFAULT = "5m";
    +  public static final String HOST_BALANCER_POOL_RECHECK_KEY = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.pool.check";
    +  private static final String HOST_BALANCER_POOL_RECHECK_DEFAULT = "5m";
    +  public static final String HOST_BALANCER_REGEX_USING_IPS = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.is.ip";
    +  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 long lastOOBCheck = System.currentTimeMillis();
    +  private long lastPoolRecheck = 0;
    +  private boolean isIpBasedRegex = false;
    +  private Map<String,SortedMap<TServerInstance,TabletServerStatus>> pools = new HashMap<String,SortedMap<TServerInstance,TabletServerStatus>>();
    +
    +  /**
    +   * Group the set of current tservers by pool name. Tservers that don't match a regex are put into a default ppol.
    +   *
    +   * @param current
    +   *          map of current tservers
    +   * @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) {
    +      Map<String,SortedMap<TServerInstance,TabletServerStatus>> newPools = new HashMap<String,SortedMap<TServerInstance,TabletServerStatus>>();
    +      for (Entry<TServerInstance,TabletServerStatus> e : current.entrySet()) {
    +        String tableName = getPoolNameForHost(e.getKey().host());
    +        if (!newPools.containsKey(tableName)) {
    +          newPools.put(tableName, new TreeMap<TServerInstance,TabletServerStatus>());
    +        }
    +        newPools.get(tableName).put(e.getKey(), e.getValue());
    +      }
    +      // Ensure that no host is in more than one pool
    +      // TODO: I'm not sure that I need to check for disjoint as the call to getPoolNameForHost checks for more than one match
    +      boolean error = false;
    +      for (SortedMap<TServerInstance,TabletServerStatus> s1 : newPools.values()) {
    +        for (SortedMap<TServerInstance,TabletServerStatus> s2 : newPools.values()) {
    +          if (s1 == s2) {
    +            continue;
    +          }
    +          if (!Collections.disjoint(s1.keySet(), s2.keySet())) {
    +            LOG.error("Pools are not disjoint: {}, there is a problem with your regular expressions. Putting all servers in default pool", newPools);
    +            error = true;
    +          }
    +        }
    +      }
    +      if (error) {
    +        // Put all servers into the default pool
    +        newPools.clear();
    +        newPools.put(DEFAULT_POOL, new TreeMap<TServerInstance,TabletServerStatus>());
    +        for (Entry<TServerInstance,TabletServerStatus> e : current.entrySet()) {
    +          newPools.get(DEFAULT_POOL).put(e.getKey(), e.getValue());
    +        }
    +      }
    +      pools = newPools;
    +      this.lastPoolRecheck = System.currentTimeMillis();
    +    }
    +    return pools;
    +  }
    +
    +  /**
    +   * Matches host against the regexes and returns the matching pool name
    +   *
    +   * @param host
    +   *          tablet server host
    +   * @return name of pool. will return default pool if host matches more than one regex
    +   */
    +  protected String getPoolNameForHost(String host) {
    +    String test = host;
    +    String table = DEFAULT_POOL;
    +    if (!isIpBasedRegex) {
    +      try {
    +        test = getNameFromIp(host);
    +      } catch (UnknownHostException e1) {
    +        LOG.error("Unable to determine host name for IP: " + host + ", setting to default pool", e1);
    +        return table;
    +      }
    +    }
    +    for (Entry<String,Pattern> e : poolNameToRegexPattern.entrySet()) {
    +      if (e.getValue().matcher(test).matches()) {
    +        if (!table.equals(DEFAULT_POOL)) {
    +          LOG.warn("host {} matches more than one regex, assigning to default pool", host);
    +          return DEFAULT_POOL;
    +        }
    +        table = e.getKey();
    +      }
    +    }
    +    return table;
    +  }
    +
    +  protected String getNameFromIp(String hostIp) throws UnknownHostException {
    +    return InetAddress.getByName(hostIp).getHostName();
    +  }
    +
    +  /**
    +   * Matches table name against pool names, returns matching pool name or DEFAULT_POOL.
    +   *
    +   * @param tableName
    +   *          name of table
    +   * @return tablet server pool name (table name or DEFAULT_POOL)
    +   */
    +  protected String getPoolNameForTable(String tableName) {
    +    if (null == tableName) {
    +      return DEFAULT_POOL;
    +    }
    +    return poolNameToRegexPattern.containsKey(tableName) ? tableName : DEFAULT_POOL;
    +  }
    +
    +  /**
    +   * Parse configuration and extract properties
    +   *
    +   * @param conf
    +   *          server configuration
    +   */
    +  protected void parseConfiguration(ServerConfiguration conf) {
    +    tableIdToTableName = new HashMap<>();
    +    poolNameToRegexPattern = new HashMap<>();
    +    for (Entry<String,String> table : getTableOperations().tableIdMap().entrySet()) {
    +      tableIdToTableName.put(table.getValue(), table.getKey());
    +      Map<String,String> customProps = conf.getTableConfiguration(table.getValue()).getAllPropertiesWithPrefix(Property.TABLE_ARBITRARY_PROP_PREFIX);
    +      if (null != customProps && customProps.size() > 0) {
    +        for (Entry<String,String> customProp : customProps.entrySet()) {
    +          String tableName = customProp.getKey().substring(HOST_BALANCER_PREFIX.length());
    +          String regex = customProp.getValue();
    +          poolNameToRegexPattern.put(tableName, Pattern.compile(regex));
    +        }
    +      }
    +    }
    +    String oobProperty = conf.getConfiguration().get(HOST_BALANCER_OOB_CHECK);
    +    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);
    +    if (null != ipBased) {
    +      isIpBasedRegex = Boolean.parseBoolean(ipBased);
    +    }
    +  }
    +
    +  public Map<String,String> getTableIdToTableName() {
    +    return tableIdToTableName;
    +  }
    +
    +  public Map<String,Pattern> getPoolNameToRegexPattern() {
    +    return poolNameToRegexPattern;
    +  }
    +
    +  public long getOobCheckMillis() {
    +    return oobCheckMillis;
    +  }
    +
    +  public long getPoolRecheckMillis() {
    +    return poolRecheckMillis;
    +  }
    +
    +  public boolean isIpBasedRegex() {
    +    return isIpBasedRegex;
    +  }
    +
    +  @Override
    +  public void init(ServerConfiguration conf) {
    +    super.init(conf);
    +    parseConfiguration(conf);
    +  }
    +
    +  @Override
    +  public void getAssignments(SortedMap<TServerInstance,TabletServerStatus> current, Map<KeyExtent,TServerInstance> unassigned,
    +      Map<KeyExtent,TServerInstance> assignments) {
    +
    +    if ((System.currentTimeMillis() - this.lastOOBCheck) > this.oobCheckMillis) {
    +      // 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()) {
    +        String assignedPool = getPoolNameForHost(e.getKey().host());
    +        for (String pool : poolNameToRegexPattern.keySet()) {
    +          if (assignedPool.equals(pool) || pool.equals(DEFAULT_POOL)) {
    +            continue;
    +          }
    +          String tid = getTableOperations().tableIdMap().get(pool);
    --- End diff --
    
    If a table is renamed, this will likely cause problems.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request: ACCUMULO-4173: Host Regex Table Load Balanc...

Posted by dlmarion <gi...@git.apache.org>.
Github user dlmarion commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/83#discussion_r59393062
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancer.java ---
    @@ -0,0 +1,314 @@
    +/*
    + * 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.net.InetAddress;
    +import java.net.UnknownHostException;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Set;
    +import java.util.SortedMap;
    +import java.util.TreeMap;
    +import java.util.regex.Pattern;
    +
    +import org.apache.accumulo.core.client.admin.TableOperations;
    +import org.apache.accumulo.core.conf.AccumuloConfiguration;
    +import org.apache.accumulo.core.conf.Property;
    +import org.apache.accumulo.core.data.impl.KeyExtent;
    +import org.apache.accumulo.core.master.thrift.TabletServerStatus;
    +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.thrift.TException;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * This balancer will create pools of tablet servers by grouping tablet servers that match a regex into the same pool and calling the balancer set on the table
    + * to balance within the set of matching tablet servers. <br>
    + * Regex properties for this balancer are specified as:<br>
    + * <b>table.custom.balancer.host.regex.&lt;tablename&gt;=&lt;regex&gt;</b><br>
    + * Periodically (default 5m) this balancer will check to see if a tablet server is hosting tablets that it should not be according to the regex configuration.
    + * 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 5m) 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>
    + * 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>
    + *
    + */
    +public class HostRegexTableLoadBalancer extends TableLoadBalancer {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(HostRegexTableLoadBalancer.class);
    +  public static final String HOST_BALANCER_PREFIX = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.";
    +  public static final String HOST_BALANCER_OOB_CHECK = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.oob.period";
    +  private static final String HOST_BALANCER_OOB_DEFAULT = "5m";
    +  public static final String HOST_BALANCER_POOL_RECHECK_KEY = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.pool.check";
    +  private static final String HOST_BALANCER_POOL_RECHECK_DEFAULT = "5m";
    +  public static final String HOST_BALANCER_REGEX_USING_IPS = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.is.ip";
    +  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 long lastOOBCheck = System.currentTimeMillis();
    +  private long lastPoolRecheck = 0;
    +  private boolean isIpBasedRegex = false;
    +  private Map<String,SortedMap<TServerInstance,TabletServerStatus>> pools = new HashMap<String,SortedMap<TServerInstance,TabletServerStatus>>();
    +
    +  /**
    +   * Group the set of current tservers by pool name. Tservers that don't match a regex are put into a default ppol.
    +   *
    +   * @param current
    +   *          map of current tservers
    +   * @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) {
    +      Map<String,SortedMap<TServerInstance,TabletServerStatus>> newPools = new HashMap<String,SortedMap<TServerInstance,TabletServerStatus>>();
    +      for (Entry<TServerInstance,TabletServerStatus> e : current.entrySet()) {
    +        String tableName = getPoolNameForHost(e.getKey().host());
    +        if (!newPools.containsKey(tableName)) {
    +          newPools.put(tableName, new TreeMap<TServerInstance,TabletServerStatus>());
    +        }
    +        newPools.get(tableName).put(e.getKey(), e.getValue());
    +      }
    +      // Ensure that no host is in more than one pool
    +      // TODO: I'm not sure that I need to check for disjoint as the call to getPoolNameForHost checks for more than one match
    +      boolean error = false;
    +      for (SortedMap<TServerInstance,TabletServerStatus> s1 : newPools.values()) {
    +        for (SortedMap<TServerInstance,TabletServerStatus> s2 : newPools.values()) {
    +          if (s1 == s2) {
    +            continue;
    +          }
    +          if (!Collections.disjoint(s1.keySet(), s2.keySet())) {
    +            LOG.error("Pools are not disjoint: {}, there is a problem with your regular expressions. Putting all servers in default pool", newPools);
    +            error = true;
    +          }
    +        }
    +      }
    +      if (error) {
    +        // Put all servers into the default pool
    +        newPools.clear();
    +        newPools.put(DEFAULT_POOL, new TreeMap<TServerInstance,TabletServerStatus>());
    +        for (Entry<TServerInstance,TabletServerStatus> e : current.entrySet()) {
    +          newPools.get(DEFAULT_POOL).put(e.getKey(), e.getValue());
    +        }
    +      }
    +      pools = newPools;
    +      this.lastPoolRecheck = System.currentTimeMillis();
    +    }
    +    return pools;
    +  }
    +
    +  /**
    +   * Matches host against the regexes and returns the matching pool name
    +   *
    +   * @param host
    +   *          tablet server host
    +   * @return name of pool. will return default pool if host matches more than one regex
    +   */
    +  protected String getPoolNameForHost(String host) {
    +    String test = host;
    +    String table = DEFAULT_POOL;
    +    if (!isIpBasedRegex) {
    +      try {
    +        test = getNameFromIp(host);
    +      } catch (UnknownHostException e1) {
    +        LOG.error("Unable to determine host name for IP: " + host + ", setting to default pool", e1);
    +        return table;
    +      }
    +    }
    +    for (Entry<String,Pattern> e : poolNameToRegexPattern.entrySet()) {
    +      if (e.getValue().matcher(test).matches()) {
    +        if (!table.equals(DEFAULT_POOL)) {
    +          LOG.warn("host {} matches more than one regex, assigning to default pool", host);
    +          return DEFAULT_POOL;
    +        }
    +        table = e.getKey();
    +      }
    +    }
    +    return table;
    +  }
    +
    +  protected String getNameFromIp(String hostIp) throws UnknownHostException {
    +    return InetAddress.getByName(hostIp).getHostName();
    +  }
    +
    +  /**
    +   * Matches table name against pool names, returns matching pool name or DEFAULT_POOL.
    +   *
    +   * @param tableName
    +   *          name of table
    +   * @return tablet server pool name (table name or DEFAULT_POOL)
    +   */
    +  protected String getPoolNameForTable(String tableName) {
    +    if (null == tableName) {
    +      return DEFAULT_POOL;
    +    }
    +    return poolNameToRegexPattern.containsKey(tableName) ? tableName : DEFAULT_POOL;
    +  }
    +
    +  /**
    +   * Parse configuration and extract properties
    +   *
    +   * @param conf
    +   *          server configuration
    +   */
    +  protected void parseConfiguration(ServerConfiguration conf) {
    +    tableIdToTableName = new HashMap<>();
    +    poolNameToRegexPattern = new HashMap<>();
    +    for (Entry<String,String> table : getTableOperations().tableIdMap().entrySet()) {
    +      tableIdToTableName.put(table.getValue(), table.getKey());
    +      Map<String,String> customProps = conf.getTableConfiguration(table.getValue()).getAllPropertiesWithPrefix(Property.TABLE_ARBITRARY_PROP_PREFIX);
    +      if (null != customProps && customProps.size() > 0) {
    +        for (Entry<String,String> customProp : customProps.entrySet()) {
    +          String tableName = customProp.getKey().substring(HOST_BALANCER_PREFIX.length());
    +          String regex = customProp.getValue();
    +          poolNameToRegexPattern.put(tableName, Pattern.compile(regex));
    +        }
    +      }
    +    }
    +    String oobProperty = conf.getConfiguration().get(HOST_BALANCER_OOB_CHECK);
    +    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);
    +    if (null != ipBased) {
    +      isIpBasedRegex = Boolean.parseBoolean(ipBased);
    +    }
    +  }
    +
    +  public Map<String,String> getTableIdToTableName() {
    +    return tableIdToTableName;
    +  }
    +
    +  public Map<String,Pattern> getPoolNameToRegexPattern() {
    +    return poolNameToRegexPattern;
    +  }
    +
    +  public long getOobCheckMillis() {
    +    return oobCheckMillis;
    +  }
    +
    +  public long getPoolRecheckMillis() {
    +    return poolRecheckMillis;
    +  }
    +
    +  public boolean isIpBasedRegex() {
    +    return isIpBasedRegex;
    +  }
    +
    +  @Override
    +  public void init(ServerConfiguration conf) {
    +    super.init(conf);
    +    parseConfiguration(conf);
    +  }
    +
    +  @Override
    +  public void getAssignments(SortedMap<TServerInstance,TabletServerStatus> current, Map<KeyExtent,TServerInstance> unassigned,
    +      Map<KeyExtent,TServerInstance> assignments) {
    +
    +    if ((System.currentTimeMillis() - this.lastOOBCheck) > this.oobCheckMillis) {
    +      // 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()) {
    +        String assignedPool = getPoolNameForHost(e.getKey().host());
    +        for (String pool : poolNameToRegexPattern.keySet()) {
    +          if (assignedPool.equals(pool) || pool.equals(DEFAULT_POOL)) {
    +            continue;
    +          }
    +          String tid = getTableOperations().tableIdMap().get(pool);
    +          try {
    +            List<TabletStats> outOfBoundsTablets = getOnlineTabletsForTable(e.getKey(), tid);
    +            for (TabletStats ts : outOfBoundsTablets) {
    +              LOG.info("Tablet {} is currently outside the bounds of the regex, reassigning", ts.toString());
    +              unassigned.put(new KeyExtent(ts.getExtent()), e.getKey());
    +            }
    +          } catch (TException e1) {
    +            LOG.error("Error in OOB check getting tablets for table {} from server {}", tid, e.getKey().host(), e);
    +          }
    +        }
    +      }
    +      this.oobCheckMillis = System.currentTimeMillis();
    +    }
    +
    +    Map<String,SortedMap<TServerInstance,TabletServerStatus>> pools = splitCurrentByRegex(current);
    +    // separate the unassigned into tables
    +    Map<String,Map<KeyExtent,TServerInstance>> groupedUnassigned = new HashMap<String,Map<KeyExtent,TServerInstance>>();
    +    for (Entry<KeyExtent,TServerInstance> e : unassigned.entrySet()) {
    +      Map<KeyExtent,TServerInstance> tableUnassigned = groupedUnassigned.get(e.getKey().getTableId());
    +      if (tableUnassigned == null) {
    +        tableUnassigned = new HashMap<KeyExtent,TServerInstance>();
    +        groupedUnassigned.put(e.getKey().getTableId(), tableUnassigned);
    +      }
    +      tableUnassigned.put(e.getKey(), e.getValue());
    +    }
    +    // Send a view of the current servers to the tables tablet balancer
    +    for (Entry<String,Map<KeyExtent,TServerInstance>> e : groupedUnassigned.entrySet()) {
    +      Map<KeyExtent,TServerInstance> newAssignments = new HashMap<KeyExtent,TServerInstance>();
    +      String tableName = tableIdToTableName.get(e.getKey());
    +      String poolName = getPoolNameForTable(tableName);
    +      SortedMap<TServerInstance,TabletServerStatus> currentView = pools.get(poolName);
    +      getBalancerForTable(e.getKey()).getAssignments(currentView, e.getValue(), newAssignments);
    +      assignments.putAll(newAssignments);
    +    }
    +  }
    +
    +  @Override
    +  public long balance(SortedMap<TServerInstance,TabletServerStatus> current, Set<KeyExtent> migrations, List<TabletMigration> migrationsOut) {
    +    long minBalanceTime = 5 * 1000;
    +    // Iterate over the tables and balance each of them
    +    TableOperations t = getTableOperations();
    +    if (t == null)
    --- End diff --
    
    Fixed in other location


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request: ACCUMULO-4173: Host Regex Table Load Balanc...

Posted by brianloss <gi...@git.apache.org>.
Github user brianloss commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/83#discussion_r57929241
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancer.java ---
    @@ -0,0 +1,314 @@
    +/*
    + * 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.net.InetAddress;
    +import java.net.UnknownHostException;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Set;
    +import java.util.SortedMap;
    +import java.util.TreeMap;
    +import java.util.regex.Pattern;
    +
    +import org.apache.accumulo.core.client.admin.TableOperations;
    +import org.apache.accumulo.core.conf.AccumuloConfiguration;
    +import org.apache.accumulo.core.conf.Property;
    +import org.apache.accumulo.core.data.impl.KeyExtent;
    +import org.apache.accumulo.core.master.thrift.TabletServerStatus;
    +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.thrift.TException;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * This balancer will create pools of tablet servers by grouping tablet servers that match a regex into the same pool and calling the balancer set on the table
    + * to balance within the set of matching tablet servers. <br>
    + * Regex properties for this balancer are specified as:<br>
    + * <b>table.custom.balancer.host.regex.&lt;tablename&gt;=&lt;regex&gt;</b><br>
    + * Periodically (default 5m) this balancer will check to see if a tablet server is hosting tablets that it should not be according to the regex configuration.
    + * 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 5m) 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>
    + * 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>
    + *
    + */
    +public class HostRegexTableLoadBalancer extends TableLoadBalancer {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(HostRegexTableLoadBalancer.class);
    +  public static final String HOST_BALANCER_PREFIX = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.";
    +  public static final String HOST_BALANCER_OOB_CHECK = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.oob.period";
    +  private static final String HOST_BALANCER_OOB_DEFAULT = "5m";
    +  public static final String HOST_BALANCER_POOL_RECHECK_KEY = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.pool.check";
    +  private static final String HOST_BALANCER_POOL_RECHECK_DEFAULT = "5m";
    +  public static final String HOST_BALANCER_REGEX_USING_IPS = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.is.ip";
    +  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 long lastOOBCheck = System.currentTimeMillis();
    +  private long lastPoolRecheck = 0;
    +  private boolean isIpBasedRegex = false;
    +  private Map<String,SortedMap<TServerInstance,TabletServerStatus>> pools = new HashMap<String,SortedMap<TServerInstance,TabletServerStatus>>();
    +
    +  /**
    +   * Group the set of current tservers by pool name. Tservers that don't match a regex are put into a default ppol.
    +   *
    +   * @param current
    +   *          map of current tservers
    +   * @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) {
    +      Map<String,SortedMap<TServerInstance,TabletServerStatus>> newPools = new HashMap<String,SortedMap<TServerInstance,TabletServerStatus>>();
    +      for (Entry<TServerInstance,TabletServerStatus> e : current.entrySet()) {
    +        String tableName = getPoolNameForHost(e.getKey().host());
    +        if (!newPools.containsKey(tableName)) {
    +          newPools.put(tableName, new TreeMap<TServerInstance,TabletServerStatus>());
    +        }
    +        newPools.get(tableName).put(e.getKey(), e.getValue());
    +      }
    +      // Ensure that no host is in more than one pool
    +      // TODO: I'm not sure that I need to check for disjoint as the call to getPoolNameForHost checks for more than one match
    +      boolean error = false;
    +      for (SortedMap<TServerInstance,TabletServerStatus> s1 : newPools.values()) {
    +        for (SortedMap<TServerInstance,TabletServerStatus> s2 : newPools.values()) {
    +          if (s1 == s2) {
    +            continue;
    +          }
    +          if (!Collections.disjoint(s1.keySet(), s2.keySet())) {
    +            LOG.error("Pools are not disjoint: {}, there is a problem with your regular expressions. Putting all servers in default pool", newPools);
    +            error = true;
    +          }
    +        }
    +      }
    +      if (error) {
    +        // Put all servers into the default pool
    +        newPools.clear();
    +        newPools.put(DEFAULT_POOL, new TreeMap<TServerInstance,TabletServerStatus>());
    +        for (Entry<TServerInstance,TabletServerStatus> e : current.entrySet()) {
    --- End diff --
    
    Nitpick, so ignore if you want, but you could replace this loop with newPools.putAll(current). (Same comment as above about the comparator also applies.)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request: ACCUMULO-4173: Host Regex Table Load Balanc...

Posted by dlmarion <gi...@git.apache.org>.
Github user dlmarion closed the pull request at:

    https://github.com/apache/accumulo/pull/83


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request: ACCUMULO-4173: Host Regex Table Load Balanc...

Posted by brianloss <gi...@git.apache.org>.
Github user brianloss commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/83#discussion_r57929457
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancer.java ---
    @@ -0,0 +1,314 @@
    +/*
    + * 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.net.InetAddress;
    +import java.net.UnknownHostException;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Set;
    +import java.util.SortedMap;
    +import java.util.TreeMap;
    +import java.util.regex.Pattern;
    +
    +import org.apache.accumulo.core.client.admin.TableOperations;
    +import org.apache.accumulo.core.conf.AccumuloConfiguration;
    +import org.apache.accumulo.core.conf.Property;
    +import org.apache.accumulo.core.data.impl.KeyExtent;
    +import org.apache.accumulo.core.master.thrift.TabletServerStatus;
    +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.thrift.TException;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * This balancer will create pools of tablet servers by grouping tablet servers that match a regex into the same pool and calling the balancer set on the table
    + * to balance within the set of matching tablet servers. <br>
    + * Regex properties for this balancer are specified as:<br>
    + * <b>table.custom.balancer.host.regex.&lt;tablename&gt;=&lt;regex&gt;</b><br>
    + * Periodically (default 5m) this balancer will check to see if a tablet server is hosting tablets that it should not be according to the regex configuration.
    + * 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 5m) 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>
    + * 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>
    + *
    + */
    +public class HostRegexTableLoadBalancer extends TableLoadBalancer {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(HostRegexTableLoadBalancer.class);
    +  public static final String HOST_BALANCER_PREFIX = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.";
    +  public static final String HOST_BALANCER_OOB_CHECK = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.oob.period";
    +  private static final String HOST_BALANCER_OOB_DEFAULT = "5m";
    +  public static final String HOST_BALANCER_POOL_RECHECK_KEY = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.pool.check";
    +  private static final String HOST_BALANCER_POOL_RECHECK_DEFAULT = "5m";
    +  public static final String HOST_BALANCER_REGEX_USING_IPS = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.is.ip";
    +  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 long lastOOBCheck = System.currentTimeMillis();
    +  private long lastPoolRecheck = 0;
    +  private boolean isIpBasedRegex = false;
    +  private Map<String,SortedMap<TServerInstance,TabletServerStatus>> pools = new HashMap<String,SortedMap<TServerInstance,TabletServerStatus>>();
    +
    +  /**
    +   * Group the set of current tservers by pool name. Tservers that don't match a regex are put into a default ppol.
    +   *
    +   * @param current
    +   *          map of current tservers
    +   * @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) {
    +      Map<String,SortedMap<TServerInstance,TabletServerStatus>> newPools = new HashMap<String,SortedMap<TServerInstance,TabletServerStatus>>();
    +      for (Entry<TServerInstance,TabletServerStatus> e : current.entrySet()) {
    +        String tableName = getPoolNameForHost(e.getKey().host());
    +        if (!newPools.containsKey(tableName)) {
    +          newPools.put(tableName, new TreeMap<TServerInstance,TabletServerStatus>());
    +        }
    +        newPools.get(tableName).put(e.getKey(), e.getValue());
    +      }
    +      // Ensure that no host is in more than one pool
    +      // TODO: I'm not sure that I need to check for disjoint as the call to getPoolNameForHost checks for more than one match
    +      boolean error = false;
    +      for (SortedMap<TServerInstance,TabletServerStatus> s1 : newPools.values()) {
    +        for (SortedMap<TServerInstance,TabletServerStatus> s2 : newPools.values()) {
    +          if (s1 == s2) {
    +            continue;
    +          }
    +          if (!Collections.disjoint(s1.keySet(), s2.keySet())) {
    +            LOG.error("Pools are not disjoint: {}, there is a problem with your regular expressions. Putting all servers in default pool", newPools);
    +            error = true;
    +          }
    +        }
    +      }
    +      if (error) {
    +        // Put all servers into the default pool
    +        newPools.clear();
    +        newPools.put(DEFAULT_POOL, new TreeMap<TServerInstance,TabletServerStatus>());
    +        for (Entry<TServerInstance,TabletServerStatus> e : current.entrySet()) {
    +          newPools.get(DEFAULT_POOL).put(e.getKey(), e.getValue());
    +        }
    +      }
    +      pools = newPools;
    +      this.lastPoolRecheck = System.currentTimeMillis();
    +    }
    +    return pools;
    +  }
    +
    +  /**
    +   * Matches host against the regexes and returns the matching pool name
    +   *
    +   * @param host
    +   *          tablet server host
    +   * @return name of pool. will return default pool if host matches more than one regex
    +   */
    +  protected String getPoolNameForHost(String host) {
    +    String test = host;
    +    String table = DEFAULT_POOL;
    --- End diff --
    
    Another nitpick, but I think this variable should be called pool not table.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request: ACCUMULO-4173: Host Regex Table Load Balanc...

Posted by brianloss <gi...@git.apache.org>.
Github user brianloss commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/83#discussion_r57928954
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancer.java ---
    @@ -0,0 +1,314 @@
    +/*
    + * 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.net.InetAddress;
    +import java.net.UnknownHostException;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Set;
    +import java.util.SortedMap;
    +import java.util.TreeMap;
    +import java.util.regex.Pattern;
    +
    +import org.apache.accumulo.core.client.admin.TableOperations;
    +import org.apache.accumulo.core.conf.AccumuloConfiguration;
    +import org.apache.accumulo.core.conf.Property;
    +import org.apache.accumulo.core.data.impl.KeyExtent;
    +import org.apache.accumulo.core.master.thrift.TabletServerStatus;
    +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.thrift.TException;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * This balancer will create pools of tablet servers by grouping tablet servers that match a regex into the same pool and calling the balancer set on the table
    + * to balance within the set of matching tablet servers. <br>
    + * Regex properties for this balancer are specified as:<br>
    + * <b>table.custom.balancer.host.regex.&lt;tablename&gt;=&lt;regex&gt;</b><br>
    + * Periodically (default 5m) this balancer will check to see if a tablet server is hosting tablets that it should not be according to the regex configuration.
    + * 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 5m) 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>
    + * 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>
    + *
    + */
    +public class HostRegexTableLoadBalancer extends TableLoadBalancer {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(HostRegexTableLoadBalancer.class);
    +  public static final String HOST_BALANCER_PREFIX = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.";
    +  public static final String HOST_BALANCER_OOB_CHECK = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.oob.period";
    +  private static final String HOST_BALANCER_OOB_DEFAULT = "5m";
    +  public static final String HOST_BALANCER_POOL_RECHECK_KEY = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.pool.check";
    +  private static final String HOST_BALANCER_POOL_RECHECK_DEFAULT = "5m";
    +  public static final String HOST_BALANCER_REGEX_USING_IPS = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.is.ip";
    +  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 long lastOOBCheck = System.currentTimeMillis();
    +  private long lastPoolRecheck = 0;
    +  private boolean isIpBasedRegex = false;
    +  private Map<String,SortedMap<TServerInstance,TabletServerStatus>> pools = new HashMap<String,SortedMap<TServerInstance,TabletServerStatus>>();
    +
    +  /**
    +   * Group the set of current tservers by pool name. Tservers that don't match a regex are put into a default ppol.
    +   *
    +   * @param current
    +   *          map of current tservers
    +   * @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) {
    +      Map<String,SortedMap<TServerInstance,TabletServerStatus>> newPools = new HashMap<String,SortedMap<TServerInstance,TabletServerStatus>>();
    +      for (Entry<TServerInstance,TabletServerStatus> e : current.entrySet()) {
    +        String tableName = getPoolNameForHost(e.getKey().host());
    +        if (!newPools.containsKey(tableName)) {
    +          newPools.put(tableName, new TreeMap<TServerInstance,TabletServerStatus>());
    +        }
    +        newPools.get(tableName).put(e.getKey(), e.getValue());
    +      }
    +      // Ensure that no host is in more than one pool
    +      // TODO: I'm not sure that I need to check for disjoint as the call to getPoolNameForHost checks for more than one match
    +      boolean error = false;
    +      for (SortedMap<TServerInstance,TabletServerStatus> s1 : newPools.values()) {
    --- End diff --
    
    This nested loop is doing a lot more disjoint checks than need to be done. You could change the inner loop to iterate over a tailMap starting from the current outer loop value to get rid of the dups.
    That being said, I can also think of cases where you might actually want disjoint pools. For example, one pool could be for "archive" data which is large and not accessed very often, so you want to spread it across all tservers. Then other pools, for hot tables would be on subsets of the tservers. To support this would mean some re-architecting of the code since the check would need to be get pool for host and table name.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request: ACCUMULO-4173: Host Regex Table Load Balanc...

Posted by dlmarion <gi...@git.apache.org>.
Github user dlmarion commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/83#discussion_r59385418
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancer.java ---
    @@ -0,0 +1,334 @@
    +/*
    + * 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.net.InetAddress;
    +import java.net.UnknownHostException;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Set;
    +import java.util.SortedMap;
    +import java.util.TreeMap;
    +import java.util.regex.Pattern;
    +
    +import org.apache.accumulo.core.client.admin.TableOperations;
    +import org.apache.accumulo.core.conf.AccumuloConfiguration;
    +import org.apache.accumulo.core.conf.Property;
    +import org.apache.accumulo.core.data.impl.KeyExtent;
    +import org.apache.accumulo.core.master.thrift.TabletServerStatus;
    +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.thrift.TException;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * This balancer will create pools of tablet servers by grouping tablet servers that match a regex into the same pool and calling the balancer set on the table
    + * to balance within the set of matching tablet servers. All tablet servers that do not match a regex are grouped into a default pool.<br>
    + * Regex properties for this balancer are specified as:<br>
    + * <b>table.custom.balancer.host.regex.&lt;tablename&gt;=&lt;regex&gt;</b><br>
    + * Periodically (default 5m) this balancer will check to see if a tablet server is hosting tablets that it should not be according to the regex configuration.
    + * 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 5m) 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>
    + * 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>
    + *
    + */
    +public class HostRegexTableLoadBalancer extends TableLoadBalancer {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(HostRegexTableLoadBalancer.class);
    +  public static final String HOST_BALANCER_PREFIX = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.";
    +  public static final String HOST_BALANCER_OOB_CHECK = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.oob.period";
    +  private static final String HOST_BALANCER_OOB_DEFAULT = "5m";
    +  public static final String HOST_BALANCER_POOL_RECHECK_KEY = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.pool.check";
    +  private static final String HOST_BALANCER_POOL_RECHECK_DEFAULT = "5m";
    +  public static final String HOST_BALANCER_REGEX_USING_IPS = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.is.ip";
    +  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<String,SortedMap<TServerInstance,TabletServerStatus>>();
    +
    +  /**
    +   * Group the set of current tservers by pool name. Tservers that don't match a regex are put into a default pool.
    +   *
    +   * @param current
    +   *          map of current tservers
    +   * @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 into pools based on regex");
    +      Map<String,SortedMap<TServerInstance,TabletServerStatus>> newPools = new HashMap<String,SortedMap<TServerInstance,TabletServerStatus>>();
    +      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<TServerInstance,TabletServerStatus>(current.comparator());
    +            newPools.put(pool, np);
    +          }
    +          np.put(e.getKey(), e.getValue());
    +        }
    +      }
    +      pools = newPools;
    +      this.lastPoolRecheck = System.currentTimeMillis();
    +    }
    +    return pools;
    +  }
    +
    +  /**
    +   * Matches host against the regexes and returns the matching pool names
    +   *
    +   * @param host
    +   *          tablet server host
    +   * @return pool names, will return default pool if host matches more no regex
    +   */
    +  protected List<String> getPoolNamesForHost(String host) {
    +    String test = host;
    +    if (!isIpBasedRegex) {
    +      try {
    +        test = getNameFromIp(host);
    +      } catch (UnknownHostException e1) {
    +        LOG.error("Unable to determine host name for IP: " + host + ", setting to default pool", e1);
    +        return Collections.singletonList(DEFAULT_POOL);
    +      }
    +    }
    +    List<String> pools = new ArrayList<>();
    +    for (Entry<String,Pattern> e : poolNameToRegexPattern.entrySet()) {
    +      if (e.getValue().matcher(test).matches()) {
    +        pools.add(e.getKey());
    +      }
    +    }
    +    if (pools.size() == 0) {
    +      pools.add(DEFAULT_POOL);
    +    }
    +    return pools;
    +  }
    +
    +  protected String getNameFromIp(String hostIp) throws UnknownHostException {
    +    return InetAddress.getByName(hostIp).getHostName();
    +  }
    +
    +  /**
    +   * Matches table name against pool names, returns matching pool name or DEFAULT_POOL.
    +   *
    +   * @param tableName
    +   *          name of table
    +   * @return tablet server pool name (table name or DEFAULT_POOL)
    +   */
    +  protected String getPoolNameForTable(String tableName) {
    +    if (null == tableName) {
    +      return DEFAULT_POOL;
    +    }
    +    return poolNameToRegexPattern.containsKey(tableName) ? tableName : DEFAULT_POOL;
    +  }
    +
    +  /**
    +   * Parse configuration and extract properties
    +   *
    +   * @param conf
    +   *          server configuration
    +   */
    +  protected void parseConfiguration(ServerConfiguration conf) {
    +    tableIdToTableName = new HashMap<>();
    +    poolNameToRegexPattern = new HashMap<>();
    +    for (Entry<String,String> table : getTableOperations().tableIdMap().entrySet()) {
    +      tableIdToTableName.put(table.getValue(), table.getKey());
    +      Map<String,String> customProps = conf.getTableConfiguration(table.getValue()).getAllPropertiesWithPrefix(Property.TABLE_ARBITRARY_PROP_PREFIX);
    +      if (null != customProps && customProps.size() > 0) {
    +        for (Entry<String,String> customProp : customProps.entrySet()) {
    +          if (customProp.getKey().startsWith(HOST_BALANCER_PREFIX)) {
    +            String tableName = customProp.getKey().substring(HOST_BALANCER_PREFIX.length());
    +            String regex = customProp.getValue();
    +            poolNameToRegexPattern.put(tableName, Pattern.compile(regex));
    +          }
    +        }
    +      }
    +    }
    +    String oobProperty = conf.getConfiguration().get(HOST_BALANCER_OOB_CHECK);
    +    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);
    +    if (null != ipBased) {
    +      isIpBasedRegex = Boolean.parseBoolean(ipBased);
    +    }
    +  }
    +
    +  public Map<String,String> getTableIdToTableName() {
    +    return tableIdToTableName;
    +  }
    +
    +  public Map<String,Pattern> getPoolNameToRegexPattern() {
    +    return poolNameToRegexPattern;
    +  }
    +
    +  public long getOobCheckMillis() {
    +    return oobCheckMillis;
    +  }
    +
    +  public long getPoolRecheckMillis() {
    +    return poolRecheckMillis;
    +  }
    +
    +  public boolean isIpBasedRegex() {
    +    return isIpBasedRegex;
    +  }
    +
    +  @Override
    +  public void init(ServerConfiguration conf) {
    +    super.init(conf);
    +    parseConfiguration(conf);
    +  }
    +
    +  @Override
    +  public void getAssignments(SortedMap<TServerInstance,TabletServerStatus> current, Map<KeyExtent,TServerInstance> unassigned,
    +      Map<KeyExtent,TServerInstance> assignments) {
    +
    +    Map<String,SortedMap<TServerInstance,TabletServerStatus>> pools = splitCurrentByRegex(current);
    +    // group the unassigned into tables
    +    Map<String,Map<KeyExtent,TServerInstance>> groupedUnassigned = new HashMap<String,Map<KeyExtent,TServerInstance>>();
    +    for (Entry<KeyExtent,TServerInstance> e : unassigned.entrySet()) {
    +      Map<KeyExtent,TServerInstance> tableUnassigned = groupedUnassigned.get(e.getKey().getTableId());
    +      if (tableUnassigned == null) {
    +        tableUnassigned = new HashMap<KeyExtent,TServerInstance>();
    +        groupedUnassigned.put(e.getKey().getTableId(), tableUnassigned);
    +      }
    +      tableUnassigned.put(e.getKey(), e.getValue());
    +    }
    +    // Send a view of the current servers to the tables tablet balancer
    +    for (Entry<String,Map<KeyExtent,TServerInstance>> e : groupedUnassigned.entrySet()) {
    +      Map<KeyExtent,TServerInstance> newAssignments = new HashMap<KeyExtent,TServerInstance>();
    +      String tableName = tableIdToTableName.get(e.getKey());
    +      String poolName = getPoolNameForTable(tableName);
    +      SortedMap<TServerInstance,TabletServerStatus> currentView = pools.get(poolName);
    +      if (null == currentView || currentView.size() == 0) {
    +        LOG.warn("No tablet servers online for table {}, assigning within default pool", tableName);
    +        currentView = pools.get(DEFAULT_POOL);
    +        if (null == currentView) {
    +          LOG.error("No tablet servers exist in the default pool, unable to assign tablets for table {}", tableName);
    --- End diff --
    
    Good catch. I didn't hit it in my testing. I'll fix.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request: ACCUMULO-4173: Host Regex Table Load Balanc...

Posted by dlmarion <gi...@git.apache.org>.
Github user dlmarion commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/83#discussion_r59392425
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancer.java ---
    @@ -0,0 +1,334 @@
    +/*
    + * 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.net.InetAddress;
    +import java.net.UnknownHostException;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Set;
    +import java.util.SortedMap;
    +import java.util.TreeMap;
    +import java.util.regex.Pattern;
    +
    +import org.apache.accumulo.core.client.admin.TableOperations;
    +import org.apache.accumulo.core.conf.AccumuloConfiguration;
    +import org.apache.accumulo.core.conf.Property;
    +import org.apache.accumulo.core.data.impl.KeyExtent;
    +import org.apache.accumulo.core.master.thrift.TabletServerStatus;
    +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.thrift.TException;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * This balancer will create pools of tablet servers by grouping tablet servers that match a regex into the same pool and calling the balancer set on the table
    + * to balance within the set of matching tablet servers. All tablet servers that do not match a regex are grouped into a default pool.<br>
    + * Regex properties for this balancer are specified as:<br>
    + * <b>table.custom.balancer.host.regex.&lt;tablename&gt;=&lt;regex&gt;</b><br>
    + * Periodically (default 5m) this balancer will check to see if a tablet server is hosting tablets that it should not be according to the regex configuration.
    + * 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 5m) 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>
    + * 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>
    + *
    + */
    +public class HostRegexTableLoadBalancer extends TableLoadBalancer {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(HostRegexTableLoadBalancer.class);
    +  public static final String HOST_BALANCER_PREFIX = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.";
    +  public static final String HOST_BALANCER_OOB_CHECK = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.oob.period";
    +  private static final String HOST_BALANCER_OOB_DEFAULT = "5m";
    +  public static final String HOST_BALANCER_POOL_RECHECK_KEY = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.pool.check";
    +  private static final String HOST_BALANCER_POOL_RECHECK_DEFAULT = "5m";
    +  public static final String HOST_BALANCER_REGEX_USING_IPS = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.is.ip";
    +  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<String,SortedMap<TServerInstance,TabletServerStatus>>();
    +
    +  /**
    +   * Group the set of current tservers by pool name. Tservers that don't match a regex are put into a default pool.
    +   *
    +   * @param current
    +   *          map of current tservers
    +   * @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 into pools based on regex");
    +      Map<String,SortedMap<TServerInstance,TabletServerStatus>> newPools = new HashMap<String,SortedMap<TServerInstance,TabletServerStatus>>();
    +      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<TServerInstance,TabletServerStatus>(current.comparator());
    +            newPools.put(pool, np);
    +          }
    +          np.put(e.getKey(), e.getValue());
    +        }
    +      }
    +      pools = newPools;
    +      this.lastPoolRecheck = System.currentTimeMillis();
    +    }
    +    return pools;
    +  }
    +
    +  /**
    +   * Matches host against the regexes and returns the matching pool names
    +   *
    +   * @param host
    +   *          tablet server host
    +   * @return pool names, will return default pool if host matches more no regex
    +   */
    +  protected List<String> getPoolNamesForHost(String host) {
    +    String test = host;
    +    if (!isIpBasedRegex) {
    +      try {
    +        test = getNameFromIp(host);
    +      } catch (UnknownHostException e1) {
    +        LOG.error("Unable to determine host name for IP: " + host + ", setting to default pool", e1);
    +        return Collections.singletonList(DEFAULT_POOL);
    +      }
    +    }
    +    List<String> pools = new ArrayList<>();
    +    for (Entry<String,Pattern> e : poolNameToRegexPattern.entrySet()) {
    +      if (e.getValue().matcher(test).matches()) {
    +        pools.add(e.getKey());
    +      }
    +    }
    +    if (pools.size() == 0) {
    +      pools.add(DEFAULT_POOL);
    +    }
    +    return pools;
    +  }
    +
    +  protected String getNameFromIp(String hostIp) throws UnknownHostException {
    +    return InetAddress.getByName(hostIp).getHostName();
    +  }
    +
    +  /**
    +   * Matches table name against pool names, returns matching pool name or DEFAULT_POOL.
    +   *
    +   * @param tableName
    +   *          name of table
    +   * @return tablet server pool name (table name or DEFAULT_POOL)
    +   */
    +  protected String getPoolNameForTable(String tableName) {
    +    if (null == tableName) {
    +      return DEFAULT_POOL;
    +    }
    +    return poolNameToRegexPattern.containsKey(tableName) ? tableName : DEFAULT_POOL;
    +  }
    +
    +  /**
    +   * Parse configuration and extract properties
    +   *
    +   * @param conf
    +   *          server configuration
    +   */
    +  protected void parseConfiguration(ServerConfiguration conf) {
    +    tableIdToTableName = new HashMap<>();
    +    poolNameToRegexPattern = new HashMap<>();
    +    for (Entry<String,String> table : getTableOperations().tableIdMap().entrySet()) {
    +      tableIdToTableName.put(table.getValue(), table.getKey());
    +      Map<String,String> customProps = conf.getTableConfiguration(table.getValue()).getAllPropertiesWithPrefix(Property.TABLE_ARBITRARY_PROP_PREFIX);
    +      if (null != customProps && customProps.size() > 0) {
    +        for (Entry<String,String> customProp : customProps.entrySet()) {
    +          if (customProp.getKey().startsWith(HOST_BALANCER_PREFIX)) {
    +            String tableName = customProp.getKey().substring(HOST_BALANCER_PREFIX.length());
    +            String regex = customProp.getValue();
    +            poolNameToRegexPattern.put(tableName, Pattern.compile(regex));
    +          }
    +        }
    +      }
    +    }
    +    String oobProperty = conf.getConfiguration().get(HOST_BALANCER_OOB_CHECK);
    +    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);
    +    if (null != ipBased) {
    +      isIpBasedRegex = Boolean.parseBoolean(ipBased);
    +    }
    +  }
    +
    +  public Map<String,String> getTableIdToTableName() {
    +    return tableIdToTableName;
    +  }
    +
    +  public Map<String,Pattern> getPoolNameToRegexPattern() {
    +    return poolNameToRegexPattern;
    +  }
    +
    +  public long getOobCheckMillis() {
    +    return oobCheckMillis;
    +  }
    +
    +  public long getPoolRecheckMillis() {
    +    return poolRecheckMillis;
    +  }
    +
    +  public boolean isIpBasedRegex() {
    +    return isIpBasedRegex;
    +  }
    +
    +  @Override
    +  public void init(ServerConfiguration conf) {
    +    super.init(conf);
    +    parseConfiguration(conf);
    +  }
    +
    +  @Override
    +  public void getAssignments(SortedMap<TServerInstance,TabletServerStatus> current, Map<KeyExtent,TServerInstance> unassigned,
    +      Map<KeyExtent,TServerInstance> assignments) {
    +
    +    Map<String,SortedMap<TServerInstance,TabletServerStatus>> pools = splitCurrentByRegex(current);
    --- End diff --
    
    I'm fine with lowering it. I'm about to push another commit that addresses the other comments as well.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request: ACCUMULO-4173: Host Regex Table Load Balanc...

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on the pull request:

    https://github.com/apache/accumulo/pull/83#issuecomment-208921993
  
    > Any final comments?
    
    I'll take a look again since you asked :)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request: ACCUMULO-4173: Host Regex Table Load Balanc...

Posted by brianloss <gi...@git.apache.org>.
Github user brianloss commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/83#discussion_r57930356
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancer.java ---
    @@ -0,0 +1,314 @@
    +/*
    + * 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.net.InetAddress;
    +import java.net.UnknownHostException;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Set;
    +import java.util.SortedMap;
    +import java.util.TreeMap;
    +import java.util.regex.Pattern;
    +
    +import org.apache.accumulo.core.client.admin.TableOperations;
    +import org.apache.accumulo.core.conf.AccumuloConfiguration;
    +import org.apache.accumulo.core.conf.Property;
    +import org.apache.accumulo.core.data.impl.KeyExtent;
    +import org.apache.accumulo.core.master.thrift.TabletServerStatus;
    +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.thrift.TException;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * This balancer will create pools of tablet servers by grouping tablet servers that match a regex into the same pool and calling the balancer set on the table
    + * to balance within the set of matching tablet servers. <br>
    + * Regex properties for this balancer are specified as:<br>
    + * <b>table.custom.balancer.host.regex.&lt;tablename&gt;=&lt;regex&gt;</b><br>
    + * Periodically (default 5m) this balancer will check to see if a tablet server is hosting tablets that it should not be according to the regex configuration.
    + * 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 5m) 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>
    + * 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>
    + *
    + */
    +public class HostRegexTableLoadBalancer extends TableLoadBalancer {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(HostRegexTableLoadBalancer.class);
    +  public static final String HOST_BALANCER_PREFIX = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.";
    +  public static final String HOST_BALANCER_OOB_CHECK = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.oob.period";
    +  private static final String HOST_BALANCER_OOB_DEFAULT = "5m";
    +  public static final String HOST_BALANCER_POOL_RECHECK_KEY = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.pool.check";
    +  private static final String HOST_BALANCER_POOL_RECHECK_DEFAULT = "5m";
    +  public static final String HOST_BALANCER_REGEX_USING_IPS = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.is.ip";
    +  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 long lastOOBCheck = System.currentTimeMillis();
    +  private long lastPoolRecheck = 0;
    +  private boolean isIpBasedRegex = false;
    +  private Map<String,SortedMap<TServerInstance,TabletServerStatus>> pools = new HashMap<String,SortedMap<TServerInstance,TabletServerStatus>>();
    +
    +  /**
    +   * Group the set of current tservers by pool name. Tservers that don't match a regex are put into a default ppol.
    +   *
    +   * @param current
    +   *          map of current tservers
    +   * @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) {
    +      Map<String,SortedMap<TServerInstance,TabletServerStatus>> newPools = new HashMap<String,SortedMap<TServerInstance,TabletServerStatus>>();
    +      for (Entry<TServerInstance,TabletServerStatus> e : current.entrySet()) {
    +        String tableName = getPoolNameForHost(e.getKey().host());
    +        if (!newPools.containsKey(tableName)) {
    +          newPools.put(tableName, new TreeMap<TServerInstance,TabletServerStatus>());
    +        }
    +        newPools.get(tableName).put(e.getKey(), e.getValue());
    +      }
    +      // Ensure that no host is in more than one pool
    +      // TODO: I'm not sure that I need to check for disjoint as the call to getPoolNameForHost checks for more than one match
    +      boolean error = false;
    +      for (SortedMap<TServerInstance,TabletServerStatus> s1 : newPools.values()) {
    +        for (SortedMap<TServerInstance,TabletServerStatus> s2 : newPools.values()) {
    +          if (s1 == s2) {
    +            continue;
    +          }
    +          if (!Collections.disjoint(s1.keySet(), s2.keySet())) {
    +            LOG.error("Pools are not disjoint: {}, there is a problem with your regular expressions. Putting all servers in default pool", newPools);
    +            error = true;
    +          }
    +        }
    +      }
    +      if (error) {
    +        // Put all servers into the default pool
    +        newPools.clear();
    +        newPools.put(DEFAULT_POOL, new TreeMap<TServerInstance,TabletServerStatus>());
    +        for (Entry<TServerInstance,TabletServerStatus> e : current.entrySet()) {
    +          newPools.get(DEFAULT_POOL).put(e.getKey(), e.getValue());
    +        }
    +      }
    +      pools = newPools;
    +      this.lastPoolRecheck = System.currentTimeMillis();
    +    }
    +    return pools;
    +  }
    +
    +  /**
    +   * Matches host against the regexes and returns the matching pool name
    +   *
    +   * @param host
    +   *          tablet server host
    +   * @return name of pool. will return default pool if host matches more than one regex
    +   */
    +  protected String getPoolNameForHost(String host) {
    +    String test = host;
    +    String table = DEFAULT_POOL;
    +    if (!isIpBasedRegex) {
    +      try {
    +        test = getNameFromIp(host);
    +      } catch (UnknownHostException e1) {
    +        LOG.error("Unable to determine host name for IP: " + host + ", setting to default pool", e1);
    +        return table;
    +      }
    +    }
    +    for (Entry<String,Pattern> e : poolNameToRegexPattern.entrySet()) {
    +      if (e.getValue().matcher(test).matches()) {
    +        if (!table.equals(DEFAULT_POOL)) {
    +          LOG.warn("host {} matches more than one regex, assigning to default pool", host);
    +          return DEFAULT_POOL;
    +        }
    +        table = e.getKey();
    +      }
    +    }
    +    return table;
    +  }
    +
    +  protected String getNameFromIp(String hostIp) throws UnknownHostException {
    +    return InetAddress.getByName(hostIp).getHostName();
    +  }
    +
    +  /**
    +   * Matches table name against pool names, returns matching pool name or DEFAULT_POOL.
    +   *
    +   * @param tableName
    +   *          name of table
    +   * @return tablet server pool name (table name or DEFAULT_POOL)
    +   */
    +  protected String getPoolNameForTable(String tableName) {
    +    if (null == tableName) {
    +      return DEFAULT_POOL;
    +    }
    +    return poolNameToRegexPattern.containsKey(tableName) ? tableName : DEFAULT_POOL;
    +  }
    +
    +  /**
    +   * Parse configuration and extract properties
    +   *
    +   * @param conf
    +   *          server configuration
    +   */
    +  protected void parseConfiguration(ServerConfiguration conf) {
    +    tableIdToTableName = new HashMap<>();
    +    poolNameToRegexPattern = new HashMap<>();
    +    for (Entry<String,String> table : getTableOperations().tableIdMap().entrySet()) {
    --- End diff --
    
    What happens if a new table is created after the balance is initialized, and the properties are updated to define a pool for that table? Also, what happens if a table is renamed?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request: ACCUMULO-4173: Host Regex Table Load Balanc...

Posted by brianloss <gi...@git.apache.org>.
Github user brianloss commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/83#discussion_r57927227
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancer.java ---
    @@ -0,0 +1,314 @@
    +/*
    + * 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.net.InetAddress;
    +import java.net.UnknownHostException;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Set;
    +import java.util.SortedMap;
    +import java.util.TreeMap;
    +import java.util.regex.Pattern;
    +
    +import org.apache.accumulo.core.client.admin.TableOperations;
    +import org.apache.accumulo.core.conf.AccumuloConfiguration;
    +import org.apache.accumulo.core.conf.Property;
    +import org.apache.accumulo.core.data.impl.KeyExtent;
    +import org.apache.accumulo.core.master.thrift.TabletServerStatus;
    +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.thrift.TException;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * This balancer will create pools of tablet servers by grouping tablet servers that match a regex into the same pool and calling the balancer set on the table
    + * to balance within the set of matching tablet servers. <br>
    + * Regex properties for this balancer are specified as:<br>
    + * <b>table.custom.balancer.host.regex.&lt;tablename&gt;=&lt;regex&gt;</b><br>
    + * Periodically (default 5m) this balancer will check to see if a tablet server is hosting tablets that it should not be according to the regex configuration.
    + * 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 5m) 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>
    + * 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>
    + *
    + */
    +public class HostRegexTableLoadBalancer extends TableLoadBalancer {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(HostRegexTableLoadBalancer.class);
    +  public static final String HOST_BALANCER_PREFIX = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.";
    +  public static final String HOST_BALANCER_OOB_CHECK = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.oob.period";
    +  private static final String HOST_BALANCER_OOB_DEFAULT = "5m";
    +  public static final String HOST_BALANCER_POOL_RECHECK_KEY = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.pool.check";
    +  private static final String HOST_BALANCER_POOL_RECHECK_DEFAULT = "5m";
    +  public static final String HOST_BALANCER_REGEX_USING_IPS = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.is.ip";
    +  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 long lastOOBCheck = System.currentTimeMillis();
    +  private long lastPoolRecheck = 0;
    +  private boolean isIpBasedRegex = false;
    +  private Map<String,SortedMap<TServerInstance,TabletServerStatus>> pools = new HashMap<String,SortedMap<TServerInstance,TabletServerStatus>>();
    +
    +  /**
    +   * Group the set of current tservers by pool name. Tservers that don't match a regex are put into a default ppol.
    +   *
    +   * @param current
    +   *          map of current tservers
    +   * @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) {
    +      Map<String,SortedMap<TServerInstance,TabletServerStatus>> newPools = new HashMap<String,SortedMap<TServerInstance,TabletServerStatus>>();
    +      for (Entry<TServerInstance,TabletServerStatus> e : current.entrySet()) {
    +        String tableName = getPoolNameForHost(e.getKey().host());
    +        if (!newPools.containsKey(tableName)) {
    +          newPools.put(tableName, new TreeMap<TServerInstance,TabletServerStatus>());
    --- End diff --
    
    Is there a need to ensure the TreeMap objects you create here have the same comparator as that used by current (see SortedMap.comparator())? If the incoming map isn't sorted based on the natural ordering, then this could be a problem.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request: ACCUMULO-4173: Host Regex Table Load Balanc...

Posted by dlmarion <gi...@git.apache.org>.
Github user dlmarion commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/83#discussion_r59392931
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancer.java ---
    @@ -0,0 +1,334 @@
    +/*
    + * 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.net.InetAddress;
    +import java.net.UnknownHostException;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Set;
    +import java.util.SortedMap;
    +import java.util.TreeMap;
    +import java.util.regex.Pattern;
    +
    +import org.apache.accumulo.core.client.admin.TableOperations;
    +import org.apache.accumulo.core.conf.AccumuloConfiguration;
    +import org.apache.accumulo.core.conf.Property;
    +import org.apache.accumulo.core.data.impl.KeyExtent;
    +import org.apache.accumulo.core.master.thrift.TabletServerStatus;
    +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.thrift.TException;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * This balancer will create pools of tablet servers by grouping tablet servers that match a regex into the same pool and calling the balancer set on the table
    + * to balance within the set of matching tablet servers. All tablet servers that do not match a regex are grouped into a default pool.<br>
    + * Regex properties for this balancer are specified as:<br>
    + * <b>table.custom.balancer.host.regex.&lt;tablename&gt;=&lt;regex&gt;</b><br>
    + * Periodically (default 5m) this balancer will check to see if a tablet server is hosting tablets that it should not be according to the regex configuration.
    + * 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 5m) 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>
    + * 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>
    + *
    + */
    +public class HostRegexTableLoadBalancer extends TableLoadBalancer {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(HostRegexTableLoadBalancer.class);
    +  public static final String HOST_BALANCER_PREFIX = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.";
    +  public static final String HOST_BALANCER_OOB_CHECK = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.oob.period";
    +  private static final String HOST_BALANCER_OOB_DEFAULT = "5m";
    +  public static final String HOST_BALANCER_POOL_RECHECK_KEY = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.pool.check";
    +  private static final String HOST_BALANCER_POOL_RECHECK_DEFAULT = "5m";
    +  public static final String HOST_BALANCER_REGEX_USING_IPS = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.is.ip";
    +  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<String,SortedMap<TServerInstance,TabletServerStatus>>();
    +
    +  /**
    +   * Group the set of current tservers by pool name. Tservers that don't match a regex are put into a default pool.
    +   *
    +   * @param current
    +   *          map of current tservers
    +   * @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) {
    --- End diff --
    
    Added a comment to the method javadoc.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request: ACCUMULO-4173: Host Regex Table Load Balanc...

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/83#discussion_r59382019
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancer.java ---
    @@ -0,0 +1,334 @@
    +/*
    + * 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.net.InetAddress;
    +import java.net.UnknownHostException;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Set;
    +import java.util.SortedMap;
    +import java.util.TreeMap;
    +import java.util.regex.Pattern;
    +
    +import org.apache.accumulo.core.client.admin.TableOperations;
    +import org.apache.accumulo.core.conf.AccumuloConfiguration;
    +import org.apache.accumulo.core.conf.Property;
    +import org.apache.accumulo.core.data.impl.KeyExtent;
    +import org.apache.accumulo.core.master.thrift.TabletServerStatus;
    +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.thrift.TException;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * This balancer will create pools of tablet servers by grouping tablet servers that match a regex into the same pool and calling the balancer set on the table
    + * to balance within the set of matching tablet servers. All tablet servers that do not match a regex are grouped into a default pool.<br>
    + * Regex properties for this balancer are specified as:<br>
    + * <b>table.custom.balancer.host.regex.&lt;tablename&gt;=&lt;regex&gt;</b><br>
    + * Periodically (default 5m) this balancer will check to see if a tablet server is hosting tablets that it should not be according to the regex configuration.
    + * 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 5m) 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>
    + * 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>
    + *
    + */
    +public class HostRegexTableLoadBalancer extends TableLoadBalancer {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(HostRegexTableLoadBalancer.class);
    +  public static final String HOST_BALANCER_PREFIX = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.";
    +  public static final String HOST_BALANCER_OOB_CHECK = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.oob.period";
    +  private static final String HOST_BALANCER_OOB_DEFAULT = "5m";
    +  public static final String HOST_BALANCER_POOL_RECHECK_KEY = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.pool.check";
    +  private static final String HOST_BALANCER_POOL_RECHECK_DEFAULT = "5m";
    +  public static final String HOST_BALANCER_REGEX_USING_IPS = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.is.ip";
    +  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<String,SortedMap<TServerInstance,TabletServerStatus>>();
    +
    +  /**
    +   * Group the set of current tservers by pool name. Tservers that don't match a regex are put into a default pool.
    +   *
    +   * @param current
    +   *          map of current tservers
    +   * @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) {
    --- End diff --
    
    This is probably fine when the defaults are on the order of minutes, but could be problematic if someone reduces the period. Maybe worthy of a comment to state the expectation that `poolRecheckMillis` is commonly order-minutes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request: ACCUMULO-4173: Host Regex Table Load Balanc...

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/83#discussion_r59384715
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancer.java ---
    @@ -0,0 +1,334 @@
    +/*
    + * 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.net.InetAddress;
    +import java.net.UnknownHostException;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Set;
    +import java.util.SortedMap;
    +import java.util.TreeMap;
    +import java.util.regex.Pattern;
    +
    +import org.apache.accumulo.core.client.admin.TableOperations;
    +import org.apache.accumulo.core.conf.AccumuloConfiguration;
    +import org.apache.accumulo.core.conf.Property;
    +import org.apache.accumulo.core.data.impl.KeyExtent;
    +import org.apache.accumulo.core.master.thrift.TabletServerStatus;
    +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.thrift.TException;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * This balancer will create pools of tablet servers by grouping tablet servers that match a regex into the same pool and calling the balancer set on the table
    + * to balance within the set of matching tablet servers. All tablet servers that do not match a regex are grouped into a default pool.<br>
    + * Regex properties for this balancer are specified as:<br>
    + * <b>table.custom.balancer.host.regex.&lt;tablename&gt;=&lt;regex&gt;</b><br>
    + * Periodically (default 5m) this balancer will check to see if a tablet server is hosting tablets that it should not be according to the regex configuration.
    + * 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 5m) 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>
    + * 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>
    + *
    + */
    +public class HostRegexTableLoadBalancer extends TableLoadBalancer {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(HostRegexTableLoadBalancer.class);
    +  public static final String HOST_BALANCER_PREFIX = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.";
    +  public static final String HOST_BALANCER_OOB_CHECK = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.oob.period";
    +  private static final String HOST_BALANCER_OOB_DEFAULT = "5m";
    +  public static final String HOST_BALANCER_POOL_RECHECK_KEY = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.pool.check";
    +  private static final String HOST_BALANCER_POOL_RECHECK_DEFAULT = "5m";
    +  public static final String HOST_BALANCER_REGEX_USING_IPS = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.is.ip";
    +  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<String,SortedMap<TServerInstance,TabletServerStatus>>();
    +
    +  /**
    +   * Group the set of current tservers by pool name. Tservers that don't match a regex are put into a default pool.
    +   *
    +   * @param current
    +   *          map of current tservers
    +   * @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 into pools based on regex");
    +      Map<String,SortedMap<TServerInstance,TabletServerStatus>> newPools = new HashMap<String,SortedMap<TServerInstance,TabletServerStatus>>();
    +      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<TServerInstance,TabletServerStatus>(current.comparator());
    +            newPools.put(pool, np);
    +          }
    +          np.put(e.getKey(), e.getValue());
    +        }
    +      }
    +      pools = newPools;
    +      this.lastPoolRecheck = System.currentTimeMillis();
    +    }
    +    return pools;
    +  }
    +
    +  /**
    +   * Matches host against the regexes and returns the matching pool names
    +   *
    +   * @param host
    +   *          tablet server host
    +   * @return pool names, will return default pool if host matches more no regex
    +   */
    +  protected List<String> getPoolNamesForHost(String host) {
    +    String test = host;
    +    if (!isIpBasedRegex) {
    +      try {
    +        test = getNameFromIp(host);
    +      } catch (UnknownHostException e1) {
    +        LOG.error("Unable to determine host name for IP: " + host + ", setting to default pool", e1);
    +        return Collections.singletonList(DEFAULT_POOL);
    +      }
    +    }
    +    List<String> pools = new ArrayList<>();
    +    for (Entry<String,Pattern> e : poolNameToRegexPattern.entrySet()) {
    +      if (e.getValue().matcher(test).matches()) {
    +        pools.add(e.getKey());
    +      }
    +    }
    +    if (pools.size() == 0) {
    +      pools.add(DEFAULT_POOL);
    +    }
    +    return pools;
    +  }
    +
    +  protected String getNameFromIp(String hostIp) throws UnknownHostException {
    +    return InetAddress.getByName(hostIp).getHostName();
    +  }
    +
    +  /**
    +   * Matches table name against pool names, returns matching pool name or DEFAULT_POOL.
    +   *
    +   * @param tableName
    +   *          name of table
    +   * @return tablet server pool name (table name or DEFAULT_POOL)
    +   */
    +  protected String getPoolNameForTable(String tableName) {
    +    if (null == tableName) {
    +      return DEFAULT_POOL;
    +    }
    +    return poolNameToRegexPattern.containsKey(tableName) ? tableName : DEFAULT_POOL;
    +  }
    +
    +  /**
    +   * Parse configuration and extract properties
    +   *
    +   * @param conf
    +   *          server configuration
    +   */
    +  protected void parseConfiguration(ServerConfiguration conf) {
    +    tableIdToTableName = new HashMap<>();
    +    poolNameToRegexPattern = new HashMap<>();
    +    for (Entry<String,String> table : getTableOperations().tableIdMap().entrySet()) {
    +      tableIdToTableName.put(table.getValue(), table.getKey());
    +      Map<String,String> customProps = conf.getTableConfiguration(table.getValue()).getAllPropertiesWithPrefix(Property.TABLE_ARBITRARY_PROP_PREFIX);
    +      if (null != customProps && customProps.size() > 0) {
    +        for (Entry<String,String> customProp : customProps.entrySet()) {
    +          if (customProp.getKey().startsWith(HOST_BALANCER_PREFIX)) {
    +            String tableName = customProp.getKey().substring(HOST_BALANCER_PREFIX.length());
    +            String regex = customProp.getValue();
    +            poolNameToRegexPattern.put(tableName, Pattern.compile(regex));
    +          }
    +        }
    +      }
    +    }
    +    String oobProperty = conf.getConfiguration().get(HOST_BALANCER_OOB_CHECK);
    +    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);
    +    if (null != ipBased) {
    +      isIpBasedRegex = Boolean.parseBoolean(ipBased);
    +    }
    +  }
    +
    +  public Map<String,String> getTableIdToTableName() {
    +    return tableIdToTableName;
    +  }
    +
    +  public Map<String,Pattern> getPoolNameToRegexPattern() {
    +    return poolNameToRegexPattern;
    +  }
    +
    +  public long getOobCheckMillis() {
    +    return oobCheckMillis;
    +  }
    +
    +  public long getPoolRecheckMillis() {
    +    return poolRecheckMillis;
    +  }
    +
    +  public boolean isIpBasedRegex() {
    +    return isIpBasedRegex;
    +  }
    +
    +  @Override
    +  public void init(ServerConfiguration conf) {
    +    super.init(conf);
    +    parseConfiguration(conf);
    +  }
    +
    +  @Override
    +  public void getAssignments(SortedMap<TServerInstance,TabletServerStatus> current, Map<KeyExtent,TServerInstance> unassigned,
    +      Map<KeyExtent,TServerInstance> assignments) {
    +
    +    Map<String,SortedMap<TServerInstance,TabletServerStatus>> pools = splitCurrentByRegex(current);
    +    // group the unassigned into tables
    +    Map<String,Map<KeyExtent,TServerInstance>> groupedUnassigned = new HashMap<String,Map<KeyExtent,TServerInstance>>();
    +    for (Entry<KeyExtent,TServerInstance> e : unassigned.entrySet()) {
    +      Map<KeyExtent,TServerInstance> tableUnassigned = groupedUnassigned.get(e.getKey().getTableId());
    +      if (tableUnassigned == null) {
    +        tableUnassigned = new HashMap<KeyExtent,TServerInstance>();
    +        groupedUnassigned.put(e.getKey().getTableId(), tableUnassigned);
    +      }
    +      tableUnassigned.put(e.getKey(), e.getValue());
    +    }
    +    // Send a view of the current servers to the tables tablet balancer
    +    for (Entry<String,Map<KeyExtent,TServerInstance>> e : groupedUnassigned.entrySet()) {
    +      Map<KeyExtent,TServerInstance> newAssignments = new HashMap<KeyExtent,TServerInstance>();
    +      String tableName = tableIdToTableName.get(e.getKey());
    +      String poolName = getPoolNameForTable(tableName);
    +      SortedMap<TServerInstance,TabletServerStatus> currentView = pools.get(poolName);
    +      if (null == currentView || currentView.size() == 0) {
    +        LOG.warn("No tablet servers online for table {}, assigning within default pool", tableName);
    +        currentView = pools.get(DEFAULT_POOL);
    +        if (null == currentView) {
    +          LOG.error("No tablet servers exist in the default pool, unable to assign tablets for table {}", tableName);
    --- End diff --
    
    Findbugs caught a bug here. You're likely missing a `continue`?. When `null == currentView`, you'll fall through and call `keySet()` on it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request: ACCUMULO-4173: Host Regex Table Load Balanc...

Posted by brianloss <gi...@git.apache.org>.
Github user brianloss commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/83#discussion_r57935222
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancer.java ---
    @@ -0,0 +1,314 @@
    +/*
    + * 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.net.InetAddress;
    +import java.net.UnknownHostException;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Set;
    +import java.util.SortedMap;
    +import java.util.TreeMap;
    +import java.util.regex.Pattern;
    +
    +import org.apache.accumulo.core.client.admin.TableOperations;
    +import org.apache.accumulo.core.conf.AccumuloConfiguration;
    +import org.apache.accumulo.core.conf.Property;
    +import org.apache.accumulo.core.data.impl.KeyExtent;
    +import org.apache.accumulo.core.master.thrift.TabletServerStatus;
    +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.thrift.TException;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * This balancer will create pools of tablet servers by grouping tablet servers that match a regex into the same pool and calling the balancer set on the table
    + * to balance within the set of matching tablet servers. <br>
    + * Regex properties for this balancer are specified as:<br>
    + * <b>table.custom.balancer.host.regex.&lt;tablename&gt;=&lt;regex&gt;</b><br>
    + * Periodically (default 5m) this balancer will check to see if a tablet server is hosting tablets that it should not be according to the regex configuration.
    + * 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 5m) 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>
    + * 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>
    + *
    + */
    +public class HostRegexTableLoadBalancer extends TableLoadBalancer {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(HostRegexTableLoadBalancer.class);
    +  public static final String HOST_BALANCER_PREFIX = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.";
    +  public static final String HOST_BALANCER_OOB_CHECK = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.oob.period";
    +  private static final String HOST_BALANCER_OOB_DEFAULT = "5m";
    +  public static final String HOST_BALANCER_POOL_RECHECK_KEY = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.pool.check";
    +  private static final String HOST_BALANCER_POOL_RECHECK_DEFAULT = "5m";
    +  public static final String HOST_BALANCER_REGEX_USING_IPS = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.is.ip";
    +  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 long lastOOBCheck = System.currentTimeMillis();
    +  private long lastPoolRecheck = 0;
    +  private boolean isIpBasedRegex = false;
    +  private Map<String,SortedMap<TServerInstance,TabletServerStatus>> pools = new HashMap<String,SortedMap<TServerInstance,TabletServerStatus>>();
    +
    +  /**
    +   * Group the set of current tservers by pool name. Tservers that don't match a regex are put into a default ppol.
    +   *
    +   * @param current
    +   *          map of current tservers
    +   * @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) {
    +      Map<String,SortedMap<TServerInstance,TabletServerStatus>> newPools = new HashMap<String,SortedMap<TServerInstance,TabletServerStatus>>();
    +      for (Entry<TServerInstance,TabletServerStatus> e : current.entrySet()) {
    +        String tableName = getPoolNameForHost(e.getKey().host());
    +        if (!newPools.containsKey(tableName)) {
    +          newPools.put(tableName, new TreeMap<TServerInstance,TabletServerStatus>());
    +        }
    +        newPools.get(tableName).put(e.getKey(), e.getValue());
    +      }
    +      // Ensure that no host is in more than one pool
    +      // TODO: I'm not sure that I need to check for disjoint as the call to getPoolNameForHost checks for more than one match
    +      boolean error = false;
    +      for (SortedMap<TServerInstance,TabletServerStatus> s1 : newPools.values()) {
    +        for (SortedMap<TServerInstance,TabletServerStatus> s2 : newPools.values()) {
    +          if (s1 == s2) {
    +            continue;
    +          }
    +          if (!Collections.disjoint(s1.keySet(), s2.keySet())) {
    +            LOG.error("Pools are not disjoint: {}, there is a problem with your regular expressions. Putting all servers in default pool", newPools);
    +            error = true;
    +          }
    +        }
    +      }
    +      if (error) {
    +        // Put all servers into the default pool
    +        newPools.clear();
    +        newPools.put(DEFAULT_POOL, new TreeMap<TServerInstance,TabletServerStatus>());
    +        for (Entry<TServerInstance,TabletServerStatus> e : current.entrySet()) {
    +          newPools.get(DEFAULT_POOL).put(e.getKey(), e.getValue());
    +        }
    +      }
    +      pools = newPools;
    +      this.lastPoolRecheck = System.currentTimeMillis();
    +    }
    +    return pools;
    +  }
    +
    +  /**
    +   * Matches host against the regexes and returns the matching pool name
    +   *
    +   * @param host
    +   *          tablet server host
    +   * @return name of pool. will return default pool if host matches more than one regex
    +   */
    +  protected String getPoolNameForHost(String host) {
    +    String test = host;
    +    String table = DEFAULT_POOL;
    +    if (!isIpBasedRegex) {
    +      try {
    +        test = getNameFromIp(host);
    +      } catch (UnknownHostException e1) {
    +        LOG.error("Unable to determine host name for IP: " + host + ", setting to default pool", e1);
    +        return table;
    +      }
    +    }
    +    for (Entry<String,Pattern> e : poolNameToRegexPattern.entrySet()) {
    +      if (e.getValue().matcher(test).matches()) {
    +        if (!table.equals(DEFAULT_POOL)) {
    +          LOG.warn("host {} matches more than one regex, assigning to default pool", host);
    +          return DEFAULT_POOL;
    +        }
    +        table = e.getKey();
    +      }
    +    }
    +    return table;
    +  }
    +
    +  protected String getNameFromIp(String hostIp) throws UnknownHostException {
    +    return InetAddress.getByName(hostIp).getHostName();
    +  }
    +
    +  /**
    +   * Matches table name against pool names, returns matching pool name or DEFAULT_POOL.
    +   *
    +   * @param tableName
    +   *          name of table
    +   * @return tablet server pool name (table name or DEFAULT_POOL)
    +   */
    +  protected String getPoolNameForTable(String tableName) {
    +    if (null == tableName) {
    +      return DEFAULT_POOL;
    +    }
    +    return poolNameToRegexPattern.containsKey(tableName) ? tableName : DEFAULT_POOL;
    +  }
    +
    +  /**
    +   * Parse configuration and extract properties
    +   *
    +   * @param conf
    +   *          server configuration
    +   */
    +  protected void parseConfiguration(ServerConfiguration conf) {
    +    tableIdToTableName = new HashMap<>();
    +    poolNameToRegexPattern = new HashMap<>();
    +    for (Entry<String,String> table : getTableOperations().tableIdMap().entrySet()) {
    +      tableIdToTableName.put(table.getValue(), table.getKey());
    +      Map<String,String> customProps = conf.getTableConfiguration(table.getValue()).getAllPropertiesWithPrefix(Property.TABLE_ARBITRARY_PROP_PREFIX);
    +      if (null != customProps && customProps.size() > 0) {
    +        for (Entry<String,String> customProp : customProps.entrySet()) {
    +          String tableName = customProp.getKey().substring(HOST_BALANCER_PREFIX.length());
    +          String regex = customProp.getValue();
    +          poolNameToRegexPattern.put(tableName, Pattern.compile(regex));
    +        }
    +      }
    +    }
    +    String oobProperty = conf.getConfiguration().get(HOST_BALANCER_OOB_CHECK);
    +    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);
    +    if (null != ipBased) {
    +      isIpBasedRegex = Boolean.parseBoolean(ipBased);
    +    }
    +  }
    +
    +  public Map<String,String> getTableIdToTableName() {
    +    return tableIdToTableName;
    +  }
    +
    +  public Map<String,Pattern> getPoolNameToRegexPattern() {
    +    return poolNameToRegexPattern;
    +  }
    +
    +  public long getOobCheckMillis() {
    +    return oobCheckMillis;
    +  }
    +
    +  public long getPoolRecheckMillis() {
    +    return poolRecheckMillis;
    +  }
    +
    +  public boolean isIpBasedRegex() {
    +    return isIpBasedRegex;
    +  }
    +
    +  @Override
    +  public void init(ServerConfiguration conf) {
    +    super.init(conf);
    +    parseConfiguration(conf);
    +  }
    +
    +  @Override
    +  public void getAssignments(SortedMap<TServerInstance,TabletServerStatus> current, Map<KeyExtent,TServerInstance> unassigned,
    +      Map<KeyExtent,TServerInstance> assignments) {
    +
    +    if ((System.currentTimeMillis() - this.lastOOBCheck) > this.oobCheckMillis) {
    +      // 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()) {
    +        String assignedPool = getPoolNameForHost(e.getKey().host());
    +        for (String pool : poolNameToRegexPattern.keySet()) {
    +          if (assignedPool.equals(pool) || pool.equals(DEFAULT_POOL)) {
    +            continue;
    +          }
    +          String tid = getTableOperations().tableIdMap().get(pool);
    +          try {
    +            List<TabletStats> outOfBoundsTablets = getOnlineTabletsForTable(e.getKey(), tid);
    +            for (TabletStats ts : outOfBoundsTablets) {
    +              LOG.info("Tablet {} is currently outside the bounds of the regex, reassigning", ts.toString());
    +              unassigned.put(new KeyExtent(ts.getExtent()), e.getKey());
    +            }
    +          } catch (TException e1) {
    +            LOG.error("Error in OOB check getting tablets for table {} from server {}", tid, e.getKey().host(), e);
    +          }
    +        }
    +      }
    +      this.oobCheckMillis = System.currentTimeMillis();
    +    }
    +
    +    Map<String,SortedMap<TServerInstance,TabletServerStatus>> pools = splitCurrentByRegex(current);
    +    // separate the unassigned into tables
    +    Map<String,Map<KeyExtent,TServerInstance>> groupedUnassigned = new HashMap<String,Map<KeyExtent,TServerInstance>>();
    +    for (Entry<KeyExtent,TServerInstance> e : unassigned.entrySet()) {
    +      Map<KeyExtent,TServerInstance> tableUnassigned = groupedUnassigned.get(e.getKey().getTableId());
    +      if (tableUnassigned == null) {
    +        tableUnassigned = new HashMap<KeyExtent,TServerInstance>();
    +        groupedUnassigned.put(e.getKey().getTableId(), tableUnassigned);
    +      }
    +      tableUnassigned.put(e.getKey(), e.getValue());
    +    }
    +    // Send a view of the current servers to the tables tablet balancer
    +    for (Entry<String,Map<KeyExtent,TServerInstance>> e : groupedUnassigned.entrySet()) {
    +      Map<KeyExtent,TServerInstance> newAssignments = new HashMap<KeyExtent,TServerInstance>();
    +      String tableName = tableIdToTableName.get(e.getKey());
    +      String poolName = getPoolNameForTable(tableName);
    +      SortedMap<TServerInstance,TabletServerStatus> currentView = pools.get(poolName);
    +      getBalancerForTable(e.getKey()).getAssignments(currentView, e.getValue(), newAssignments);
    +      assignments.putAll(newAssignments);
    +    }
    +  }
    +
    +  @Override
    +  public long balance(SortedMap<TServerInstance,TabletServerStatus> current, Set<KeyExtent> migrations, List<TabletMigration> migrationsOut) {
    +    long minBalanceTime = 5 * 1000;
    +    // Iterate over the tables and balance each of them
    +    TableOperations t = getTableOperations();
    +    if (t == null)
    --- End diff --
    
    You call getTableOperations() in other places in this code, and assume the response won't be null. Why is it treated differently here? Either this check is superfluous, or the other places need to be fixed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request: ACCUMULO-4173: Host Regex Table Load Balanc...

Posted by dlmarion <gi...@git.apache.org>.
Github user dlmarion commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/83#discussion_r59388944
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancer.java ---
    @@ -0,0 +1,334 @@
    +/*
    + * 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.net.InetAddress;
    +import java.net.UnknownHostException;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Set;
    +import java.util.SortedMap;
    +import java.util.TreeMap;
    +import java.util.regex.Pattern;
    +
    +import org.apache.accumulo.core.client.admin.TableOperations;
    +import org.apache.accumulo.core.conf.AccumuloConfiguration;
    +import org.apache.accumulo.core.conf.Property;
    +import org.apache.accumulo.core.data.impl.KeyExtent;
    +import org.apache.accumulo.core.master.thrift.TabletServerStatus;
    +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.thrift.TException;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * This balancer will create pools of tablet servers by grouping tablet servers that match a regex into the same pool and calling the balancer set on the table
    + * to balance within the set of matching tablet servers. All tablet servers that do not match a regex are grouped into a default pool.<br>
    + * Regex properties for this balancer are specified as:<br>
    + * <b>table.custom.balancer.host.regex.&lt;tablename&gt;=&lt;regex&gt;</b><br>
    + * Periodically (default 5m) this balancer will check to see if a tablet server is hosting tablets that it should not be according to the regex configuration.
    + * 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 5m) 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>
    + * 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>
    + *
    + */
    +public class HostRegexTableLoadBalancer extends TableLoadBalancer {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(HostRegexTableLoadBalancer.class);
    +  public static final String HOST_BALANCER_PREFIX = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.";
    +  public static final String HOST_BALANCER_OOB_CHECK = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.oob.period";
    +  private static final String HOST_BALANCER_OOB_DEFAULT = "5m";
    +  public static final String HOST_BALANCER_POOL_RECHECK_KEY = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.pool.check";
    +  private static final String HOST_BALANCER_POOL_RECHECK_DEFAULT = "5m";
    +  public static final String HOST_BALANCER_REGEX_USING_IPS = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.is.ip";
    +  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<String,SortedMap<TServerInstance,TabletServerStatus>>();
    +
    +  /**
    +   * Group the set of current tservers by pool name. Tservers that don't match a regex are put into a default pool.
    +   *
    +   * @param current
    +   *          map of current tservers
    +   * @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 into pools based on regex");
    +      Map<String,SortedMap<TServerInstance,TabletServerStatus>> newPools = new HashMap<String,SortedMap<TServerInstance,TabletServerStatus>>();
    +      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<TServerInstance,TabletServerStatus>(current.comparator());
    +            newPools.put(pool, np);
    +          }
    +          np.put(e.getKey(), e.getValue());
    +        }
    +      }
    +      pools = newPools;
    +      this.lastPoolRecheck = System.currentTimeMillis();
    +    }
    +    return pools;
    +  }
    +
    +  /**
    +   * Matches host against the regexes and returns the matching pool names
    +   *
    +   * @param host
    +   *          tablet server host
    +   * @return pool names, will return default pool if host matches more no regex
    +   */
    +  protected List<String> getPoolNamesForHost(String host) {
    +    String test = host;
    +    if (!isIpBasedRegex) {
    +      try {
    +        test = getNameFromIp(host);
    +      } catch (UnknownHostException e1) {
    +        LOG.error("Unable to determine host name for IP: " + host + ", setting to default pool", e1);
    +        return Collections.singletonList(DEFAULT_POOL);
    +      }
    +    }
    +    List<String> pools = new ArrayList<>();
    +    for (Entry<String,Pattern> e : poolNameToRegexPattern.entrySet()) {
    +      if (e.getValue().matcher(test).matches()) {
    +        pools.add(e.getKey());
    +      }
    +    }
    +    if (pools.size() == 0) {
    +      pools.add(DEFAULT_POOL);
    +    }
    +    return pools;
    +  }
    +
    +  protected String getNameFromIp(String hostIp) throws UnknownHostException {
    +    return InetAddress.getByName(hostIp).getHostName();
    +  }
    +
    +  /**
    +   * Matches table name against pool names, returns matching pool name or DEFAULT_POOL.
    +   *
    +   * @param tableName
    +   *          name of table
    +   * @return tablet server pool name (table name or DEFAULT_POOL)
    +   */
    +  protected String getPoolNameForTable(String tableName) {
    +    if (null == tableName) {
    +      return DEFAULT_POOL;
    +    }
    +    return poolNameToRegexPattern.containsKey(tableName) ? tableName : DEFAULT_POOL;
    +  }
    +
    +  /**
    +   * Parse configuration and extract properties
    +   *
    +   * @param conf
    +   *          server configuration
    +   */
    +  protected void parseConfiguration(ServerConfiguration conf) {
    +    tableIdToTableName = new HashMap<>();
    +    poolNameToRegexPattern = new HashMap<>();
    +    for (Entry<String,String> table : getTableOperations().tableIdMap().entrySet()) {
    +      tableIdToTableName.put(table.getValue(), table.getKey());
    +      Map<String,String> customProps = conf.getTableConfiguration(table.getValue()).getAllPropertiesWithPrefix(Property.TABLE_ARBITRARY_PROP_PREFIX);
    +      if (null != customProps && customProps.size() > 0) {
    +        for (Entry<String,String> customProp : customProps.entrySet()) {
    +          if (customProp.getKey().startsWith(HOST_BALANCER_PREFIX)) {
    +            String tableName = customProp.getKey().substring(HOST_BALANCER_PREFIX.length());
    +            String regex = customProp.getValue();
    +            poolNameToRegexPattern.put(tableName, Pattern.compile(regex));
    +          }
    +        }
    +      }
    +    }
    +    String oobProperty = conf.getConfiguration().get(HOST_BALANCER_OOB_CHECK);
    +    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);
    +    if (null != ipBased) {
    +      isIpBasedRegex = Boolean.parseBoolean(ipBased);
    +    }
    +  }
    +
    +  public Map<String,String> getTableIdToTableName() {
    +    return tableIdToTableName;
    +  }
    +
    +  public Map<String,Pattern> getPoolNameToRegexPattern() {
    +    return poolNameToRegexPattern;
    +  }
    +
    +  public long getOobCheckMillis() {
    +    return oobCheckMillis;
    +  }
    +
    +  public long getPoolRecheckMillis() {
    +    return poolRecheckMillis;
    +  }
    +
    +  public boolean isIpBasedRegex() {
    +    return isIpBasedRegex;
    +  }
    +
    +  @Override
    +  public void init(ServerConfiguration conf) {
    +    super.init(conf);
    +    parseConfiguration(conf);
    +  }
    +
    +  @Override
    +  public void getAssignments(SortedMap<TServerInstance,TabletServerStatus> current, Map<KeyExtent,TServerInstance> unassigned,
    +      Map<KeyExtent,TServerInstance> assignments) {
    +
    +    Map<String,SortedMap<TServerInstance,TabletServerStatus>> pools = splitCurrentByRegex(current);
    --- End diff --
    
    No, thats correct. You can set the pool recheck time to a low value, 1 to 2 minutes would seem reasonable, but then you pay the price of the recalculation.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request: ACCUMULO-4173: Host Regex Table Load Balanc...

Posted by ivakegg <gi...@git.apache.org>.
Github user ivakegg commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/83#discussion_r57940694
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancer.java ---
    @@ -0,0 +1,294 @@
    +/*
    + * 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.net.InetAddress;
    +import java.net.UnknownHostException;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Set;
    +import java.util.SortedMap;
    +import java.util.TreeMap;
    +import java.util.regex.Pattern;
    +
    +import org.apache.accumulo.core.client.admin.TableOperations;
    +import org.apache.accumulo.core.conf.AccumuloConfiguration;
    +import org.apache.accumulo.core.conf.Property;
    +import org.apache.accumulo.core.data.impl.KeyExtent;
    +import org.apache.accumulo.core.master.thrift.TabletServerStatus;
    +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.thrift.TException;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * This balancer will create pools of tablet servers by grouping tablet servers that match a regex into the same pool and calling the balancer set on the table
    + * to balance within the set of matching tablet servers. <br>
    + * Regex properties for this balancer are specified as:<br>
    + * <b>table.custom.balancer.host.regex.&lt;tablename&gt;=&lt;regex&gt;</b><br>
    + * Periodically (default 5m) this balancer will check to see if a tablet server is hosting tablets that it should not be according to the regex configuration.
    + * 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 5m) 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>
    + * 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>
    + *
    + */
    +public class HostRegexTableLoadBalancer extends TableLoadBalancer {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(HostRegexTableLoadBalancer.class);
    +  public static final String HOST_BALANCER_PREFIX = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.";
    +  public static final String HOST_BALANCER_OOB_CHECK = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.oob.period";
    +  private static final String HOST_BALANCER_OOB_DEFAULT = "5m";
    +  public static final String HOST_BALANCER_POOL_RECHECK_KEY = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.pool.check";
    +  private static final String HOST_BALANCER_POOL_RECHECK_DEFAULT = "5m";
    +  public static final String HOST_BALANCER_REGEX_USING_IPS = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.is.ip";
    +  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 long lastOOBCheck = System.currentTimeMillis();
    +  private long lastPoolRecheck = 0;
    +  private boolean isIpBasedRegex = false;
    +  private Map<String,SortedMap<TServerInstance,TabletServerStatus>> pools = new HashMap<String,SortedMap<TServerInstance,TabletServerStatus>>();
    +
    +  /**
    +   * Group the set of current tservers by pool name. Tservers that don't match a regex are put into a default ppol.
    +   *
    +   * @param current
    +   *          map of current tservers
    +   * @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) {
    +      Map<String,SortedMap<TServerInstance,TabletServerStatus>> newPools = new HashMap<String,SortedMap<TServerInstance,TabletServerStatus>>();
    +      for (Entry<TServerInstance,TabletServerStatus> e : current.entrySet()) {
    +        List<String> poolNames = getPoolNamesForHost(e.getKey().host());
    +        for (String pool : poolNames) {
    +          if (!newPools.containsKey(pool)) {
    +            newPools.put(pool, new TreeMap<TServerInstance,TabletServerStatus>(current.comparator()));
    +          }
    +          newPools.get(pool).put(e.getKey(), e.getValue());
    +        }
    +      }
    +      pools = newPools;
    +      this.lastPoolRecheck = System.currentTimeMillis();
    +    }
    +    return pools;
    +  }
    +
    +  /**
    +   * Matches host against the regexes and returns the matching pool names
    +   *
    +   * @param host
    +   *          tablet server host
    +   * @return pool names, will return default pool if host matches more no regex
    +   */
    +  protected List<String> getPoolNamesForHost(String host) {
    +    String test = host;
    +    if (!isIpBasedRegex) {
    +      try {
    +        test = getNameFromIp(host);
    +      } catch (UnknownHostException e1) {
    +        LOG.error("Unable to determine host name for IP: " + host + ", setting to default pool", e1);
    +        return Collections.singletonList(DEFAULT_POOL);
    +      }
    +    }
    +    List<String> pools = new ArrayList<>();
    +    for (Entry<String,Pattern> e : poolNameToRegexPattern.entrySet()) {
    +      if (e.getValue().matcher(test).matches()) {
    +        pools.add(e.getKey());
    +      }
    +    }
    +    if (pools.size() == 0) {
    +      pools.add(DEFAULT_POOL);
    +    }
    +    return pools;
    +  }
    +
    +  protected String getNameFromIp(String hostIp) throws UnknownHostException {
    +    return InetAddress.getByName(hostIp).getHostName();
    +  }
    +
    +  /**
    +   * Matches table name against pool names, returns matching pool name or DEFAULT_POOL.
    +   *
    +   * @param tableName
    +   *          name of table
    +   * @return tablet server pool name (table name or DEFAULT_POOL)
    +   */
    +  protected String getPoolNameForTable(String tableName) {
    +    if (null == tableName) {
    +      return DEFAULT_POOL;
    +    }
    +    return poolNameToRegexPattern.containsKey(tableName) ? tableName : DEFAULT_POOL;
    +  }
    +
    +  /**
    +   * Parse configuration and extract properties
    +   *
    +   * @param conf
    +   *          server configuration
    +   */
    +  protected void parseConfiguration(ServerConfiguration conf) {
    +    tableIdToTableName = new HashMap<>();
    +    poolNameToRegexPattern = new HashMap<>();
    +    for (Entry<String,String> table : getTableOperations().tableIdMap().entrySet()) {
    +      tableIdToTableName.put(table.getValue(), table.getKey());
    +      Map<String,String> customProps = conf.getTableConfiguration(table.getValue()).getAllPropertiesWithPrefix(Property.TABLE_ARBITRARY_PROP_PREFIX);
    +      if (null != customProps && customProps.size() > 0) {
    +        for (Entry<String,String> customProp : customProps.entrySet()) {
    +          String tableName = customProp.getKey().substring(HOST_BALANCER_PREFIX.length());
    +          String regex = customProp.getValue();
    +          poolNameToRegexPattern.put(tableName, Pattern.compile(regex));
    +        }
    +      }
    +    }
    +    String oobProperty = conf.getConfiguration().get(HOST_BALANCER_OOB_CHECK);
    +    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);
    +    if (null != ipBased) {
    +      isIpBasedRegex = Boolean.parseBoolean(ipBased);
    +    }
    +  }
    +
    +  public Map<String,String> getTableIdToTableName() {
    +    return tableIdToTableName;
    +  }
    +
    +  public Map<String,Pattern> getPoolNameToRegexPattern() {
    +    return poolNameToRegexPattern;
    +  }
    +
    +  public long getOobCheckMillis() {
    +    return oobCheckMillis;
    +  }
    +
    +  public long getPoolRecheckMillis() {
    +    return poolRecheckMillis;
    +  }
    +
    +  public boolean isIpBasedRegex() {
    +    return isIpBasedRegex;
    +  }
    +
    +  @Override
    +  public void init(ServerConfiguration conf) {
    +    super.init(conf);
    +    parseConfiguration(conf);
    +  }
    +
    +  @Override
    +  public void getAssignments(SortedMap<TServerInstance,TabletServerStatus> current, Map<KeyExtent,TServerInstance> unassigned,
    +      Map<KeyExtent,TServerInstance> assignments) {
    +
    +    if ((System.currentTimeMillis() - this.lastOOBCheck) > this.oobCheckMillis) {
    +      // 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 assignedPool : getPoolNamesForHost(e.getKey().host())) {
    +          for (String pool : poolNameToRegexPattern.keySet()) {
    --- End diff --
    
    rename "pool" to "table" or "tableName" here and add a comment that the poolNames in poolNameToRegexPattern are the same as table names.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request: ACCUMULO-4173: Host Regex Table Load Balanc...

Posted by dlmarion <gi...@git.apache.org>.
Github user dlmarion commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/83#discussion_r57930648
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancer.java ---
    @@ -0,0 +1,314 @@
    +/*
    + * 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.net.InetAddress;
    +import java.net.UnknownHostException;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Set;
    +import java.util.SortedMap;
    +import java.util.TreeMap;
    +import java.util.regex.Pattern;
    +
    +import org.apache.accumulo.core.client.admin.TableOperations;
    +import org.apache.accumulo.core.conf.AccumuloConfiguration;
    +import org.apache.accumulo.core.conf.Property;
    +import org.apache.accumulo.core.data.impl.KeyExtent;
    +import org.apache.accumulo.core.master.thrift.TabletServerStatus;
    +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.thrift.TException;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * This balancer will create pools of tablet servers by grouping tablet servers that match a regex into the same pool and calling the balancer set on the table
    + * to balance within the set of matching tablet servers. <br>
    + * Regex properties for this balancer are specified as:<br>
    + * <b>table.custom.balancer.host.regex.&lt;tablename&gt;=&lt;regex&gt;</b><br>
    + * Periodically (default 5m) this balancer will check to see if a tablet server is hosting tablets that it should not be according to the regex configuration.
    + * 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 5m) 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>
    + * 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>
    + *
    + */
    +public class HostRegexTableLoadBalancer extends TableLoadBalancer {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(HostRegexTableLoadBalancer.class);
    +  public static final String HOST_BALANCER_PREFIX = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.";
    +  public static final String HOST_BALANCER_OOB_CHECK = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.oob.period";
    +  private static final String HOST_BALANCER_OOB_DEFAULT = "5m";
    +  public static final String HOST_BALANCER_POOL_RECHECK_KEY = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.pool.check";
    +  private static final String HOST_BALANCER_POOL_RECHECK_DEFAULT = "5m";
    +  public static final String HOST_BALANCER_REGEX_USING_IPS = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.is.ip";
    +  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 long lastOOBCheck = System.currentTimeMillis();
    +  private long lastPoolRecheck = 0;
    +  private boolean isIpBasedRegex = false;
    +  private Map<String,SortedMap<TServerInstance,TabletServerStatus>> pools = new HashMap<String,SortedMap<TServerInstance,TabletServerStatus>>();
    +
    +  /**
    +   * Group the set of current tservers by pool name. Tservers that don't match a regex are put into a default ppol.
    +   *
    +   * @param current
    +   *          map of current tservers
    +   * @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) {
    +      Map<String,SortedMap<TServerInstance,TabletServerStatus>> newPools = new HashMap<String,SortedMap<TServerInstance,TabletServerStatus>>();
    +      for (Entry<TServerInstance,TabletServerStatus> e : current.entrySet()) {
    +        String tableName = getPoolNameForHost(e.getKey().host());
    +        if (!newPools.containsKey(tableName)) {
    +          newPools.put(tableName, new TreeMap<TServerInstance,TabletServerStatus>());
    +        }
    +        newPools.get(tableName).put(e.getKey(), e.getValue());
    +      }
    +      // Ensure that no host is in more than one pool
    +      // TODO: I'm not sure that I need to check for disjoint as the call to getPoolNameForHost checks for more than one match
    +      boolean error = false;
    +      for (SortedMap<TServerInstance,TabletServerStatus> s1 : newPools.values()) {
    --- End diff --
    
    I have no issue removing this check. However, getPoolNameForHost only returns one pool name; either the name of the pool that matches one regex, or the DEFAULT. I believe to support this example, getPoolNameForHost would have to change to return all matching pool names, and return DEFAULT when it matches none.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request: ACCUMULO-4173: Host Regex Table Load Balanc...

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/83#discussion_r57936424
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancer.java ---
    @@ -0,0 +1,314 @@
    +/*
    + * 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.net.InetAddress;
    +import java.net.UnknownHostException;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Set;
    +import java.util.SortedMap;
    +import java.util.TreeMap;
    +import java.util.regex.Pattern;
    +
    +import org.apache.accumulo.core.client.admin.TableOperations;
    +import org.apache.accumulo.core.conf.AccumuloConfiguration;
    +import org.apache.accumulo.core.conf.Property;
    +import org.apache.accumulo.core.data.impl.KeyExtent;
    +import org.apache.accumulo.core.master.thrift.TabletServerStatus;
    +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.thrift.TException;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * This balancer will create pools of tablet servers by grouping tablet servers that match a regex into the same pool and calling the balancer set on the table
    + * to balance within the set of matching tablet servers. <br>
    + * Regex properties for this balancer are specified as:<br>
    + * <b>table.custom.balancer.host.regex.&lt;tablename&gt;=&lt;regex&gt;</b><br>
    + * Periodically (default 5m) this balancer will check to see if a tablet server is hosting tablets that it should not be according to the regex configuration.
    + * 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 5m) 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>
    + * 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>
    + *
    + */
    +public class HostRegexTableLoadBalancer extends TableLoadBalancer {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(HostRegexTableLoadBalancer.class);
    +  public static final String HOST_BALANCER_PREFIX = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.";
    +  public static final String HOST_BALANCER_OOB_CHECK = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.oob.period";
    +  private static final String HOST_BALANCER_OOB_DEFAULT = "5m";
    +  public static final String HOST_BALANCER_POOL_RECHECK_KEY = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.pool.check";
    +  private static final String HOST_BALANCER_POOL_RECHECK_DEFAULT = "5m";
    +  public static final String HOST_BALANCER_REGEX_USING_IPS = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.is.ip";
    +  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 long lastOOBCheck = System.currentTimeMillis();
    +  private long lastPoolRecheck = 0;
    +  private boolean isIpBasedRegex = false;
    +  private Map<String,SortedMap<TServerInstance,TabletServerStatus>> pools = new HashMap<String,SortedMap<TServerInstance,TabletServerStatus>>();
    +
    +  /**
    +   * Group the set of current tservers by pool name. Tservers that don't match a regex are put into a default ppol.
    +   *
    +   * @param current
    +   *          map of current tservers
    +   * @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) {
    +      Map<String,SortedMap<TServerInstance,TabletServerStatus>> newPools = new HashMap<String,SortedMap<TServerInstance,TabletServerStatus>>();
    +      for (Entry<TServerInstance,TabletServerStatus> e : current.entrySet()) {
    +        String tableName = getPoolNameForHost(e.getKey().host());
    +        if (!newPools.containsKey(tableName)) {
    +          newPools.put(tableName, new TreeMap<TServerInstance,TabletServerStatus>());
    +        }
    +        newPools.get(tableName).put(e.getKey(), e.getValue());
    +      }
    +      // Ensure that no host is in more than one pool
    +      // TODO: I'm not sure that I need to check for disjoint as the call to getPoolNameForHost checks for more than one match
    +      boolean error = false;
    +      for (SortedMap<TServerInstance,TabletServerStatus> s1 : newPools.values()) {
    +        for (SortedMap<TServerInstance,TabletServerStatus> s2 : newPools.values()) {
    +          if (s1 == s2) {
    +            continue;
    +          }
    +          if (!Collections.disjoint(s1.keySet(), s2.keySet())) {
    +            LOG.error("Pools are not disjoint: {}, there is a problem with your regular expressions. Putting all servers in default pool", newPools);
    +            error = true;
    +          }
    +        }
    +      }
    +      if (error) {
    +        // Put all servers into the default pool
    +        newPools.clear();
    +        newPools.put(DEFAULT_POOL, new TreeMap<TServerInstance,TabletServerStatus>());
    +        for (Entry<TServerInstance,TabletServerStatus> e : current.entrySet()) {
    +          newPools.get(DEFAULT_POOL).put(e.getKey(), e.getValue());
    +        }
    +      }
    +      pools = newPools;
    +      this.lastPoolRecheck = System.currentTimeMillis();
    +    }
    +    return pools;
    +  }
    +
    +  /**
    +   * Matches host against the regexes and returns the matching pool name
    +   *
    +   * @param host
    +   *          tablet server host
    +   * @return name of pool. will return default pool if host matches more than one regex
    +   */
    +  protected String getPoolNameForHost(String host) {
    +    String test = host;
    +    String table = DEFAULT_POOL;
    +    if (!isIpBasedRegex) {
    +      try {
    +        test = getNameFromIp(host);
    +      } catch (UnknownHostException e1) {
    +        LOG.error("Unable to determine host name for IP: " + host + ", setting to default pool", e1);
    +        return table;
    +      }
    +    }
    +    for (Entry<String,Pattern> e : poolNameToRegexPattern.entrySet()) {
    +      if (e.getValue().matcher(test).matches()) {
    +        if (!table.equals(DEFAULT_POOL)) {
    +          LOG.warn("host {} matches more than one regex, assigning to default pool", host);
    +          return DEFAULT_POOL;
    +        }
    +        table = e.getKey();
    +      }
    +    }
    +    return table;
    +  }
    +
    +  protected String getNameFromIp(String hostIp) throws UnknownHostException {
    +    return InetAddress.getByName(hostIp).getHostName();
    +  }
    +
    +  /**
    +   * Matches table name against pool names, returns matching pool name or DEFAULT_POOL.
    +   *
    +   * @param tableName
    +   *          name of table
    +   * @return tablet server pool name (table name or DEFAULT_POOL)
    +   */
    +  protected String getPoolNameForTable(String tableName) {
    +    if (null == tableName) {
    +      return DEFAULT_POOL;
    +    }
    +    return poolNameToRegexPattern.containsKey(tableName) ? tableName : DEFAULT_POOL;
    +  }
    +
    +  /**
    +   * Parse configuration and extract properties
    +   *
    +   * @param conf
    +   *          server configuration
    +   */
    +  protected void parseConfiguration(ServerConfiguration conf) {
    +    tableIdToTableName = new HashMap<>();
    +    poolNameToRegexPattern = new HashMap<>();
    +    for (Entry<String,String> table : getTableOperations().tableIdMap().entrySet()) {
    --- End diff --
    
    There's logic to handle configuration updates via a Callback. Seems logical that active Balancer could also tie into that callback to process configuration updates. (could be follow-on work since it already doesn't do that)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request: ACCUMULO-4173: Host Regex Table Load Balanc...

Posted by ivakegg <gi...@git.apache.org>.
Github user ivakegg commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/83#discussion_r57935942
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancer.java ---
    @@ -0,0 +1,294 @@
    +/*
    + * 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.net.InetAddress;
    +import java.net.UnknownHostException;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Set;
    +import java.util.SortedMap;
    +import java.util.TreeMap;
    +import java.util.regex.Pattern;
    +
    +import org.apache.accumulo.core.client.admin.TableOperations;
    +import org.apache.accumulo.core.conf.AccumuloConfiguration;
    +import org.apache.accumulo.core.conf.Property;
    +import org.apache.accumulo.core.data.impl.KeyExtent;
    +import org.apache.accumulo.core.master.thrift.TabletServerStatus;
    +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.thrift.TException;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * This balancer will create pools of tablet servers by grouping tablet servers that match a regex into the same pool and calling the balancer set on the table
    + * to balance within the set of matching tablet servers. <br>
    + * Regex properties for this balancer are specified as:<br>
    + * <b>table.custom.balancer.host.regex.&lt;tablename&gt;=&lt;regex&gt;</b><br>
    + * Periodically (default 5m) this balancer will check to see if a tablet server is hosting tablets that it should not be according to the regex configuration.
    + * 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 5m) 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>
    + * 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>
    + *
    + */
    +public class HostRegexTableLoadBalancer extends TableLoadBalancer {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(HostRegexTableLoadBalancer.class);
    +  public static final String HOST_BALANCER_PREFIX = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.";
    +  public static final String HOST_BALANCER_OOB_CHECK = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.oob.period";
    +  private static final String HOST_BALANCER_OOB_DEFAULT = "5m";
    +  public static final String HOST_BALANCER_POOL_RECHECK_KEY = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.pool.check";
    +  private static final String HOST_BALANCER_POOL_RECHECK_DEFAULT = "5m";
    +  public static final String HOST_BALANCER_REGEX_USING_IPS = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.is.ip";
    +  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 long lastOOBCheck = System.currentTimeMillis();
    +  private long lastPoolRecheck = 0;
    +  private boolean isIpBasedRegex = false;
    +  private Map<String,SortedMap<TServerInstance,TabletServerStatus>> pools = new HashMap<String,SortedMap<TServerInstance,TabletServerStatus>>();
    +
    +  /**
    +   * Group the set of current tservers by pool name. Tservers that don't match a regex are put into a default ppol.
    --- End diff --
    
    typo: ppol


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request: ACCUMULO-4173: Host Regex Table Load Balanc...

Posted by dlmarion <gi...@git.apache.org>.
Github user dlmarion commented on the pull request:

    https://github.com/apache/accumulo/pull/83#issuecomment-208955560
  
    I would normally wait, but I have tested this and its working and other have laid eyes on it and commented. I plan on manually applying this to 1.6 and merging up. I think I will do that and commit it and address anything he raises in another issue. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request: ACCUMULO-4173: Host Regex Table Load Balanc...

Posted by joshelser <gi...@git.apache.org>.
Github user joshelser commented on the pull request:

    https://github.com/apache/accumulo/pull/83#issuecomment-208953363
  
    I think I'm good with this. FWIW, I did ask Eric if he would have any time to take a look, and he mentioned that, if he does, it wouldn't be until after normal working hours. Up to you if you want to wait for him to give a pass over it, of course.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request: ACCUMULO-4173: Host Regex Table Load Balanc...

Posted by dlmarion <gi...@git.apache.org>.
Github user dlmarion commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/83#discussion_r59393036
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancer.java ---
    @@ -0,0 +1,334 @@
    +/*
    + * 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.net.InetAddress;
    +import java.net.UnknownHostException;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Set;
    +import java.util.SortedMap;
    +import java.util.TreeMap;
    +import java.util.regex.Pattern;
    +
    +import org.apache.accumulo.core.client.admin.TableOperations;
    +import org.apache.accumulo.core.conf.AccumuloConfiguration;
    +import org.apache.accumulo.core.conf.Property;
    +import org.apache.accumulo.core.data.impl.KeyExtent;
    +import org.apache.accumulo.core.master.thrift.TabletServerStatus;
    +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.thrift.TException;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * This balancer will create pools of tablet servers by grouping tablet servers that match a regex into the same pool and calling the balancer set on the table
    + * to balance within the set of matching tablet servers. All tablet servers that do not match a regex are grouped into a default pool.<br>
    + * Regex properties for this balancer are specified as:<br>
    + * <b>table.custom.balancer.host.regex.&lt;tablename&gt;=&lt;regex&gt;</b><br>
    + * Periodically (default 5m) this balancer will check to see if a tablet server is hosting tablets that it should not be according to the regex configuration.
    + * 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 5m) 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>
    + * 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>
    + *
    + */
    +public class HostRegexTableLoadBalancer extends TableLoadBalancer {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(HostRegexTableLoadBalancer.class);
    +  public static final String HOST_BALANCER_PREFIX = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.";
    +  public static final String HOST_BALANCER_OOB_CHECK = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.oob.period";
    +  private static final String HOST_BALANCER_OOB_DEFAULT = "5m";
    +  public static final String HOST_BALANCER_POOL_RECHECK_KEY = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.pool.check";
    +  private static final String HOST_BALANCER_POOL_RECHECK_DEFAULT = "5m";
    +  public static final String HOST_BALANCER_REGEX_USING_IPS = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.is.ip";
    +  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<String,SortedMap<TServerInstance,TabletServerStatus>>();
    +
    +  /**
    +   * Group the set of current tservers by pool name. Tservers that don't match a regex are put into a default pool.
    +   *
    +   * @param current
    +   *          map of current tservers
    +   * @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 into pools based on regex");
    +      Map<String,SortedMap<TServerInstance,TabletServerStatus>> newPools = new HashMap<String,SortedMap<TServerInstance,TabletServerStatus>>();
    +      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<TServerInstance,TabletServerStatus>(current.comparator());
    +            newPools.put(pool, np);
    +          }
    +          np.put(e.getKey(), e.getValue());
    +        }
    +      }
    +      pools = newPools;
    +      this.lastPoolRecheck = System.currentTimeMillis();
    +    }
    +    return pools;
    +  }
    +
    +  /**
    +   * Matches host against the regexes and returns the matching pool names
    +   *
    +   * @param host
    +   *          tablet server host
    +   * @return pool names, will return default pool if host matches more no regex
    +   */
    +  protected List<String> getPoolNamesForHost(String host) {
    +    String test = host;
    +    if (!isIpBasedRegex) {
    +      try {
    +        test = getNameFromIp(host);
    +      } catch (UnknownHostException e1) {
    +        LOG.error("Unable to determine host name for IP: " + host + ", setting to default pool", e1);
    +        return Collections.singletonList(DEFAULT_POOL);
    +      }
    +    }
    +    List<String> pools = new ArrayList<>();
    +    for (Entry<String,Pattern> e : poolNameToRegexPattern.entrySet()) {
    +      if (e.getValue().matcher(test).matches()) {
    +        pools.add(e.getKey());
    +      }
    +    }
    +    if (pools.size() == 0) {
    +      pools.add(DEFAULT_POOL);
    +    }
    +    return pools;
    +  }
    +
    +  protected String getNameFromIp(String hostIp) throws UnknownHostException {
    +    return InetAddress.getByName(hostIp).getHostName();
    +  }
    +
    +  /**
    +   * Matches table name against pool names, returns matching pool name or DEFAULT_POOL.
    +   *
    +   * @param tableName
    +   *          name of table
    +   * @return tablet server pool name (table name or DEFAULT_POOL)
    +   */
    +  protected String getPoolNameForTable(String tableName) {
    +    if (null == tableName) {
    +      return DEFAULT_POOL;
    +    }
    +    return poolNameToRegexPattern.containsKey(tableName) ? tableName : DEFAULT_POOL;
    +  }
    +
    +  /**
    +   * Parse configuration and extract properties
    +   *
    +   * @param conf
    +   *          server configuration
    +   */
    +  protected void parseConfiguration(ServerConfiguration conf) {
    +    tableIdToTableName = new HashMap<>();
    +    poolNameToRegexPattern = new HashMap<>();
    +    for (Entry<String,String> table : getTableOperations().tableIdMap().entrySet()) {
    +      tableIdToTableName.put(table.getValue(), table.getKey());
    +      Map<String,String> customProps = conf.getTableConfiguration(table.getValue()).getAllPropertiesWithPrefix(Property.TABLE_ARBITRARY_PROP_PREFIX);
    +      if (null != customProps && customProps.size() > 0) {
    +        for (Entry<String,String> customProp : customProps.entrySet()) {
    +          if (customProp.getKey().startsWith(HOST_BALANCER_PREFIX)) {
    +            String tableName = customProp.getKey().substring(HOST_BALANCER_PREFIX.length());
    +            String regex = customProp.getValue();
    +            poolNameToRegexPattern.put(tableName, Pattern.compile(regex));
    +          }
    +        }
    +      }
    +    }
    +    String oobProperty = conf.getConfiguration().get(HOST_BALANCER_OOB_CHECK);
    +    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);
    +    if (null != ipBased) {
    +      isIpBasedRegex = Boolean.parseBoolean(ipBased);
    +    }
    +  }
    +
    +  public Map<String,String> getTableIdToTableName() {
    +    return tableIdToTableName;
    +  }
    +
    +  public Map<String,Pattern> getPoolNameToRegexPattern() {
    +    return poolNameToRegexPattern;
    +  }
    +
    +  public long getOobCheckMillis() {
    +    return oobCheckMillis;
    +  }
    +
    +  public long getPoolRecheckMillis() {
    +    return poolRecheckMillis;
    +  }
    +
    +  public boolean isIpBasedRegex() {
    +    return isIpBasedRegex;
    +  }
    +
    +  @Override
    +  public void init(ServerConfiguration conf) {
    +    super.init(conf);
    +    parseConfiguration(conf);
    +  }
    +
    +  @Override
    +  public void getAssignments(SortedMap<TServerInstance,TabletServerStatus> current, Map<KeyExtent,TServerInstance> unassigned,
    +      Map<KeyExtent,TServerInstance> assignments) {
    +
    +    Map<String,SortedMap<TServerInstance,TabletServerStatus>> pools = splitCurrentByRegex(current);
    +    // group the unassigned into tables
    +    Map<String,Map<KeyExtent,TServerInstance>> groupedUnassigned = new HashMap<String,Map<KeyExtent,TServerInstance>>();
    +    for (Entry<KeyExtent,TServerInstance> e : unassigned.entrySet()) {
    +      Map<KeyExtent,TServerInstance> tableUnassigned = groupedUnassigned.get(e.getKey().getTableId());
    +      if (tableUnassigned == null) {
    +        tableUnassigned = new HashMap<KeyExtent,TServerInstance>();
    +        groupedUnassigned.put(e.getKey().getTableId(), tableUnassigned);
    +      }
    +      tableUnassigned.put(e.getKey(), e.getValue());
    +    }
    +    // Send a view of the current servers to the tables tablet balancer
    +    for (Entry<String,Map<KeyExtent,TServerInstance>> e : groupedUnassigned.entrySet()) {
    +      Map<KeyExtent,TServerInstance> newAssignments = new HashMap<KeyExtent,TServerInstance>();
    +      String tableName = tableIdToTableName.get(e.getKey());
    +      String poolName = getPoolNameForTable(tableName);
    +      SortedMap<TServerInstance,TabletServerStatus> currentView = pools.get(poolName);
    +      if (null == currentView || currentView.size() == 0) {
    +        LOG.warn("No tablet servers online for table {}, assigning within default pool", tableName);
    +        currentView = pools.get(DEFAULT_POOL);
    +        if (null == currentView) {
    +          LOG.error("No tablet servers exist in the default pool, unable to assign tablets for table {}", tableName);
    --- End diff --
    
    Fixed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] accumulo pull request: ACCUMULO-4173: Host Regex Table Load Balanc...

Posted by brianloss <gi...@git.apache.org>.
Github user brianloss commented on a diff in the pull request:

    https://github.com/apache/accumulo/pull/83#discussion_r57938201
  
    --- Diff: server/base/src/main/java/org/apache/accumulo/server/master/balancer/HostRegexTableLoadBalancer.java ---
    @@ -0,0 +1,294 @@
    +/*
    + * 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.net.InetAddress;
    +import java.net.UnknownHostException;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Map.Entry;
    +import java.util.Set;
    +import java.util.SortedMap;
    +import java.util.TreeMap;
    +import java.util.regex.Pattern;
    +
    +import org.apache.accumulo.core.client.admin.TableOperations;
    +import org.apache.accumulo.core.conf.AccumuloConfiguration;
    +import org.apache.accumulo.core.conf.Property;
    +import org.apache.accumulo.core.data.impl.KeyExtent;
    +import org.apache.accumulo.core.master.thrift.TabletServerStatus;
    +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.thrift.TException;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +/**
    + * This balancer will create pools of tablet servers by grouping tablet servers that match a regex into the same pool and calling the balancer set on the table
    + * to balance within the set of matching tablet servers. <br>
    + * Regex properties for this balancer are specified as:<br>
    + * <b>table.custom.balancer.host.regex.&lt;tablename&gt;=&lt;regex&gt;</b><br>
    + * Periodically (default 5m) this balancer will check to see if a tablet server is hosting tablets that it should not be according to the regex configuration.
    + * 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 5m) 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>
    + * 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>
    + *
    + */
    +public class HostRegexTableLoadBalancer extends TableLoadBalancer {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(HostRegexTableLoadBalancer.class);
    +  public static final String HOST_BALANCER_PREFIX = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.";
    +  public static final String HOST_BALANCER_OOB_CHECK = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.oob.period";
    +  private static final String HOST_BALANCER_OOB_DEFAULT = "5m";
    +  public static final String HOST_BALANCER_POOL_RECHECK_KEY = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.pool.check";
    +  private static final String HOST_BALANCER_POOL_RECHECK_DEFAULT = "5m";
    +  public static final String HOST_BALANCER_REGEX_USING_IPS = Property.TABLE_ARBITRARY_PROP_PREFIX.getKey() + "balancer.host.regex.is.ip";
    +  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 long lastOOBCheck = System.currentTimeMillis();
    +  private long lastPoolRecheck = 0;
    +  private boolean isIpBasedRegex = false;
    +  private Map<String,SortedMap<TServerInstance,TabletServerStatus>> pools = new HashMap<String,SortedMap<TServerInstance,TabletServerStatus>>();
    +
    +  /**
    +   * Group the set of current tservers by pool name. Tservers that don't match a regex are put into a default ppol.
    +   *
    +   * @param current
    +   *          map of current tservers
    +   * @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) {
    +      Map<String,SortedMap<TServerInstance,TabletServerStatus>> newPools = new HashMap<String,SortedMap<TServerInstance,TabletServerStatus>>();
    +      for (Entry<TServerInstance,TabletServerStatus> e : current.entrySet()) {
    +        List<String> poolNames = getPoolNamesForHost(e.getKey().host());
    +        for (String pool : poolNames) {
    +          if (!newPools.containsKey(pool)) {
    +            newPools.put(pool, new TreeMap<TServerInstance,TabletServerStatus>(current.comparator()));
    +          }
    +          newPools.get(pool).put(e.getKey(), e.getValue());
    +        }
    +      }
    +      pools = newPools;
    +      this.lastPoolRecheck = System.currentTimeMillis();
    +    }
    +    return pools;
    +  }
    +
    +  /**
    +   * Matches host against the regexes and returns the matching pool names
    +   *
    +   * @param host
    +   *          tablet server host
    +   * @return pool names, will return default pool if host matches more no regex
    +   */
    +  protected List<String> getPoolNamesForHost(String host) {
    +    String test = host;
    +    if (!isIpBasedRegex) {
    +      try {
    +        test = getNameFromIp(host);
    +      } catch (UnknownHostException e1) {
    +        LOG.error("Unable to determine host name for IP: " + host + ", setting to default pool", e1);
    +        return Collections.singletonList(DEFAULT_POOL);
    +      }
    +    }
    +    List<String> pools = new ArrayList<>();
    +    for (Entry<String,Pattern> e : poolNameToRegexPattern.entrySet()) {
    +      if (e.getValue().matcher(test).matches()) {
    +        pools.add(e.getKey());
    +      }
    +    }
    +    if (pools.size() == 0) {
    +      pools.add(DEFAULT_POOL);
    +    }
    +    return pools;
    +  }
    +
    +  protected String getNameFromIp(String hostIp) throws UnknownHostException {
    +    return InetAddress.getByName(hostIp).getHostName();
    +  }
    +
    +  /**
    +   * Matches table name against pool names, returns matching pool name or DEFAULT_POOL.
    +   *
    +   * @param tableName
    +   *          name of table
    +   * @return tablet server pool name (table name or DEFAULT_POOL)
    +   */
    +  protected String getPoolNameForTable(String tableName) {
    +    if (null == tableName) {
    +      return DEFAULT_POOL;
    +    }
    +    return poolNameToRegexPattern.containsKey(tableName) ? tableName : DEFAULT_POOL;
    +  }
    +
    +  /**
    +   * Parse configuration and extract properties
    +   *
    +   * @param conf
    +   *          server configuration
    +   */
    +  protected void parseConfiguration(ServerConfiguration conf) {
    +    tableIdToTableName = new HashMap<>();
    +    poolNameToRegexPattern = new HashMap<>();
    +    for (Entry<String,String> table : getTableOperations().tableIdMap().entrySet()) {
    +      tableIdToTableName.put(table.getValue(), table.getKey());
    +      Map<String,String> customProps = conf.getTableConfiguration(table.getValue()).getAllPropertiesWithPrefix(Property.TABLE_ARBITRARY_PROP_PREFIX);
    +      if (null != customProps && customProps.size() > 0) {
    +        for (Entry<String,String> customProp : customProps.entrySet()) {
    +          String tableName = customProp.getKey().substring(HOST_BALANCER_PREFIX.length());
    +          String regex = customProp.getValue();
    +          poolNameToRegexPattern.put(tableName, Pattern.compile(regex));
    +        }
    +      }
    +    }
    +    String oobProperty = conf.getConfiguration().get(HOST_BALANCER_OOB_CHECK);
    +    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);
    +    if (null != ipBased) {
    +      isIpBasedRegex = Boolean.parseBoolean(ipBased);
    +    }
    +  }
    +
    +  public Map<String,String> getTableIdToTableName() {
    +    return tableIdToTableName;
    +  }
    +
    +  public Map<String,Pattern> getPoolNameToRegexPattern() {
    +    return poolNameToRegexPattern;
    +  }
    +
    +  public long getOobCheckMillis() {
    +    return oobCheckMillis;
    +  }
    +
    +  public long getPoolRecheckMillis() {
    +    return poolRecheckMillis;
    +  }
    +
    +  public boolean isIpBasedRegex() {
    +    return isIpBasedRegex;
    +  }
    +
    +  @Override
    +  public void init(ServerConfiguration conf) {
    +    super.init(conf);
    +    parseConfiguration(conf);
    +  }
    +
    +  @Override
    +  public void getAssignments(SortedMap<TServerInstance,TabletServerStatus> current, Map<KeyExtent,TServerInstance> unassigned,
    +      Map<KeyExtent,TServerInstance> assignments) {
    +
    +    if ((System.currentTimeMillis() - this.lastOOBCheck) > this.oobCheckMillis) {
    +      // 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 assignedPool : getPoolNamesForHost(e.getKey().host())) {
    +          for (String pool : poolNameToRegexPattern.keySet()) {
    +            if (assignedPool.equals(pool) || pool.equals(DEFAULT_POOL)) {
    +              continue;
    +            }
    +            String tid = getTableOperations().tableIdMap().get(pool);
    +            try {
    +              List<TabletStats> outOfBoundsTablets = getOnlineTabletsForTable(e.getKey(), tid);
    +              for (TabletStats ts : outOfBoundsTablets) {
    +                LOG.info("Tablet {} is currently outside the bounds of the regex, reassigning", ts.toString());
    +                unassigned.put(new KeyExtent(ts.getExtent()), e.getKey());
    +              }
    +            } catch (TException e1) {
    +              LOG.error("Error in OOB check getting tablets for table {} from server {}", tid, e.getKey().host(), e);
    +            }
    +          }
    +        }
    +      }
    +      this.oobCheckMillis = System.currentTimeMillis();
    +    }
    +
    +    Map<String,SortedMap<TServerInstance,TabletServerStatus>> pools = splitCurrentByRegex(current);
    +    // separate the unassigned into tables
    +    Map<String,Map<KeyExtent,TServerInstance>> groupedUnassigned = new HashMap<String,Map<KeyExtent,TServerInstance>>();
    +    for (Entry<KeyExtent,TServerInstance> e : unassigned.entrySet()) {
    +      Map<KeyExtent,TServerInstance> tableUnassigned = groupedUnassigned.get(e.getKey().getTableId());
    +      if (tableUnassigned == null) {
    +        tableUnassigned = new HashMap<KeyExtent,TServerInstance>();
    +        groupedUnassigned.put(e.getKey().getTableId(), tableUnassigned);
    +      }
    +      tableUnassigned.put(e.getKey(), e.getValue());
    +    }
    +    // Send a view of the current servers to the tables tablet balancer
    +    for (Entry<String,Map<KeyExtent,TServerInstance>> e : groupedUnassigned.entrySet()) {
    +      Map<KeyExtent,TServerInstance> newAssignments = new HashMap<KeyExtent,TServerInstance>();
    +      String tableName = tableIdToTableName.get(e.getKey());
    +      String poolName = getPoolNameForTable(tableName);
    +      SortedMap<TServerInstance,TabletServerStatus> currentView = pools.get(poolName);
    +      getBalancerForTable(e.getKey()).getAssignments(currentView, e.getValue(), newAssignments);
    --- End diff --
    
    Many balancers will assign to the last known location (the entry part of the unassigned map). This code isn't doing any verification to ensure that the unassigned value is a tserver that corresponds to the pool for the table. That is, when you add to unassigned in the oob check above and when you group the unassigned, you should check to make sure the last value is valid (and I guess clear it if it is a server that doesn't belong to the target pool).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---