Skip to content

Commit ce65db3

Browse files
bitterfoxsaintstack
authored andcommitted
HBASE-23185 Fix high cpu usage because getTable()#put() gets config value every time (#731)
* HBASE-23185 Fix high cpu usage because getTable()#put() gets config value every time * Fix checkstyle
1 parent e183c90 commit ce65db3

File tree

5 files changed

+123
-61
lines changed

5 files changed

+123
-61
lines changed

hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java

Lines changed: 60 additions & 30 deletions
Original file line numberDiff line numberDiff line change
@@ -323,35 +323,60 @@ public AsyncProcess(ClusterConnection hc, Configuration conf, ExecutorService po
323323

324324
this.id = COUNTER.incrementAndGet();
325325

326-
this.pause = conf.getLong(HConstants.HBASE_CLIENT_PAUSE,
327-
HConstants.DEFAULT_HBASE_CLIENT_PAUSE);
328-
long configuredPauseForCQTBE = conf.getLong(HConstants.HBASE_CLIENT_PAUSE_FOR_CQTBE, pause);
329-
if (configuredPauseForCQTBE < pause) {
330-
LOG.warn("The " + HConstants.HBASE_CLIENT_PAUSE_FOR_CQTBE + " setting: "
331-
+ configuredPauseForCQTBE + " is smaller than " + HConstants.HBASE_CLIENT_PAUSE
332-
+ ", will use " + pause + " instead.");
333-
this.pauseForCQTBE = pause;
334-
} else {
335-
this.pauseForCQTBE = configuredPauseForCQTBE;
336-
}
337-
this.numTries = conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
338-
HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
326+
ConnectionConfiguration connConf =
327+
hc.getConfiguration() == conf
328+
? hc.getConnectionConfiguration()
329+
// Slow: parse conf in ConnectionConfiguration constructor
330+
: new ConnectionConfiguration(conf);
331+
if (connConf == null) {
332+
// Slow: parse conf in ConnectionConfiguration constructor
333+
connConf = new ConnectionConfiguration(conf);
334+
}
335+
336+
this.pause = connConf.getPause();
337+
this.pauseForCQTBE = connConf.getPauseForCQTBE();
338+
339+
this.numTries = connConf.getRetriesNumber();
339340
this.rpcTimeout = rpcTimeout;
340-
this.operationTimeout = conf.getInt(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT,
341-
HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT);
342-
this.primaryCallTimeoutMicroseconds = conf.getInt(PRIMARY_CALL_TIMEOUT_KEY, 10000);
343-
344-
this.maxTotalConcurrentTasks = conf.getInt(HConstants.HBASE_CLIENT_MAX_TOTAL_TASKS,
345-
HConstants.DEFAULT_HBASE_CLIENT_MAX_TOTAL_TASKS);
346-
this.maxConcurrentTasksPerServer = conf.getInt(HConstants.HBASE_CLIENT_MAX_PERSERVER_TASKS,
347-
HConstants.DEFAULT_HBASE_CLIENT_MAX_PERSERVER_TASKS);
348-
this.maxConcurrentTasksPerRegion = conf.getInt(HConstants.HBASE_CLIENT_MAX_PERREGION_TASKS,
349-
HConstants.DEFAULT_HBASE_CLIENT_MAX_PERREGION_TASKS);
350-
this.maxHeapSizePerRequest = conf.getLong(HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE,
351-
DEFAULT_HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE);
352-
this.maxHeapSizeSubmit = conf.getLong(HBASE_CLIENT_MAX_SUBMIT_HEAPSIZE, DEFAULT_HBASE_CLIENT_MAX_SUBMIT_HEAPSIZE);
341+
this.operationTimeout = connConf.getOperationTimeout();
342+
343+
// Parse config once and reuse config values of hc's AsyncProcess in AsyncProcess for put
344+
// Can be null when constructing hc's AsyncProcess or it's not reusable
345+
AsyncProcess globalAsyncProcess = hc.getConfiguration() == conf ? hc.getAsyncProcess() : null;
346+
347+
this.primaryCallTimeoutMicroseconds =
348+
globalAsyncProcess == null
349+
? conf.getInt(PRIMARY_CALL_TIMEOUT_KEY, 10000)
350+
: globalAsyncProcess.primaryCallTimeoutMicroseconds;
351+
352+
this.maxTotalConcurrentTasks =
353+
globalAsyncProcess == null
354+
? conf.getInt(HConstants.HBASE_CLIENT_MAX_TOTAL_TASKS,
355+
HConstants.DEFAULT_HBASE_CLIENT_MAX_TOTAL_TASKS)
356+
: globalAsyncProcess.maxTotalConcurrentTasks;
357+
this.maxConcurrentTasksPerServer =
358+
globalAsyncProcess == null
359+
? conf.getInt(HConstants.HBASE_CLIENT_MAX_PERSERVER_TASKS,
360+
HConstants.DEFAULT_HBASE_CLIENT_MAX_PERSERVER_TASKS)
361+
: globalAsyncProcess.maxConcurrentTasksPerServer;
362+
this.maxConcurrentTasksPerRegion =
363+
globalAsyncProcess == null
364+
? conf.getInt(HConstants.HBASE_CLIENT_MAX_PERREGION_TASKS,
365+
HConstants.DEFAULT_HBASE_CLIENT_MAX_PERREGION_TASKS)
366+
: globalAsyncProcess.maxConcurrentTasksPerRegion;
367+
this.maxHeapSizePerRequest =
368+
globalAsyncProcess == null
369+
? conf.getLong(HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE,
370+
DEFAULT_HBASE_CLIENT_MAX_PERREQUEST_HEAPSIZE)
371+
: globalAsyncProcess.maxHeapSizePerRequest;
372+
this.maxHeapSizeSubmit =
373+
globalAsyncProcess == null
374+
? conf.getLong(HBASE_CLIENT_MAX_SUBMIT_HEAPSIZE, DEFAULT_HBASE_CLIENT_MAX_SUBMIT_HEAPSIZE)
375+
: globalAsyncProcess.maxHeapSizeSubmit;
353376
this.startLogErrorsCnt =
354-
conf.getInt(START_LOG_ERRORS_AFTER_COUNT_KEY, DEFAULT_START_LOG_ERRORS_AFTER_COUNT);
377+
globalAsyncProcess == null
378+
? conf.getInt(START_LOG_ERRORS_AFTER_COUNT_KEY, DEFAULT_START_LOG_ERRORS_AFTER_COUNT)
379+
: globalAsyncProcess.startLogErrorsCnt;
355380

356381
if (this.maxTotalConcurrentTasks <= 0) {
357382
throw new IllegalArgumentException("maxTotalConcurrentTasks=" + maxTotalConcurrentTasks);
@@ -387,11 +412,16 @@ public AsyncProcess(ClusterConnection hc, Configuration conf, ExecutorService po
387412

388413
this.rpcCallerFactory = rpcCaller;
389414
this.rpcFactory = rpcFactory;
390-
this.logBatchErrorDetails = conf.getBoolean(LOG_DETAILS_FOR_BATCH_ERROR, false);
415+
this.logBatchErrorDetails =
416+
globalAsyncProcess == null
417+
? conf.getBoolean(LOG_DETAILS_FOR_BATCH_ERROR, false)
418+
: globalAsyncProcess.logBatchErrorDetails;
391419

392420
this.thresholdToLogUndoneTaskDetails =
393-
conf.getInt(THRESHOLD_TO_LOG_UNDONE_TASK_DETAILS,
394-
DEFAULT_THRESHOLD_TO_LOG_UNDONE_TASK_DETAILS);
421+
globalAsyncProcess == null
422+
? conf.getInt(THRESHOLD_TO_LOG_UNDONE_TASK_DETAILS,
423+
DEFAULT_THRESHOLD_TO_LOG_UNDONE_TASK_DETAILS)
424+
: globalAsyncProcess.thresholdToLogUndoneTaskDetails;
395425
}
396426

397427
public void setRpcTimeout(int rpcTimeout) {

hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java

Lines changed: 12 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -34,7 +34,6 @@
3434
import org.apache.commons.logging.Log;
3535
import org.apache.commons.logging.LogFactory;
3636
import org.apache.hadoop.conf.Configuration;
37-
import org.apache.hadoop.hbase.HConstants; // Needed for write rpc timeout
3837
import org.apache.hadoop.hbase.TableName;
3938
import org.apache.hadoop.hbase.classification.InterfaceAudience;
4039
import org.apache.hadoop.hbase.classification.InterfaceStability;
@@ -112,32 +111,32 @@ public class BufferedMutatorImpl implements BufferedMutator {
112111
this.pool = params.getPool();
113112
this.listener = params.getListener();
114113

115-
ConnectionConfiguration tableConf = new ConnectionConfiguration(conf);
114+
ConnectionConfiguration connConf = conn.getConnectionConfiguration();
115+
if (connConf == null) {
116+
// Slow: parse conf in ConnectionConfiguration constructor
117+
connConf = new ConnectionConfiguration(conf);
118+
}
116119
this.writeBufferSize = params.getWriteBufferSize() != BufferedMutatorParams.UNSET ?
117-
params.getWriteBufferSize() : tableConf.getWriteBufferSize();
120+
params.getWriteBufferSize() : connConf.getWriteBufferSize();
118121

119122
// Set via the setter because it does value validation and starts/stops the TimerTask
120123
long newWriteBufferPeriodicFlushTimeoutMs =
121124
params.getWriteBufferPeriodicFlushTimeoutMs() != UNSET
122125
? params.getWriteBufferPeriodicFlushTimeoutMs()
123-
: tableConf.getWriteBufferPeriodicFlushTimeoutMs();
126+
: connConf.getWriteBufferPeriodicFlushTimeoutMs();
124127
long newWriteBufferPeriodicFlushTimerTickMs =
125128
params.getWriteBufferPeriodicFlushTimerTickMs() != UNSET
126129
? params.getWriteBufferPeriodicFlushTimerTickMs()
127-
: tableConf.getWriteBufferPeriodicFlushTimerTickMs();
130+
: connConf.getWriteBufferPeriodicFlushTimerTickMs();
128131
this.setWriteBufferPeriodicFlush(
129132
newWriteBufferPeriodicFlushTimeoutMs,
130133
newWriteBufferPeriodicFlushTimerTickMs);
131134

132135
this.maxKeyValueSize = params.getMaxKeyValueSize() != BufferedMutatorParams.UNSET ?
133-
params.getMaxKeyValueSize() : tableConf.getMaxKeyValueSize();
134-
135-
this.writeRpcTimeout = conn.getConfiguration().getInt(HConstants.HBASE_RPC_WRITE_TIMEOUT_KEY,
136-
conn.getConfiguration().getInt(HConstants.HBASE_RPC_TIMEOUT_KEY,
137-
HConstants.DEFAULT_HBASE_RPC_TIMEOUT));
138-
this.operationTimeout = conn.getConfiguration().getInt(
139-
HConstants.HBASE_CLIENT_OPERATION_TIMEOUT,
140-
HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT);
136+
params.getMaxKeyValueSize() : connConf.getMaxKeyValueSize();
137+
138+
this.writeRpcTimeout = connConf.getWriteRpcTimeout();
139+
this.operationTimeout = connConf.getOperationTimeout();
141140
// puts need to track errors globally due to how the APIs currently work.
142141
ap = new AsyncProcess(connection, conf, pool, rpcCallerFactory, true, rpcFactory, writeRpcTimeout);
143142
}

hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionConfiguration.java

Lines changed: 47 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -12,6 +12,8 @@
1212
package org.apache.hadoop.hbase.client;
1313

1414
import com.google.common.annotations.VisibleForTesting;
15+
import org.apache.commons.logging.Log;
16+
import org.apache.commons.logging.LogFactory;
1517
import org.apache.hadoop.conf.Configuration;
1618
import org.apache.hadoop.hbase.HConstants;
1719
import org.apache.hadoop.hbase.classification.InterfaceAudience;
@@ -26,6 +28,7 @@
2628
*/
2729
@InterfaceAudience.Private
2830
public class ConnectionConfiguration {
31+
static final Log LOG = LogFactory.getLog(ConnectionConfiguration.class);
2932

3033
public static final String WRITE_BUFFER_SIZE_KEY = "hbase.client.write.buffer";
3134
public static final long WRITE_BUFFER_SIZE_DEFAULT = 2097152;
@@ -50,6 +53,10 @@ public class ConnectionConfiguration {
5053
private final int metaReplicaCallTimeoutMicroSecondScan;
5154
private final int retries;
5255
private final int maxKeyValueSize;
56+
private final int readRpcTimeout;
57+
private final int writeRpcTimeout;
58+
private final long pause;
59+
private final long pauseForCQTBE;
5360

5461
/**
5562
* Constructor
@@ -90,9 +97,28 @@ public class ConnectionConfiguration {
9097
HConstants.HBASE_CLIENT_META_REPLICA_SCAN_TIMEOUT_DEFAULT);
9198

9299
this.retries = conf.getInt(
93-
HConstants.HBASE_CLIENT_RETRIES_NUMBER, HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
100+
HConstants.HBASE_CLIENT_RETRIES_NUMBER, HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
94101

95102
this.maxKeyValueSize = conf.getInt(MAX_KEYVALUE_SIZE_KEY, MAX_KEYVALUE_SIZE_DEFAULT);
103+
104+
this.readRpcTimeout = conf.getInt(HConstants.HBASE_RPC_READ_TIMEOUT_KEY,
105+
conf.getInt(HConstants.HBASE_RPC_TIMEOUT_KEY,
106+
HConstants.DEFAULT_HBASE_RPC_TIMEOUT));
107+
108+
this.writeRpcTimeout = conf.getInt(HConstants.HBASE_RPC_WRITE_TIMEOUT_KEY,
109+
conf.getInt(HConstants.HBASE_RPC_TIMEOUT_KEY,
110+
HConstants.DEFAULT_HBASE_RPC_TIMEOUT));
111+
112+
this.pause = conf.getLong(HConstants.HBASE_CLIENT_PAUSE, HConstants.DEFAULT_HBASE_CLIENT_PAUSE);
113+
long configuredPauseForCQTBE = conf.getLong(HConstants.HBASE_CLIENT_PAUSE_FOR_CQTBE, pause);
114+
if (configuredPauseForCQTBE < pause) {
115+
LOG.warn("The " + HConstants.HBASE_CLIENT_PAUSE_FOR_CQTBE + " setting: "
116+
+ configuredPauseForCQTBE + " is smaller than " + HConstants.HBASE_CLIENT_PAUSE
117+
+ ", will use " + pause + " instead.");
118+
this.pauseForCQTBE = pause;
119+
} else {
120+
this.pauseForCQTBE = configuredPauseForCQTBE;
121+
}
96122
}
97123

98124
/**
@@ -115,6 +141,10 @@ protected ConnectionConfiguration() {
115141
HConstants.HBASE_CLIENT_META_REPLICA_SCAN_TIMEOUT_DEFAULT;
116142
this.retries = HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER;
117143
this.maxKeyValueSize = MAX_KEYVALUE_SIZE_DEFAULT;
144+
this.readRpcTimeout = HConstants.DEFAULT_HBASE_RPC_TIMEOUT;
145+
this.writeRpcTimeout = HConstants.DEFAULT_HBASE_RPC_TIMEOUT;
146+
this.pause = HConstants.DEFAULT_HBASE_CLIENT_PAUSE;
147+
this.pauseForCQTBE = HConstants.DEFAULT_HBASE_CLIENT_PAUSE;
118148
}
119149

120150
public long getWriteBufferSize() {
@@ -164,4 +194,20 @@ public int getMaxKeyValueSize() {
164194
public long getScannerMaxResultSize() {
165195
return scannerMaxResultSize;
166196
}
197+
198+
public int getReadRpcTimeout() {
199+
return readRpcTimeout;
200+
}
201+
202+
public int getWriteRpcTimeout() {
203+
return writeRpcTimeout;
204+
}
205+
206+
public long getPause() {
207+
return pause;
208+
}
209+
210+
public long getPauseForCQTBE() {
211+
return pauseForCQTBE;
212+
}
167213
}

hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionManager.java

Lines changed: 2 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -669,17 +669,8 @@ static class HConnectionImplementation implements ClusterConnection, Closeable {
669669
this.managed = managed;
670670
this.connectionConfig = new ConnectionConfiguration(conf);
671671
this.closed = false;
672-
this.pause = conf.getLong(HConstants.HBASE_CLIENT_PAUSE,
673-
HConstants.DEFAULT_HBASE_CLIENT_PAUSE);
674-
long configuredPauseForCQTBE = conf.getLong(HConstants.HBASE_CLIENT_PAUSE_FOR_CQTBE, pause);
675-
if (configuredPauseForCQTBE < pause) {
676-
LOG.warn("The " + HConstants.HBASE_CLIENT_PAUSE_FOR_CQTBE + " setting: "
677-
+ configuredPauseForCQTBE + " is smaller than " + HConstants.HBASE_CLIENT_PAUSE
678-
+ ", will use " + pause + " instead.");
679-
this.pauseForCQTBE = pause;
680-
} else {
681-
this.pauseForCQTBE = configuredPauseForCQTBE;
682-
}
672+
this.pause = connectionConfig.getPause();
673+
this.pauseForCQTBE = connectionConfig.getPauseForCQTBE();
683674
this.useMetaReplicas = conf.getBoolean(HConstants.USE_META_REPLICAS,
684675
HConstants.DEFAULT_USE_META_REPLICAS);
685676
this.metaReplicaCallTimeoutScanInMicroSecond =

hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java

Lines changed: 2 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -366,12 +366,8 @@ private void finishSetup() throws IOException {
366366
}
367367
this.operationTimeout = tableName.isSystemTable() ?
368368
connConfiguration.getMetaOperationTimeout() : connConfiguration.getOperationTimeout();
369-
this.readRpcTimeout = configuration.getInt(HConstants.HBASE_RPC_READ_TIMEOUT_KEY,
370-
configuration.getInt(HConstants.HBASE_RPC_TIMEOUT_KEY,
371-
HConstants.DEFAULT_HBASE_RPC_TIMEOUT));
372-
this.writeRpcTimeout = configuration.getInt(HConstants.HBASE_RPC_WRITE_TIMEOUT_KEY,
373-
configuration.getInt(HConstants.HBASE_RPC_TIMEOUT_KEY,
374-
HConstants.DEFAULT_HBASE_RPC_TIMEOUT));
369+
this.readRpcTimeout = connConfiguration.getReadRpcTimeout();
370+
this.writeRpcTimeout = connConfiguration.getWriteRpcTimeout();
375371
this.scannerCaching = connConfiguration.getScannerCaching();
376372
this.scannerMaxResultSize = connConfiguration.getScannerMaxResultSize();
377373
if (this.rpcCallerFactory == null) {

0 commit comments

Comments
 (0)