hbase-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Todd Lipcon <t...@cloudera.com>
Subject Re: Config Properties Review
Date Tue, 23 Nov 2010 17:32:35 GMT
wow, nice list.

Can we move this to a JIRA? I agree we should document many of those,
whether in hbase-default or in some kind of 'advanced tunables' doc.

On Tue, Nov 23, 2010 at 6:19 AM, Lars George <lars.george@gmail.com> wrote:

> And for those in the code but *not* in hbase-default.xml:
>
> hbase.admin.scanner.caching
> src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java:    final
> int batchCount = this.conf.getInt("hbase.admin.scanner.caching", 10);
>
> hbase.balancer.period
> src/main/java/org/apache/hadoop/hbase/master/HMaster.java:    int
> period = master.getConfiguration().getInt("hbase.balancer.period",
> 300000);
>
> I guess that is the "hbase.master.balancer.period" I asked about
> below? So there is a typo on one side of this, code or XML?
>
> hbase.bulk.assignment.threadpool.size
> src/main/java/org/apache/hadoop/hbase/master/BulkAssigner.java:
> getInt("hbase.bulk.assignment.threadpool.size", 20);
>
> hbase.bulk.assignment.waiton.empty.rit
> src/main/java/org/apache/hadoop/hbase/master/BulkAssigner.java:
> getLong("hbase.bulk.assignment.waiton.empty.rit", 10 * 60 * 1000);
>
> hbase.catalog.verification.timeout
> src/main/java/org/apache/hadoop/hbase/master/HMaster.java:    long
> timeout = this.conf.getLong("hbase.catalog.verification.timeout",
> 1000);
>
> hbase.catalogjanitor.interval
> src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java:
> server.getConfiguration().getInt("hbase.catalogjanitor.interval",
> 300000),
>
> hbase.client.catalog.timeout
> src/main/java/org/apache/hadoop/hbase/master/HMaster.java:    int
> timeout = conf.getInt("hbase.client.catalog.timeout", 10000);
>
> hbase.client.prefetch.limit
> src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java:
>    this.prefetchRegionLimit =
> conf.getInt("hbase.client.prefetch.limit",
> src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide.java:
>  int prefetchRegionNumber =
> conf.getInt("hbase.client.prefetch.limit", 10);
>
> hbase.client.rpc.maxattempts
> src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java:
>    this.maxRPCAttempts = conf.getInt("hbase.client.rpc.maxattempts",
> 1);
>
> hbase.client.scanner.max.result.size
> src/main/java/org/apache/hadoop/hbase/HConstants.java:  public static
> String HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE_KEY =
> "hbase.client.scanner.max.result.size";
>
> hbase.fs.shutdown.hook.wait
> src/main/java/org/apache/hadoop/hbase/regionserver/ShutdownHook.java:
> public static final String FS_SHUTDOWN_HOOK_WAIT =
> "hbase.fs.shutdown.hook.wait";
>
> hbase.hlog.split.batch.size
> src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java:
>  * in batches of (hbase.hlog.split.batch.size)
> src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java:
>   int logFilesPerStep = conf.getInt("hbase.hlog.split.batch.size",
> 3);
>
> hbase.hlog.split.skip.errors
> src/docbkx/book.xml:
> <title><varname>hbase.hlog.split.skip.errors</varname></title>
> src/docbkx/book.xml:            When hbase.hlog.split.skip.errors is
> set to false, we fail the
> src/docbkx/book.xml:        even when
> <varname>hbase.hlog.split.skip.errors</varname> ==
> src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java:
>   boolean skipErrors =
> conf.getBoolean("hbase.hlog.split.skip.errors", false);
>
> The above is only documented on book.xml but not anywhere else.
>
> hbase.hlog.splitter.impl
> src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java:
>  private static final String LOG_SPLITTER_IMPL =
> "hbase.hlog.splitter.impl";
>
> hbase.hregion.impl
> src/main/java/org/apache/hadoop/hbase/HConstants.java:  public static
> final String REGION_IMPL = "hbase.hregion.impl";
>
> hbase.hregion.majorcompaction.jitter
> src/main/java/org/apache/hadoop/hbase/regionserver/Store.java:
> double jitterPct =
> conf.getFloat("hbase.hregion.majorcompaction.jitter",
>
> hbase.hregion.memstore.optionalflushcount
> src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java:
>
> TEST_UTIL.getConfiguration().setInt("hbase.hregion.memstore.optionalflushcount",
> 2);
>
> The above is used only in a test. But it does not have anyone reading it.
>
> hbase.hstore.close.check.interval
> src/main/java/org/apache/hadoop/hbase/regionserver/Store.java:
>  "hbase.hstore.close.check.interval", 10*1000*1000 /* 10 MB */);
>
> hbase.hstore.compaction.ratio
> src/main/java/org/apache/hadoop/hbase/regionserver/Store.java:
> this.compactRatio = conf.getFloat("hbase.hstore.compaction.ratio",
> 1.2F);
>
> hbase.hstore.report.interval.edits
> src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java:
> int interval = this.conf.getInt("hbase.hstore.report.interval.edits",
> 2000);
>
> hbase.htable.threads.max
> src/main/java/org/apache/hadoop/hbase/client/HTable.java:    int
> nrThreads = conf.getInt("hbase.htable.threads.max",
> getCurrentNrHRS());
>
> hbase.ipc.client.connect.max.retries
> src/main/java/org/apache/hadoop/hbase/ipc/HBaseClient.java:
> this.maxRetries = conf.getInt("hbase.ipc.client.connect.max.retries",
> 0);
>
> hbase.ipc.client.connection.maxidletime
> src/main/java/org/apache/hadoop/hbase/ipc/HBaseClient.java:
> conf.getInt("hbase.ipc.client.connection.maxidletime", 10000); //10s
>
> hbase.ipc.client.tcpkeepalive
> src/main/java/org/apache/hadoop/hbase/ipc/HBaseClient.java:
> this.tcpKeepAlive = conf.getBoolean("hbase.ipc.client.tcpkeepalive",
> true);
>
> hbase.ipc.client.tcpnodelay
> src/main/java/org/apache/hadoop/hbase/ipc/HBaseClient.java:
> this.tcpNoDelay = conf.getBoolean("hbase.ipc.client.tcpnodelay",
> false);
>
> hbase.ipc.warn.response.size
> src/main/java/org/apache/hadoop/hbase/ipc/HBaseServer.java:
> "hbase.ipc.warn.response.size";
>
> hbase.master.assignment.timeoutmonitor.period
> src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java:
>   conf.getInt("hbase.master.assignment.timeoutmonitor.period",
> 10000),
>
> hbase.master.assignment.timeoutmonitor.timeout
> src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java:
>   conf.getInt("hbase.master.assignment.timeoutmonitor.timeout",
> 30000));
>
> hbase.master.backup
> src/main/java/org/apache/hadoop/hbase/HConstants.java:  public static
> final String MASTER_TYPE_BACKUP = "hbase.master.backup";
>
> hbase.master.catalog.timeout
> src/main/java/org/apache/hadoop/hbase/master/HMaster.java:      this,
> conf.getInt("hbase.master.catalog.timeout", Integer.MAX_VALUE));
>
> hbase.master.cleaner.interval
> src/main/java/org/apache/hadoop/hbase/master/HMaster.java:        new
> LogCleaner(conf.getInt("hbase.master.cleaner.interval", 60 * 1000),
>
> hbase.master.executor.closeregion.threads
> src/main/java/org/apache/hadoop/hbase/master/HMaster.java:
> conf.getInt("hbase.master.executor.closeregion.threads", 5));
>
> hbase.master.executor.openregion.threads
> src/main/java/org/apache/hadoop/hbase/master/HMaster.java:
> conf.getInt("hbase.master.executor.openregion.threads", 5));
>
> hbase.master.executor.serverops.threads
> src/main/java/org/apache/hadoop/hbase/master/HMaster.java:
> conf.getInt("hbase.master.executor.serverops.threads", 3));
> src/main/java/org/apache/hadoop/hbase/master/HMaster.java:
> conf.getInt("hbase.master.executor.serverops.threads", 2));
>
> hbase.master.impl
> src/main/java/org/apache/hadoop/hbase/HConstants.java:  public static
> final String MASTER_IMPL= "hbase.master.impl";
>
> hbase.master.logcleaner.maxdeletedlogs
> src/main/java/org/apache/hadoop/hbase/master/LogCleaner.java:
> conf.getInt("hbase.master.logcleaner.maxdeletedlogs", 20);
>
> hbase.master.maxclockskew
> src/main/java/org/apache/hadoop/hbase/master/ServerManager.java:
> maxSkew = c.getLong("hbase.master.maxclockskew", 30000);
>
> hbase.master.startup.retainassign
> src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java:
>   getBoolean("hbase.master.startup.retainassign", true);
>
>
> hbase.master.wait.on.region
>
> src/main/java/org/apache/hadoop/hbase/master/handler/DeleteTableHandler.java:
>     getLong("hbase.master.wait.on.region", 5 * 60 * 1000);
> src/main/java/org/apache/hadoop/hbase/master/ServerManager.java:
> getLong("hbase.master.wait.on.regionservers.interval", 3000);
>
> hbase.master.wait.on.regionservers.interval
> src/main/java/org/apache/hadoop/hbase/master/ServerManager.java:
> getLong("hbase.master.wait.on.regionservers.interval", 3000);
>
> hbase.meta.scanner.caching
> src/main/java/org/apache/hadoop/hbase/client/MetaScanner.java:
> configuration.getInt("hbase.meta.scanner.caching", 100));
>
> hbase.regions.server.count.min
> src/main/java/org/apache/hadoop/hbase/master/HMasterCommandLine.java:
>    getConf().setInt("hbase.regions.server.count.min",
>
> hbase.regions.slop
> src/test/java/org/apache/hadoop/hbase/TestRegionRebalancing.java:
> float slop = conf.getFloat("hbase.regions.slop", (float)0.1);
>
> hbase.regionserver.catalog.timeout
> src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java:
>     this, this.conf.getInt("hbase.regionserver.catalog.timeout",
> Integer.MAX_VALUE));
>
> hbase.regionserver.codec
> src/docbkx/book.xml:      <link
> linkend="hbase.regionserver.codec">hbase.regionserver.codec</link>
> src/docbkx/book.xml:    <section id="hbase.regionserver.codec">
> src/docbkx/book.xml:    hbase.regionserver.codec
> src/docbkx/book.xml:    hbase.regionserver.codec
> src/docbkx/book.xml:    hbase.regionserver.codec
> src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java:
>   String [] codecs = conf.getStrings("hbase.regionserver.codecs",
>
> Above is a typo in the book.xml?
>
> hbase.regionserver.codecs
> src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java:
>   String [] codecs = conf.getStrings("hbase.regionserver.codecs",
>
> hbase.regionserver.executor.closemeta.threads
> src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java:
>     conf.getInt("hbase.regionserver.executor.closemeta.threads", 1));
>
> hbase.regionserver.executor.closeregion.threads
> src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java:
>     conf.getInt("hbase.regionserver.executor.closeregion.threads",
> 3));
>
> hbase.regionserver.executor.closeroot.threads
> src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java:
>     conf.getInt("hbase.regionserver.executor.closeroot.threads", 1));
>
> hbase.regionserver.executor.openmeta.threads
> src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java:
>     conf.getInt("hbase.regionserver.executor.openmeta.threads", 1));
>
> hbase.regionserver.executor.openregion.threads
> src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java:
>     conf.getInt("hbase.regionserver.executor.openregion.threads",
> 3));
>
> hbase.regionserver.executor.openroot.threads
> src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java:
>     conf.getInt("hbase.regionserver.executor.openroot.threads", 1));
>
> hbase.regionserver.hlog.blocksize
> src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java:
> this.blocksize = conf.getLong("hbase.regionserver.hlog.blocksize",
>
> src/main/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogWriter.java:
>     conf.getLong("hbase.regionserver.hlog.blocksize",
>
> hbase.regionserver.hlog.enabled
> src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java:
> this.enabled = conf.getBoolean("hbase.regionserver.hlog.enabled",
> true);
>
> hbase.regionserver.hlog.keyclass
> src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java:
>  conf.getClass("hbase.regionserver.hlog.keyclass", HLogKey.class);
>
> hbase.regionserver.hlog.replication
>
> src/main/java/org/apache/hadoop/hbase/regionserver/wal/SequenceFileLogWriter.java:
>     (short) conf.getInt("hbase.regionserver.hlog.replication",
>
> hbase.regionserver.hlog.splitlog.corrupt.dir
> src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java:
>       "hbase.regionserver.hlog.splitlog.corrupt.dir", ".corrupt"));
>
> hbase.regionserver.hlog.splitlog.writer.threads
> src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java:
>       "hbase.regionserver.hlog.splitlog.writer.threads", 3);
>
> hbase.regionserver.impl
> src/main/java/org/apache/hadoop/hbase/HConstants.java:  public static
> final String REGION_SERVER_IMPL= "hbase.regionserver.impl";
>
> hbase.regionserver.logroll.multiplier
> src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java:
> float multi = conf.getFloat("hbase.regionserver.logroll.multiplier",
> 0.95f);
>
> hbase.regionserver.maxlogs
> CHANGES.txt:   HBASE-3241  check to see if we exceeded
> hbase.regionserver.maxlogs limit is
> src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java:
> this.maxLogs = conf.getInt("hbase.regionserver.maxlogs", 32);
>
> src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java:
>           conf.getInt("hbase.regionserver.maxlogs", 32),
>
> hbase.regionserver.metahandler.count
> src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java:
>       conf.getInt("hbase.regionserver.metahandler.count", 10),
>
> hbase.regionserver.numregionstoreport
> src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java:
>       "hbase.regionserver.numregionstoreport", 10);
>
> hbase.regionserver.split.daughter.open.log.interval
> src/main/java/org/apache/hadoop/hbase/regionserver/SplitTransaction.java:
>     this.interval =
> c.getLong("hbase.regionserver.split.daughter.open.log.interval",
>
> hbase.rest.client.max.retries
> src/main/java/org/apache/hadoop/hbase/rest/client/RemoteAdmin.java:
> this.maxRetries = conf.getInt("hbase.rest.client.max.retries", 10);
> src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java:
>  this.maxRetries = conf.getInt("hbase.rest.client.max.retries", 10);
>
> hbase.rest.client.sleep
> src/main/java/org/apache/hadoop/hbase/rest/client/RemoteAdmin.java:
> this.sleepTime = conf.getLong("hbase.rest.client.sleep", 1000);
> src/main/java/org/apache/hadoop/hbase/rest/client/RemoteHTable.java:
>  this.sleepTime = conf.getLong("hbase.rest.client.sleep", 1000);
>
> hbase.rest.transform.check.interval
> src/main/java/org/apache/hadoop/hbase/rest/TableResource.java:    //
> Setting hbase.rest.transform.check.interval to <= 0 disables
> rescanning.
> src/main/java/org/apache/hadoop/hbase/rest/TableResource.java:
> .getLong("hbase.rest.transform.check.interval", 60000);
>
> hbase.rpc.timeout
> src/main/java/org/apache/hadoop/hbase/HConstants.java:  public static
> String HBASE_RPC_TIMEOUT_KEY = "hbase.rpc.timeout";
>
> hbase.shutdown.hook
> src/main/java/org/apache/hadoop/hbase/regionserver/ShutdownHook.java:
> public static final String RUN_SHUTDOWN_HOOK = "hbase.shutdown.hook";
>
> hbase.skip.errors
> src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java:
>  boolean skipErrors = conf.getBoolean("hbase.skip.errors", false);
> src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java:
>   LOG.error("hbase.skip.errors=true so continuing. Renamed " + edits
> +
> src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogSplitter.java:
>   boolean skipErrors = conf.getBoolean("hbase.skip.errors", false);
>
> hbase.zookeeper.property.tickTime
> src/main/java/org/apache/hadoop/hbase/master/HMasterCommandLine.java:
>
>  zooKeeperCluster.setTickTime(conf.getInt("hbase.zookeeper.property.tickTime",
> 3000));
>
> hbase.zookeeper.recoverable.waittime
> src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java:
>     long wait = conf.getLong("hbase.zookeeper.recoverable.waittime",
> 10000);
>
> hbase.zookeeper.watcher.sync.connected.wait
> src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java:
>         this.conf.getLong("hbase.zookeeper.watcher.sync.connected.wait",
> 2000);
>
> io.storefile.bloom.enabled
> src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java:
> static final String IO_STOREFILE_BLOOM_ENABLED =
> "io.storefile.bloom.enabled";
>
> io.storefile.bloom.error.rate
> src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java:
> static final String IO_STOREFILE_BLOOM_ERROR_RATE =
> "io.storefile.bloom.error.rate";
>
> io.storefile.bloom.max.fold
> src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java:
> static final String IO_STOREFILE_BLOOM_MAX_FOLD =
> "io.storefile.bloom.max.fold";
>
> io.storefile.bloom.max.keys
> src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java:
> static final String IO_STOREFILE_BLOOM_MAX_KEYS =
> "io.storefile.bloom.max.keys";
>
> ipc.client.connect.max.retries
> src/main/java/org/apache/hadoop/hbase/ipc/HBaseClient.java:
> this.maxRetries = conf.getInt("hbase.ipc.client.connect.max.retries",
> 0);
>
> ipc.client.connection.maxidletime
> src/main/java/org/apache/hadoop/hbase/ipc/HBaseClient.java:
> conf.getInt("hbase.ipc.client.connection.maxidletime", 10000); //10s
> src/main/java/org/apache/hadoop/hbase/ipc/HBaseServer.java:
> this.maxIdleTime = 2*conf.getInt("ipc.client.connection.maxidletime",
> 1000);
>
> ipc.client.idlethreshold
> src/main/java/org/apache/hadoop/hbase/ipc/HBaseServer.java:
> this.thresholdIdleConnections =
> conf.getInt("ipc.client.idlethreshold", 4000);
>
> ipc.client.kill.max
> src/main/java/org/apache/hadoop/hbase/ipc/HBaseServer.java:
> this.maxConnectionsToNuke = conf.getInt("ipc.client.kill.max", 10);
>
> ipc.client.timeout
>
> src/test/java/org/apache/hadoop/hbase/regionserver/DisabledTestRegionServerExit.java:
>   conf.setInt("ipc.client.timeout", 10000);         // and ipc
> timeout
>
> ipc.ping.interval
> src/main/java/org/apache/hadoop/hbase/ipc/HBaseClient.java:  final
> private static String PING_INTERVAL_NAME = "ipc.ping.interval";
>
> ipc.server.listen.queue.size
> src/main/java/org/apache/hadoop/hbase/ipc/HBaseServer.java:    private
> int backlogLength = conf.getInt("ipc.server.listen.queue.size", 128);
>
> ipc.server.read.threadpool.size
> src/main/java/org/apache/hadoop/hbase/ipc/HBaseServer.java:
> "ipc.server.read.threadpool.size",
>
> ipc.server.tcpkeepalive
> src/main/java/org/apache/hadoop/hbase/ipc/HBaseServer.java:
> this.tcpKeepAlive = conf.getBoolean("ipc.server.tcpkeepalive", true);
>
> ipc.server.tcpnodelay
> src/main/java/org/apache/hadoop/hbase/ipc/HBaseServer.java:
> this.tcpNoDelay = conf.getBoolean("ipc.server.tcpnodelay", false);
>
> replication.replicationsource.implementation
>
> src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSourceManager.java:
>     Class c =
> Class.forName(conf.get("replication.replicationsource.implementation",
>
> replication.sink.htablepool.capacity
>
> src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSink.java:
>       conf.getInt("replication.sink.htablepool.capacity", 10));
>
> replication.source.maxretriesmultiplier
>
> src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java:
>       this.conf.getLong("replication.source.maxretriesmultiplier",
> 10);
>
> replication.source.nb.capacity
>
> src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java:
>       this.conf.getInt("replication.source.nb.capacity", 25000);
>
> replication.source.ratio
>
> src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java:
>   this.ratio = this.conf.getFloat("replication.source.ratio", 0.1f);
>
> replication.source.size.capacity
>
> src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java:
>       this.conf.getLong("replication.source.size.capacity",
> 1024*1024*64);
>
> replication.source.sleepforretries
>
> src/main/java/org/apache/hadoop/hbase/replication/regionserver/ReplicationSource.java:
>       this.conf.getLong("replication.source.sleepforretries", 1000);
>
> zookeeper.znode.master
> bin/rolling-restart.sh:  zmaster=`$bin/hbase
> org.apache.hadoop.hbase.util.HBaseConfTool zookeeper.znode.master`
> src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java:
>       conf.get("zookeeper.znode.master", "master"));
>
> zookeeper.znode.replication
>
> src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeper.java:
>       conf.get("zookeeper.znode.replication", "replication");
>
> zookeeper.znode.replication.clusterId
>
> src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeper.java:
>       conf.get("zookeeper.znode.replication.clusterId", "clusterId");
>
> zookeeper.znode.replication.master
>
> src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeper.java:
>       conf.get("zookeeper.znode.replication.master", "master");
>
> zookeeper.znode.replication.peers
>
> src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeper.java:
>       conf.get("zookeeper.znode.replication.peers", "peers");
>
> zookeeper.znode.replication.rs
>
> src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeper.java:
>       conf.get("zookeeper.znode.replication.rs", "rs");
>
> zookeeper.znode.replication.state
>
> src/main/java/org/apache/hadoop/hbase/replication/ReplicationZookeeper.java:
>       conf.get("zookeeper.znode.replication.state", "state");
>
> zookeeper.znode.rs
> src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java:
>       conf.get("zookeeper.znode.rs", "rs"));
>
> zookeeper.znode.state
> src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java:
>       conf.get("zookeeper.znode.state", "shutdown"));
>
> zookeeper.znode.tableEnableDisable
> src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java:
>       conf.get("zookeeper.znode.tableEnableDisable", "table"));
>
> zookeeper.znode.unassigned
> bin/rolling-restart.sh:  zunassigned=`$bin/hbase
> org.apache.hadoop.hbase.util.HBaseConfTool zookeeper.znode.unassigned`
> src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java:
>       conf.get("zookeeper.znode.unassigned", "unassigned"));
>
>
> That is all, seems like we have more undocumented settings than
> documented once. Would it make sense (yes! rhetorical question) to go
> through and see which should be documented in the hbase-default.xml.
> Plus a few fixes like typos and inconsistencies.
>
> Cheers,
> Lars
>
>
> On Tue, Nov 23, 2010 at 2:05 PM, Lars George <lars.george@gmail.com>
> wrote:
> > Hi,
> >
> > I went through the config values as per the defaults XML file (still
> > going through it again now based on what is actually in the code, i.e.
> > those not in defaults). Here is what I found:
> >
> > hbase.master.balancer.period - Only used in hbase-default.xml?
> >
> > hbase.regions.percheckin, hbase.regions.slop - Some tests still have
> > it but not used anywhere else
> >
> > zookeeper.pause, zookeeper.retries - Never used? Only in
> hbase-defaults.xml
> >
> >
> > And then there are differences between hardcoded and XML based defaults:
> >
> > hbase.client.pause - XML: 1000, hardcoded: 2000 (HBaseClient) and 30 *
> > 1000 (HBaseAdmin)
> >
> > hbase.client.retries.number - XML: 10, hardcoded 5 (HBaseAdmin) and 2
> (HMaster)
> >
> > hbase.hstore.blockingStoreFiles - XML: 7, hardcoded: -1
> >
> > hbase.hstore.compactionThreshold - XML: 3, hardcoded: 2
> >
> > hbase.regionserver.global.memstore.lowerLimit - XML: 0.35, hardcoded:
> 0.25
> >
> > hbase.regionserver.handler.count - XML: 25, hardcoded: 10
> >
> > hbase.regionserver.msginterval - XML: 3000, hardcoded: 1000
> >
> > hbase.rest.port - XML: 8080, hardcoded: 9090
> >
> > hfile.block.cache.size - XML: 0.2, hardcoded: 0.0
> >
> >
> > Finally, some keys are already in HConstants, some are in local
> > classes and others used as literals. There is an issue open to fix
> > this though. Just saying.
> >
> > Thoughts?
> >
> > Thanks,
> > Lars
> >
>



-- 
Todd Lipcon
Software Engineer, Cloudera

Mime
  • Unnamed multipart/alternative (inline, None, 0 bytes)
View raw message