Return-Path: X-Original-To: apmail-accumulo-dev-archive@www.apache.org Delivered-To: apmail-accumulo-dev-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 7332A19F87 for ; Tue, 12 Apr 2016 15:07:54 +0000 (UTC) Received: (qmail 93040 invoked by uid 500); 12 Apr 2016 15:07:54 -0000 Delivered-To: apmail-accumulo-dev-archive@accumulo.apache.org Received: (qmail 92998 invoked by uid 500); 12 Apr 2016 15:07:54 -0000 Mailing-List: contact dev-help@accumulo.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@accumulo.apache.org Delivered-To: mailing list dev@accumulo.apache.org Received: (qmail 92953 invoked by uid 99); 12 Apr 2016 15:07:54 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 12 Apr 2016 15:07:54 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id E6815DFCDF; Tue, 12 Apr 2016 15:07:53 +0000 (UTC) From: dlmarion To: dev@accumulo.apache.org Reply-To: dev@accumulo.apache.org References: In-Reply-To: Subject: [GitHub] accumulo pull request: ACCUMULO-4173: Host Regex Table Load Balanc... Content-Type: text/plain Message-Id: <20160412150753.E6815DFCDF@git1-us-west.apache.org> Date: Tue, 12 Apr 2016 15:07:53 +0000 (UTC) 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.
+ * Regex properties for this balancer are specified as:
+ * table.custom.balancer.host.regex.<tablename>=<regex>
+ * 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:
+ * table.custom.balancer.host.regex.oob.period=5m
+ * 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:
+ * table.custom.balancer.host.regex.pool.check=5m
+ * 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:
+ * table.custom.balancer.host.regex.is.ip=true + * + */ +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 tableIdToTableName = null; + private Map poolNameToRegexPattern = null; + private volatile long lastOOBCheck = System.currentTimeMillis(); + private volatile long lastPoolRecheck = 0; + private boolean isIpBasedRegex = false; + private Map> pools = new HashMap>(); + + /** + * 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> splitCurrentByRegex(SortedMap current) { + if ((System.currentTimeMillis() - lastPoolRecheck) > poolRecheckMillis) { + LOG.debug("Performing pool recheck - regrouping tablet servers into pools based on regex"); + Map> newPools = new HashMap>(); + for (Entry e : current.entrySet()) { + List poolNames = getPoolNamesForHost(e.getKey().host()); + for (String pool : poolNames) { + SortedMap np = newPools.get(pool); + if (null == np) { + np = new TreeMap(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 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 pools = new ArrayList<>(); + for (Entry 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 table : getTableOperations().tableIdMap().entrySet()) { + tableIdToTableName.put(table.getValue(), table.getKey()); + Map customProps = conf.getTableConfiguration(table.getValue()).getAllPropertiesWithPrefix(Property.TABLE_ARBITRARY_PROP_PREFIX); + if (null != customProps && customProps.size() > 0) { + for (Entry 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 getTableIdToTableName() { + return tableIdToTableName; + } + + public Map 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 current, Map unassigned, + Map assignments) { + + Map> 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. ---