Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 2DCCB200D63 for ; Thu, 21 Dec 2017 23:07:35 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id 2C5EC160C12; Thu, 21 Dec 2017 22:07:35 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id C7EE4160C2B for ; Thu, 21 Dec 2017 23:07:33 +0100 (CET) Received: (qmail 46777 invoked by uid 500); 21 Dec 2017 22:07:28 -0000 Mailing-List: contact commits-help@hbase.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@hbase.apache.org Delivered-To: mailing list commits@hbase.apache.org Received: (qmail 46768 invoked by uid 99); 21 Dec 2017 22:07:28 -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; Thu, 21 Dec 2017 22:07:27 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 34CDDDFC00; Thu, 21 Dec 2017 22:07:25 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: stack@apache.org To: commits@hbase.apache.org Message-Id: <8e8850d4d8514ad489a7b9939712f390@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: hbase git commit: HBASE-19148 Reevaluate default values of configurations Date: Thu, 21 Dec 2017 22:07:25 +0000 (UTC) archived-at: Thu, 21 Dec 2017 22:07:35 -0000 Repository: hbase Updated Branches: refs/heads/master 59baf12c0 -> 4d6b92868 HBASE-19148 Reevaluate default values of configurations Removed unused: hbase.fs.tmp.dir Added hbase.master.loadbalance.bytable Edit of description text. Moved stuff around to put configs beside each other. M hbase-server/src/main/java/org/apache/hadoop/hbase/util/ServerCommandLine.java Emit some hbase configs in log on startup. Signed-off-by: Michael Stack Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/4d6b9286 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/4d6b9286 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/4d6b9286 Branch: refs/heads/master Commit: 4d6b928682cc2a17f3dfd0179fb3fd46fd9e0a1f Parents: 59baf12 Author: Michael Stack Authored: Fri Dec 15 17:56:38 2017 -0800 Committer: Michael Stack Committed: Thu Dec 21 14:07:16 2017 -0800 ---------------------------------------------------------------------- .../src/main/resources/hbase-default.xml | 177 +++++++++++-------- .../org/apache/hadoop/hbase/master/HMaster.java | 2 +- .../assignment/SplitTableRegionProcedure.java | 7 +- .../hbase/regionserver/wal/AbstractFSWAL.java | 14 +- .../hadoop/hbase/util/ServerCommandLine.java | 28 ++- 5 files changed, 140 insertions(+), 88 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/hbase/blob/4d6b9286/hbase-common/src/main/resources/hbase-default.xml ---------------------------------------------------------------------- diff --git a/hbase-common/src/main/resources/hbase-default.xml b/hbase-common/src/main/resources/hbase-default.xml index 7995c41..b42ce3f 100644 --- a/hbase-common/src/main/resources/hbase-default.xml +++ b/hbase-common/src/main/resources/hbase-default.xml @@ -63,13 +63,6 @@ possible configurations would overwhelm and obscure the important. machine restart. - hbase.fs.tmp.dir - /user/${user.name}/hbase-staging - A staging directory in default file system (HDFS) - for keeping temporary data. - - - hbase.cluster.distributed false The mode the cluster will be in. Possible values are @@ -146,8 +139,8 @@ possible configurations would overwhelm and obscure the important. hbase.master.procedurewalcleaner.ttl 604800000 - How long a Procedure WAL stays will remain in the - {hbase.rootdir}/oldWALs/masterProcedureWALs directory, after which it will be cleaned + How long a Procedure WAL will remain in the + {hbase.rootdir}/MasterProcedureWALs directory, after which it will be cleaned by a Master thread. The value is in milliseconds. @@ -166,7 +159,19 @@ possible configurations would overwhelm and obscure the important. true Whether or not the Master listens to the Master web UI port (hbase.master.info.port) and redirects requests to the web - UI server shared by the Master and RegionServer. + UI server shared by the Master and RegionServer. Config. makes + sense when Master is serving Regions (not the default). + + + hbase.master.fileSplitTimeout + 600000 + Splitting a region, how long to wait on the file-splitting + step before aborting the attempt. Default: 600000. This setting used + to be known as hbase.regionserver.fileSplitTimeout in hbase-1.x. + Split is now run master-side hence the rename (If a + 'hbase.master.fileSplitTimeout' setting found, will use it to + prime the current 'hbase.master.fileSplitTimeout' + Configuration. @@ -198,7 +203,10 @@ possible configurations would overwhelm and obscure the important. hbase.regionserver.handler.count 30 Count of RPC Listener instances spun up on RegionServers. - Same property is used by the Master for count of master handlers. + Same property is used by the Master for count of master handlers. + Too many handlers can be counter-productive. Make it a multiple of + CPU count. If mostly read-only, handlers count close to cpu count + does well. Start with twice the CPU count and tune from there. hbase.ipc.server.callqueue.handler.factor @@ -292,31 +300,37 @@ possible configurations would overwhelm and obscure the important. Updates are blocked and flushes are forced until size of all memstores in a region server hits hbase.regionserver.global.memstore.size.lower.limit. The default value in this configuration has been intentionally left empty in order to - honor the old hbase.regionserver.global.memstore.upperLimit property if present. + honor the old hbase.regionserver.global.memstore.upperLimit property if present. + hbase.regionserver.global.memstore.size.lower.limit - Maximum size of all memstores in a region server before flushes are forced. - Defaults to 95% of hbase.regionserver.global.memstore.size (0.95). - A 100% value for this value causes the minimum possible flushing to occur when updates are - blocked due to memstore limiting. - The default value in this configuration has been intentionally left empty in order to - honor the old hbase.regionserver.global.memstore.lowerLimit property if present. + Maximum size of all memstores in a region server before flushes + are forced. Defaults to 95% of hbase.regionserver.global.memstore.size + (0.95). A 100% value for this value causes the minimum possible flushing + to occur when updates are blocked due to memstore limiting. The default + value in this configuration has been intentionally left empty in order to + honor the old hbase.regionserver.global.memstore.lowerLimit property if + present. + hbase.systemtables.compacting.memstore.type NONE - Determines the type of memstore to be used for system tables like META, namespace tables etc. - By default NONE is the type and hence we use the default memstore for all the system tables. If we - need to use compacting memstore for system tables then set this property to BASIC/EAGER + Determines the type of memstore to be used for system tables like + META, namespace tables etc. By default NONE is the type and hence we use the + default memstore for all the system tables. If we need to use compacting + memstore for system tables then set this property to BASIC/EAGER + hbase.regionserver.optionalcacheflushinterval 3600000 Maximum amount of time an edit lives in memory before being automatically flushed. - Default 1 hour. Set it to 0 to disable automatic flushing. + Default 1 hour. Set it to 0 to disable automatic flushing. + hbase.regionserver.dns.interface @@ -335,19 +349,21 @@ possible configurations would overwhelm and obscure the important. hbase.regionserver.region.split.policy org.apache.hadoop.hbase.regionserver.SteppingSplitPolicy - A split policy determines when a region should be split. The various other split policies that - are available currently are BusyRegionSplitPolicy, ConstantSizeRegionSplitPolicy, DisabledRegionSplitPolicy, - DelimitedKeyPrefixRegionSplitPolicy, KeyPrefixRegionSplitPolicy, and SteppingSplitPolicy. - DisabledRegionSplitPolicy blocks manual region splitting. + A split policy determines when a region should be split. The various + other split policies that are available currently are BusyRegionSplitPolicy, + ConstantSizeRegionSplitPolicy, DisabledRegionSplitPolicy, + DelimitedKeyPrefixRegionSplitPolicy, KeyPrefixRegionSplitPolicy, and + SteppingSplitPolicy. DisabledRegionSplitPolicy blocks manual region splitting. hbase.regionserver.regionSplitLimit 1000 - Limit for the number of regions after which no more region splitting should take place. - This is not hard limit for the number of regions but acts as a guideline for the regionserver - to stop splitting after a certain limit. Default is set to 1000. + Limit for the number of regions after which no more region splitting + should take place. This is not hard limit for the number of regions + but acts as a guideline for the regionserver to stop splitting after + a certain limit. Default is set to 1000. @@ -357,14 +373,15 @@ possible configurations would overwhelm and obscure the important. 90000 ZooKeeper session timeout in milliseconds. It is used in two different ways. First, this value is used in the ZK client that HBase uses to connect to the ensemble. - It is also used by HBase when it starts a ZK server and it is passed as the 'maxSessionTimeout'. See - http://hadoop.apache.org/zookeeper/docs/current/zookeeperProgrammers.html#ch_zkSessions. + It is also used by HBase when it starts a ZK server and it is passed as the 'maxSessionTimeout'. + See http://hadoop.apache.org/zookeeper/docs/current/zookeeperProgrammers.html#ch_zkSessions. For example, if an HBase region server connects to a ZK ensemble that's also managed - by HBase, then the - session timeout will be the one specified by this configuration. But, a region server that connects - to an ensemble managed with a different configuration will be subjected that ensemble's maxSessionTimeout. So, - even though HBase might propose using 90 seconds, the ensemble can have a max timeout lower than this and - it will take precedence. The current default that ZK ships with is 40 seconds, which is lower than HBase's. + by HBase, then the session timeout will be the one specified by this configuration. + But, a region server that connects to an ensemble managed with a different configuration + will be subjected that ensemble's maxSessionTimeout. So, even though HBase might propose + using 90 seconds, the ensemble can have a max timeout lower than this and it will take + precedence. The current default that ZK ships with is 40 seconds, which is lower than + HBase's. @@ -373,7 +390,8 @@ possible configurations would overwhelm and obscure the important. Root ZNode for HBase in ZooKeeper. All of HBase's ZooKeeper files that are configured with a relative path will go under this node. By default, all of HBase's ZooKeeper file paths are configured with a - relative path, so they will all go under this directory unless changed. + relative path, so they will all go under this directory unless changed. + zookeeper.znode.acl.parent @@ -1120,6 +1138,26 @@ possible configurations would overwhelm and obscure the important. HBase shell. + hbase.coprocessor.master.classes + + A comma-separated list of + org.apache.hadoop.hbase.coprocessor.MasterObserver coprocessors that are + loaded by default on the active HMaster process. For any implemented + coprocessor methods, the listed classes will be called in order. After + implementing your own MasterObserver, just put it in HBase's classpath + and add the fully qualified class name here. + + + hbase.coprocessor.abortonerror + true + Set to true to cause the hosting server (master or regionserver) + to abort if a coprocessor fails to load, fails to initialize, or throws an + unexpected Throwable object. Setting this to false will allow the server to + continue execution but the system wide state of the coprocessor in question + will become inconsistent as it will be properly executing in only a subset + of servers, so this is most useful for debugging only. + + hbase.rest.port 8080 The port for the HBase REST server. @@ -1172,26 +1210,6 @@ possible configurations would overwhelm and obscure the important. version is X.X.X-SNAPSHOT" - hbase.coprocessor.master.classes - - A comma-separated list of - org.apache.hadoop.hbase.coprocessor.MasterObserver coprocessors that are - loaded by default on the active HMaster process. For any implemented - coprocessor methods, the listed classes will be called in order. After - implementing your own MasterObserver, just put it in HBase's classpath - and add the fully qualified class name here. - - - hbase.coprocessor.abortonerror - true - Set to true to cause the hosting server (master or regionserver) - to abort if a coprocessor fails to load, fails to initialize, or throws an - unexpected Throwable object. Setting this to false will allow the server to - continue execution but the system wide state of the coprocessor in question - will become inconsistent as it will be properly executing in only a subset - of servers, so this is most useful for debugging only. - - hbase.table.lock.enable true Set to true to enable locking the table in zookeeper for schema change operations. @@ -1388,7 +1406,6 @@ possible configurations would overwhelm and obscure the important. are NULL, CRC32, CRC32C. - hbase.client.scanner.max.result.size 2097152 @@ -1398,7 +1415,6 @@ possible configurations would overwhelm and obscure the important. With faster and/or high latency networks this value should be increased. - hbase.server.scanner.max.result.size 104857600 @@ -1408,7 +1424,6 @@ possible configurations would overwhelm and obscure the important. This is a safety setting to protect the server from OOM situations. - hbase.status.published false @@ -1447,7 +1462,6 @@ possible configurations would overwhelm and obscure the important. Multicast port to use for the status publication by multicast. - hbase.dynamic.jars.dir ${hbase.rootdir}/lib @@ -1487,6 +1501,13 @@ possible configurations would overwhelm and obscure the important. + hbase.master.loadbalance.bytable + false + Factor Table name when the balancer runs. + Default: false. + + + hbase.master.normalizer.class org.apache.hadoop.hbase.master.normalizer.SimpleRegionNormalizer @@ -1604,8 +1625,8 @@ possible configurations would overwhelm and obscure the important. hbase.security.visibility.mutations.checkauths false - This property if enabled, will check whether the labels in the visibility expression are associated - with the user issuing the mutation + This property if enabled, will check whether the labels in the visibility + expression are associated with the user issuing the mutation @@ -1633,9 +1654,9 @@ possible configurations would overwhelm and obscure the important. The maximum number of threads any replication source will use for shipping edits to the sinks in parallel. This also limits the number of - chunks each replication batch is broken into. - Larger values can improve the replication throughput between the master and - slave clusters. The default of 10 will rarely need to be changed. + chunks each replication batch is broken into. Larger values can improve + the replication throughput between the master and slave clusters. The + default of 10 will rarely need to be changed. @@ -1644,22 +1665,22 @@ possible configurations would overwhelm and obscure the important. By default, in replication we can not make sure the order of operations in slave cluster is same as the order in master. If set REPLICATION_SCOPE to 2, we will push edits by the order - of written. This configure is to set how long (in ms) we will wait before next checking if a - log can not push right now because there are some logs written before it have not been pushed. - A larger waiting will decrease the number of queries on hbase:meta but will enlarge the delay - of replication. This feature relies on zk-less assignment, so users must set + of written. This configuration is to set how long (in ms) we will wait before next checking if + a log can NOT be pushed because there are some logs written before it that have yet to be + pushed. A larger waiting will decrease the number of queries on hbase:meta but will enlarge + the delay of replication. This feature relies on zk-less assignment, so users must set hbase.assignment.usezk to false to support it. + hbase.http.staticuser.user + dr.stack The user name to filter as, on static web filters while rendering content. An example use is the HDFS web UI (user to be used for browsing files). - hbase.http.staticuser.user - dr.stack hbase.regionserver.handler.abort.on.error.percent @@ -1703,10 +1724,10 @@ possible configurations would overwhelm and obscure the important. 86400 The period that ExpiredMobFileCleanerChore runs. The unit is second. - The default value is one day. - The MOB file name uses only the date part of the file creation time in it. We use this - time for deciding TTL expiry of the files. So the removal of TTL expired files might be - delayed. The max delay might be 24 hrs. + The default value is one day. The MOB file name uses only the date part of + the file creation time in it. We use this time for deciding TTL expiry of + the files. So the removal of TTL expired files might be delayed. The max + delay might be 24 hrs. @@ -1764,14 +1785,14 @@ possible configurations would overwhelm and obscure the important. hbase.snapshot.master.timeout.millis 300000 - Timeout for master for the snapshot procedure execution + Timeout for master for the snapshot procedure execution. hbase.snapshot.region.timeout 300000 - Timeout for regionservers to keep threads in snapshot request pool waiting + Timeout for regionservers to keep threads in snapshot request pool waiting. http://git-wip-us.apache.org/repos/asf/hbase/blob/4d6b9286/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java index e31db82..d0f435b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java @@ -1399,7 +1399,7 @@ public class HMaster extends HRegionServer implements MasterServices { } } - boolean isByTable = getConfiguration().getBoolean("hbase.master.loadbalance.bytable", false); + boolean isByTable = getConfiguration().getBoolean("hbase.master.loadbalance.bytable", true); Map>> assignmentsByTable = this.assignmentManager.getRegionStates().getAssignmentsByTable(!isByTable); http://git-wip-us.apache.org/repos/asf/hbase/blob/4d6b9286/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java index 809d6f3..fd78604 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/SplitTableRegionProcedure.java @@ -609,8 +609,11 @@ public class SplitTableRegionProcedure // Shutdown the pool threadPool.shutdown(); - // Wait for all the tasks to finish - long fileSplitTimeout = conf.getLong("hbase.master.fileSplitTimeout", 30000); + // Wait for all the tasks to finish. + // When splits ran on the RegionServer, how-long-to-wait-configuration was named + // hbase.regionserver.fileSplitTimeout. If set, use its value. + long fileSplitTimeout = conf.getLong("hbase.master.fileSplitTimeout", + conf.getLong("hbase.regionserver.fileSplitTimeout", 600000)); try { boolean stillRunning = !threadPool.awaitTermination(fileSplitTimeout, TimeUnit.MILLISECONDS); if (stillRunning) { http://git-wip-us.apache.org/repos/asf/hbase/blob/4d6b9286/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java index baa7590..d0fe49b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java @@ -402,12 +402,18 @@ public abstract class AbstractFSWAL implements WAL { } this.coprocessorHost = new WALCoprocessorHost(this, conf); - // Get size to roll log at. Roll at 95% of HDFS block size so we avoid crossing HDFS blocks - // (it costs a little x'ing bocks) + // Schedule a WAL roll when the WAL is 50% of the HDFS block size. Scheduling at 50% of block + // size should make it so WAL rolls before we get to the end-of-block (Block transitions cost + // some latency). In hbase-1 we did this differently. We scheduled a roll when we hit 95% of + // the block size but experience from the field has it that this was not enough time for the + // roll to happen before end-of-block. So the new accounting makes WALs of about the same + // size as those made in hbase-1 (to prevent surprise), we now have default block size as + // 2 times the DFS default: i.e. 2 * DFS default block size rolling at 50% full will generally + // make similar size logs to 1 * DFS default block size rolling at 95% full. See HBASE-19148. final long blocksize = this.conf.getLong("hbase.regionserver.hlog.blocksize", - CommonFSUtils.getDefaultBlockSize(this.fs, this.walDir)); + CommonFSUtils.getDefaultBlockSize(this.fs, this.walDir) * 2); this.logrollsize = - (long) (blocksize * conf.getFloat("hbase.regionserver.logroll.multiplier", 0.95f)); + (long) (blocksize * conf.getFloat("hbase.regionserver.logroll.multiplier", 0.5f)); boolean maxLogsDefined = conf.get("hbase.regionserver.maxlogs") != null; if (maxLogsDefined) { http://git-wip-us.apache.org/repos/asf/hbase/blob/4d6b9286/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ServerCommandLine.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ServerCommandLine.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ServerCommandLine.java index 83ec5ff..f99a090 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ServerCommandLine.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ServerCommandLine.java @@ -26,14 +26,16 @@ import java.util.Locale; import java.util.Map.Entry; import java.util.Set; -import org.apache.yetus.audience.InterfaceAudience; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configured; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.ToolRunner; +import org.apache.yetus.audience.InterfaceAudience; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Base class for command lines that start up various HBase daemons. @@ -84,6 +86,23 @@ public abstract class ServerCommandLine extends Configured implements Tool { } /** + * Print into log some of the important hbase attributes. + */ + private static void logHBaseConfigs(Configuration conf) { + final String [] keys = new String [] { + // Expand this list as you see fit. + "hbase.tmp.dir", + HConstants.HBASE_DIR, + HConstants.CLUSTER_DISTRIBUTED, + HConstants.ZOOKEEPER_QUORUM, + + }; + for (String key: keys) { + LOG.info(key + ": " + conf.get(key)); + } + } + + /** * Logs information about the currently running JVM process including * the environment variables. Logging of env vars can be disabled by * setting {@code "hbase.envvars.logging.disabled"} to {@code "true"}. @@ -92,6 +111,8 @@ public abstract class ServerCommandLine extends Configured implements Tool { * to comma separated list of such substrings. */ public static void logProcessInfo(Configuration conf) { + logHBaseConfigs(conf); + // log environment variables unless asked not to if (conf == null || !conf.getBoolean("hbase.envvars.logging.disabled", false)) { Set skipWords = new HashSet<>(DEFAULT_SKIP_WORDS); @@ -114,6 +135,7 @@ public abstract class ServerCommandLine extends Configured implements Tool { LOG.info("env:"+entry); } } + // and JVM info logJVMInfo(); }