Repository: hbase Updated Branches: refs/heads/branch-1 c2af53be7 -> af141f2b0
HBASE-11492 Hadoop configuration overrides some ipc parameters including tcpNoDelay Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/af141f2b Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/af141f2b Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/af141f2b Branch: refs/heads/branch-1 Commit: af141f2b087ff0baecb10257f8792a33cae31615 Parents: c2af53b Author: Nicolas Liochon <[email protected]> Authored: Tue Jul 22 14:56:53 2014 +0200 Committer: Nicolas Liochon <[email protected]> Committed: Tue Jul 22 14:56:53 2014 +0200 ---------------------------------------------------------------------- .../src/main/resources/hbase-default.xml | 4 ++-- .../hadoop/hbase/ipc/FifoRpcScheduler.java | 2 +- .../org/apache/hadoop/hbase/ipc/RpcServer.java | 20 ++++++++++---------- .../hadoop/hbase/ipc/SimpleRpcScheduler.java | 11 ++++++----- .../AnnotationReadingPriorityFunction.java | 2 +- .../hadoop/hbase/regionserver/HRegion.java | 2 +- .../hadoop/hbase/TestFullLogReconstruction.java | 2 +- .../hadoop/hbase/regionserver/wal/TestHLog.java | 4 ++-- .../hbase/regionserver/wal/TestLogRolling.java | 1 - hbase-shell/src/main/ruby/hbase/hbase.rb | 2 +- src/main/docbkx/configuration.xml | 2 +- 11 files changed, 26 insertions(+), 26 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/hbase/blob/af141f2b/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 70d20a7..f7f0254 100644 --- a/hbase-common/src/main/resources/hbase-default.xml +++ b/hbase-common/src/main/resources/hbase-default.xml @@ -181,14 +181,14 @@ possible configurations would overwhelm and obscure the important. Same property is used by the Master for count of master handlers.</description> </property> <property> - <name>ipc.server.callqueue.handler.factor</name> + <name>hbase.ipc.server.callqueue.handler.factor</name> <value>0.1</value> <description>Factor to determine the number of call queues. A value of 0 means a single queue shared between all the handlers. A value of 1 means that each handler has its own queue.</description> </property> <property> - <name>ipc.server.callqueue.read.share</name> + <name>hbase.ipc.server.callqueue.read.share</name> <value>0</value> <description>Split the call queues into read and write queues. A value of 0 indicate to not split the call queues. http://git-wip-us.apache.org/repos/asf/hbase/blob/af141f2b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/FifoRpcScheduler.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/FifoRpcScheduler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/FifoRpcScheduler.java index 648ca1f..8140c1c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/FifoRpcScheduler.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/FifoRpcScheduler.java @@ -39,7 +39,7 @@ public class FifoRpcScheduler extends RpcScheduler { public FifoRpcScheduler(Configuration conf, int handlerCount) { this.handlerCount = handlerCount; - this.maxQueueLength = conf.getInt("ipc.server.max.callqueue.length", + this.maxQueueLength = conf.getInt("hbase.ipc.server.max.callqueue.length", handlerCount * RpcServer.DEFAULT_MAX_CALLQUEUE_LENGTH_PER_HANDLER); } http://git-wip-us.apache.org/repos/asf/hbase/blob/af141f2b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java index 40ec192..ef4df3e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java @@ -516,7 +516,7 @@ public class RpcServer implements RpcServerInterface { public Listener(final String name) throws IOException { super(name); - backlogLength = conf.getInt("ipc.server.listen.queue.size", 128); + backlogLength = conf.getInt("hbase.ipc.server.listen.queue.size", 128); // Create a new server socket and set to non blocking mode acceptChannel = ServerSocketChannel.open(); acceptChannel.configureBlocking(false); @@ -1704,7 +1704,7 @@ public class RpcServer implements RpcServerInterface { responder, totalRequestSize, null); ByteArrayOutputStream responseBuffer = new ByteArrayOutputStream(); setupResponse(responseBuffer, callTooBig, new CallQueueTooBigException(), - "Call queue is full, is ipc.server.max.callqueue.size too small?"); + "Call queue is full, is hbase.ipc.server.max.callqueue.size too small?"); responder.doRespond(callTooBig); return; } @@ -1868,12 +1868,12 @@ public class RpcServer implements RpcServerInterface { this.conf = conf; this.socketSendBufferSize = 0; this.maxQueueSize = - this.conf.getInt("ipc.server.max.callqueue.size", DEFAULT_MAX_CALLQUEUE_SIZE); - this.readThreads = conf.getInt("ipc.server.read.threadpool.size", 10); - this.maxIdleTime = 2 * conf.getInt("ipc.client.connection.maxidletime", 1000); - this.maxConnectionsToNuke = conf.getInt("ipc.client.kill.max", 10); - this.thresholdIdleConnections = conf.getInt("ipc.client.idlethreshold", 4000); - this.purgeTimeout = conf.getLong("ipc.client.call.purge.timeout", + this.conf.getInt("hbase.ipc.server.max.callqueue.size", DEFAULT_MAX_CALLQUEUE_SIZE); + this.readThreads = conf.getInt("hbase.ipc.server.read.threadpool.size", 10); + this.maxIdleTime = 2 * conf.getInt("hbase.ipc.client.connection.maxidletime", 1000); + this.maxConnectionsToNuke = conf.getInt("hbase.ipc.client.kill.max", 10); + this.thresholdIdleConnections = conf.getInt("hbase.ipc.client.idlethreshold", 4000); + this.purgeTimeout = conf.getLong("hbase.ipc.client.call.purge.timeout", 2 * HConstants.DEFAULT_HBASE_RPC_TIMEOUT); this.warnResponseTime = conf.getInt(WARN_RESPONSE_TIME, DEFAULT_WARN_RESPONSE_TIME); this.warnResponseSize = conf.getInt(WARN_RESPONSE_SIZE, DEFAULT_WARN_RESPONSE_SIZE); @@ -1883,8 +1883,8 @@ public class RpcServer implements RpcServerInterface { this.port = listener.getAddress().getPort(); this.metrics = new MetricsHBaseServer(name, new MetricsHBaseServerWrapperImpl(this)); - this.tcpNoDelay = conf.getBoolean("ipc.server.tcpnodelay", true); - this.tcpKeepAlive = conf.getBoolean("ipc.server.tcpkeepalive", true); + this.tcpNoDelay = conf.getBoolean("hbase.ipc.server.tcpnodelay", true); + this.tcpKeepAlive = conf.getBoolean("hbase.ipc.server.tcpkeepalive", true); this.warnDelayedCalls = conf.getInt(WARN_DELAYED_CALLS, DEFAULT_WARN_DELAYED_CALLS); this.delayedCalls = new AtomicInteger(0); http://git-wip-us.apache.org/repos/asf/hbase/blob/af141f2b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcScheduler.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcScheduler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcScheduler.java index 953bc36..0458c00 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcScheduler.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/SimpleRpcScheduler.java @@ -38,17 +38,18 @@ import org.apache.hadoop.hbase.util.BoundedPriorityBlockingQueue; public class SimpleRpcScheduler extends RpcScheduler { public static final Log LOG = LogFactory.getLog(SimpleRpcScheduler.class); - public static final String CALL_QUEUE_READ_SHARE_CONF_KEY = "ipc.server.callqueue.read.share"; + public static final String CALL_QUEUE_READ_SHARE_CONF_KEY = "hbase.ipc.server.callqueue.read.share"; public static final String CALL_QUEUE_HANDLER_FACTOR_CONF_KEY = - "ipc.server.callqueue.handler.factor"; + "hbase.ipc.server.callqueue.handler.factor"; /** If set to 'deadline', uses a priority queue and deprioritize long-running scans */ - public static final String CALL_QUEUE_TYPE_CONF_KEY = "ipc.server.callqueue.type"; + public static final String CALL_QUEUE_TYPE_CONF_KEY = "hbase.ipc.server.callqueue.type"; public static final String CALL_QUEUE_TYPE_DEADLINE_CONF_VALUE = "deadline"; public static final String CALL_QUEUE_TYPE_FIFO_CONF_VALUE = "fifo"; /** max delay in msec used to bound the deprioritized requests */ - public static final String QUEUE_MAX_CALL_DELAY_CONF_KEY = "ipc.server.queue.max.call.delay"; + public static final String QUEUE_MAX_CALL_DELAY_CONF_KEY + = "hbase.ipc.server.queue.max.call.delay"; /** * Comparator used by the "normal callQueue" if DEADLINE_CALL_QUEUE_CONF_KEY is set to true. @@ -104,7 +105,7 @@ public class SimpleRpcScheduler extends RpcScheduler { int replicationHandlerCount, PriorityFunction priority, int highPriorityLevel) { - int maxQueueLength = conf.getInt("ipc.server.max.callqueue.length", + int maxQueueLength = conf.getInt("hbase.ipc.server.max.callqueue.length", handlerCount * RpcServer.DEFAULT_MAX_CALLQUEUE_LENGTH_PER_HANDLER); this.priority = priority; this.highPriorityLevel = highPriorityLevel; http://git-wip-us.apache.org/repos/asf/hbase/blob/af141f2b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AnnotationReadingPriorityFunction.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AnnotationReadingPriorityFunction.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AnnotationReadingPriorityFunction.java index e4a32b9..ac5e5b9 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AnnotationReadingPriorityFunction.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AnnotationReadingPriorityFunction.java @@ -71,7 +71,7 @@ class AnnotationReadingPriorityFunction implements PriorityFunction { LogFactory.getLog(AnnotationReadingPriorityFunction.class.getName()); /** Used to control the scan delay, currently sqrt(numNextCall * weight) */ - public static final String SCAN_VTIME_WEIGHT_CONF_KEY = "ipc.server.scan.vtime.weight"; + public static final String SCAN_VTIME_WEIGHT_CONF_KEY = "hbase.ipc.server.scan.vtime.weight"; private final Map<String, Integer> annotatedQos; //We need to mock the regionserver instance for some unit tests (set via http://git-wip-us.apache.org/repos/asf/hbase/blob/af141f2b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java index a0ac336..ff0cb33 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java @@ -614,7 +614,7 @@ public class HRegion implements HeapSize { // , Writable{ + busyWaitDuration + ") or hbase.busy.wait.multiplier.max (" + maxBusyWaitMultiplier + "). Their product should be positive"); } - this.maxBusyWaitDuration = conf.getLong("ipc.client.call.purge.timeout", + this.maxBusyWaitDuration = conf.getLong("hbase.ipc.client.call.purge.timeout", 2 * HConstants.DEFAULT_HBASE_RPC_TIMEOUT); /* http://git-wip-us.apache.org/repos/asf/hbase/blob/af141f2b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestFullLogReconstruction.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestFullLogReconstruction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestFullLogReconstruction.java index 429f299..f4ffa2b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestFullLogReconstruction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestFullLogReconstruction.java @@ -52,7 +52,7 @@ public class TestFullLogReconstruction { c.setInt("dfs.heartbeat.interval", 1); c.setInt("dfs.client.socket-timeout", 5000); // faster failover with cluster.shutdown();fs.close() idiom - c.setInt("ipc.client.connect.max.retries", 1); + c.setInt("hbase.ipc.client.connect.max.retries", 1); c.setInt("dfs.client.block.recovery.retries", 1); c.setInt(HConstants.ZK_SESSION_TIMEOUT, 1000); TEST_UTIL.startMiniCluster(3); http://git-wip-us.apache.org/repos/asf/hbase/blob/af141f2b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java index 1a0d5d1..c7c363d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java @@ -135,11 +135,11 @@ public class TestHLog { // faster failover with cluster.shutdown();fs.close() idiom TEST_UTIL.getConfiguration() - .setInt("ipc.client.connect.max.retries", 1); + .setInt("hbase.ipc.client.connect.max.retries", 1); TEST_UTIL.getConfiguration().setInt( "dfs.client.block.recovery.retries", 1); TEST_UTIL.getConfiguration().setInt( - "ipc.client.connection.maxidletime", 500); + "hbase.ipc.client.connection.maxidletime", 500); TEST_UTIL.getConfiguration().set(CoprocessorHost.WAL_COPROCESSOR_CONF_KEY, SampleRegionWALObserver.class.getName()); TEST_UTIL.startMiniDFSCluster(3); http://git-wip-us.apache.org/repos/asf/hbase/blob/af141f2b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java ---------------------------------------------------------------------- diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java index 3c03267..9941940 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java @@ -121,7 +121,6 @@ public class TestLogRolling { TEST_UTIL.getConfiguration().setInt("hbase.regionserver.maxlogentries", 32); TEST_UTIL.getConfiguration().setInt("hbase.regionserver.logroll.errors.tolerated", 2); - TEST_UTIL.getConfiguration().setInt("ipc.socket.timeout", 10 * 1000); TEST_UTIL.getConfiguration().setInt("hbase.rpc.timeout", 10 * 1000); // For less frequently updated regions flush after every 2 flushes http://git-wip-us.apache.org/repos/asf/hbase/blob/af141f2b/hbase-shell/src/main/ruby/hbase/hbase.rb ---------------------------------------------------------------------- diff --git a/hbase-shell/src/main/ruby/hbase/hbase.rb b/hbase-shell/src/main/ruby/hbase/hbase.rb index 8c8d2b1..e75535e 100644 --- a/hbase-shell/src/main/ruby/hbase/hbase.rb +++ b/hbase-shell/src/main/ruby/hbase/hbase.rb @@ -36,7 +36,7 @@ module Hbase self.configuration = org.apache.hadoop.hbase.HBaseConfiguration.create # Turn off retries in hbase and ipc. Human doesn't want to wait on N retries. configuration.setInt("hbase.client.retries.number", 7) - configuration.setInt("ipc.client.connect.max.retries", 3) + configuration.setInt("hbase.ipc.client.connect.max.retries", 3) end end http://git-wip-us.apache.org/repos/asf/hbase/blob/af141f2b/src/main/docbkx/configuration.xml ---------------------------------------------------------------------- diff --git a/src/main/docbkx/configuration.xml b/src/main/docbkx/configuration.xml index d2798da..b0b2864 100644 --- a/src/main/docbkx/configuration.xml +++ b/src/main/docbkx/configuration.xml @@ -1309,7 +1309,7 @@ index e70ebc6..96f8c27 100644 requests to user tables. The rule of thumb is to keep this number low when the payload per request approaches the MB (big puts, scans using a large cache) and high when the payload is small (gets, small puts, ICVs, deletes). The total size of the queries in progress is - limited by the setting "ipc.server.max.callqueue.size". </para> + limited by the setting "hbase.ipc.server.max.callqueue.size". </para> <para> It is safe to set that number to the maximum number of incoming clients if their payload is small, the typical example being a cluster that serves a website since puts aren't typically buffered and most of the operations are gets. </para>
