001/*
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.client;
019
020import org.apache.hadoop.conf.Configuration;
021import org.apache.hadoop.hbase.HConstants;
022import org.apache.yetus.audience.InterfaceAudience;
023
024/**
025 * Configuration parameters for the connection. Configuration is a heavy weight registry that does a
026 * lot of string operations and regex matching. Method calls into Configuration account for high CPU
027 * usage and have huge performance impact. This class caches connection-related configuration values
028 * in the ConnectionConfiguration object so that expensive conf.getXXX() calls are avoided every
029 * time HTable, etc is instantiated. see HBASE-12128
030 */
031@InterfaceAudience.Private
032public class ConnectionConfiguration {
033
034  public static final String WRITE_BUFFER_SIZE_KEY = "hbase.client.write.buffer";
035  public static final long WRITE_BUFFER_SIZE_DEFAULT = 2097152;
036  public static final String WRITE_BUFFER_PERIODIC_FLUSH_TIMEOUT_MS =
037    "hbase.client.write.buffer.periodicflush.timeout.ms";
038  public static final String WRITE_BUFFER_PERIODIC_FLUSH_TIMERTICK_MS =
039    "hbase.client.write.buffer.periodicflush.timertick.ms";
040  public static final long WRITE_BUFFER_PERIODIC_FLUSH_TIMEOUT_MS_DEFAULT = 0; // 0 == Disabled
041  public static final long WRITE_BUFFER_PERIODIC_FLUSH_TIMERTICK_MS_DEFAULT = 1000L; // 1 second
042  public static final String MAX_KEYVALUE_SIZE_KEY = "hbase.client.keyvalue.maxsize";
043  public static final int MAX_KEYVALUE_SIZE_DEFAULT = 10485760;
044  public static final String BUFFERED_MUTATOR_MAX_MUTATIONS_KEY =
045    "hbase.client.write.buffer.maxmutations";
046  public static final int BUFFERED_MUTATOR_MAX_MUTATIONS_DEFAULT = -1;
047  public static final String PRIMARY_CALL_TIMEOUT_MICROSECOND =
048    "hbase.client.primaryCallTimeout.get";
049  public static final int PRIMARY_CALL_TIMEOUT_MICROSECOND_DEFAULT = 10000; // 10ms
050  public static final String PRIMARY_SCAN_TIMEOUT_MICROSECOND =
051    "hbase.client.replicaCallTimeout.scan";
052  public static final int PRIMARY_SCAN_TIMEOUT_MICROSECOND_DEFAULT = 1000000; // 1s
053  public static final String LOG_SCANNER_ACTIVITY = "hbase.client.log.scanner.activity";
054
055  public static final String HBASE_CLIENT_META_READ_RPC_TIMEOUT_KEY =
056    "hbase.client.meta.read.rpc.timeout";
057  public static final String HBASE_CLIENT_META_SCANNER_TIMEOUT =
058    "hbase.client.meta.scanner.timeout.period";
059
060  public static final String HBASE_CLIENT_META_CACHE_INVALIDATE_INTERVAL =
061    "hbase.client.connection.metacache.invalidate-interval.ms";
062
063  private final long writeBufferSize;
064  private final long writeBufferPeriodicFlushTimeoutMs;
065  private final long writeBufferPeriodicFlushTimerTickMs;
066  private final int metaOperationTimeout;
067  private final int operationTimeout;
068  private final int scannerCaching;
069  private final long scannerMaxResultSize;
070  private final int primaryCallTimeoutMicroSecond;
071  private final int replicaCallTimeoutMicroSecondScan;
072  private final int metaReplicaCallTimeoutMicroSecondScan;
073  private final int retries;
074  private final int maxKeyValueSize;
075  private final int bufferedMutatorMaxMutations;
076  private final int rpcTimeout;
077  private final int readRpcTimeout;
078  private final int metaReadRpcTimeout;
079  private final int writeRpcTimeout;
080  // toggle for async/sync prefetch
081  private final boolean clientScannerAsyncPrefetch;
082
083  /**
084   * Constructor
085   * @param conf Configuration object
086   */
087  ConnectionConfiguration(Configuration conf) {
088    this.writeBufferSize = conf.getLong(WRITE_BUFFER_SIZE_KEY, WRITE_BUFFER_SIZE_DEFAULT);
089
090    this.writeBufferPeriodicFlushTimeoutMs = conf.getLong(WRITE_BUFFER_PERIODIC_FLUSH_TIMEOUT_MS,
091      WRITE_BUFFER_PERIODIC_FLUSH_TIMEOUT_MS_DEFAULT);
092
093    this.writeBufferPeriodicFlushTimerTickMs = conf.getLong(
094      WRITE_BUFFER_PERIODIC_FLUSH_TIMERTICK_MS, WRITE_BUFFER_PERIODIC_FLUSH_TIMERTICK_MS_DEFAULT);
095
096    this.metaOperationTimeout = conf.getInt(HConstants.HBASE_CLIENT_META_OPERATION_TIMEOUT,
097      conf.getInt(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT,
098        HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT));
099
100    this.operationTimeout = conf.getInt(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT,
101      HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT);
102
103    this.scannerCaching = conf.getInt(HConstants.HBASE_CLIENT_SCANNER_CACHING,
104      HConstants.DEFAULT_HBASE_CLIENT_SCANNER_CACHING);
105
106    this.scannerMaxResultSize = conf.getLong(HConstants.HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE_KEY,
107      HConstants.DEFAULT_HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE);
108
109    this.primaryCallTimeoutMicroSecond =
110      conf.getInt(PRIMARY_CALL_TIMEOUT_MICROSECOND, PRIMARY_CALL_TIMEOUT_MICROSECOND_DEFAULT);
111
112    this.replicaCallTimeoutMicroSecondScan =
113      conf.getInt(PRIMARY_SCAN_TIMEOUT_MICROSECOND, PRIMARY_SCAN_TIMEOUT_MICROSECOND_DEFAULT);
114
115    this.metaReplicaCallTimeoutMicroSecondScan =
116      conf.getInt(HConstants.HBASE_CLIENT_META_REPLICA_SCAN_TIMEOUT,
117        HConstants.HBASE_CLIENT_META_REPLICA_SCAN_TIMEOUT_DEFAULT);
118
119    this.retries = conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
120      HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
121
122    this.clientScannerAsyncPrefetch = conf.getBoolean(Scan.HBASE_CLIENT_SCANNER_ASYNC_PREFETCH,
123      Scan.DEFAULT_HBASE_CLIENT_SCANNER_ASYNC_PREFETCH);
124
125    this.maxKeyValueSize = conf.getInt(MAX_KEYVALUE_SIZE_KEY, MAX_KEYVALUE_SIZE_DEFAULT);
126
127    this.bufferedMutatorMaxMutations =
128      conf.getInt(BUFFERED_MUTATOR_MAX_MUTATIONS_KEY, BUFFERED_MUTATOR_MAX_MUTATIONS_DEFAULT);
129
130    this.rpcTimeout =
131      conf.getInt(HConstants.HBASE_RPC_TIMEOUT_KEY, HConstants.DEFAULT_HBASE_RPC_TIMEOUT);
132
133    this.readRpcTimeout = conf.getInt(HConstants.HBASE_RPC_READ_TIMEOUT_KEY,
134      conf.getInt(HConstants.HBASE_RPC_TIMEOUT_KEY, HConstants.DEFAULT_HBASE_RPC_TIMEOUT));
135
136    this.metaReadRpcTimeout = conf.getInt(HBASE_CLIENT_META_READ_RPC_TIMEOUT_KEY, readRpcTimeout);
137
138    this.writeRpcTimeout = conf.getInt(HConstants.HBASE_RPC_WRITE_TIMEOUT_KEY,
139      conf.getInt(HConstants.HBASE_RPC_TIMEOUT_KEY, HConstants.DEFAULT_HBASE_RPC_TIMEOUT));
140  }
141
142  /**
143   * Constructor This is for internal testing purpose (using the default value). In real usage, we
144   * should read the configuration from the Configuration object.
145   */
146  protected ConnectionConfiguration() {
147    this.writeBufferSize = WRITE_BUFFER_SIZE_DEFAULT;
148    this.writeBufferPeriodicFlushTimeoutMs = WRITE_BUFFER_PERIODIC_FLUSH_TIMEOUT_MS_DEFAULT;
149    this.writeBufferPeriodicFlushTimerTickMs = WRITE_BUFFER_PERIODIC_FLUSH_TIMERTICK_MS_DEFAULT;
150    this.metaOperationTimeout = HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT;
151    this.operationTimeout = HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT;
152    this.scannerCaching = HConstants.DEFAULT_HBASE_CLIENT_SCANNER_CACHING;
153    this.scannerMaxResultSize = HConstants.DEFAULT_HBASE_CLIENT_SCANNER_MAX_RESULT_SIZE;
154    this.primaryCallTimeoutMicroSecond = 10000;
155    this.replicaCallTimeoutMicroSecondScan = 1000000;
156    this.metaReplicaCallTimeoutMicroSecondScan =
157      HConstants.HBASE_CLIENT_META_REPLICA_SCAN_TIMEOUT_DEFAULT;
158    this.retries = HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER;
159    this.clientScannerAsyncPrefetch = Scan.DEFAULT_HBASE_CLIENT_SCANNER_ASYNC_PREFETCH;
160    this.maxKeyValueSize = MAX_KEYVALUE_SIZE_DEFAULT;
161    this.bufferedMutatorMaxMutations = BUFFERED_MUTATOR_MAX_MUTATIONS_DEFAULT;
162    this.readRpcTimeout = HConstants.DEFAULT_HBASE_RPC_TIMEOUT;
163    this.metaReadRpcTimeout = HConstants.DEFAULT_HBASE_RPC_TIMEOUT;
164    this.writeRpcTimeout = HConstants.DEFAULT_HBASE_RPC_TIMEOUT;
165    this.rpcTimeout = HConstants.DEFAULT_HBASE_RPC_TIMEOUT;
166  }
167
168  public int getReadRpcTimeout() {
169    return readRpcTimeout;
170  }
171
172  public int getMetaReadRpcTimeout() {
173    return metaReadRpcTimeout;
174  }
175
176  public int getWriteRpcTimeout() {
177    return writeRpcTimeout;
178  }
179
180  public long getWriteBufferSize() {
181    return writeBufferSize;
182  }
183
184  public long getWriteBufferPeriodicFlushTimeoutMs() {
185    return writeBufferPeriodicFlushTimeoutMs;
186  }
187
188  public long getWriteBufferPeriodicFlushTimerTickMs() {
189    return writeBufferPeriodicFlushTimerTickMs;
190  }
191
192  public int getMetaOperationTimeout() {
193    return metaOperationTimeout;
194  }
195
196  public int getOperationTimeout() {
197    return operationTimeout;
198  }
199
200  public int getScannerCaching() {
201    return scannerCaching;
202  }
203
204  public int getPrimaryCallTimeoutMicroSecond() {
205    return primaryCallTimeoutMicroSecond;
206  }
207
208  public int getReplicaCallTimeoutMicroSecondScan() {
209    return replicaCallTimeoutMicroSecondScan;
210  }
211
212  public int getMetaReplicaCallTimeoutMicroSecondScan() {
213    return metaReplicaCallTimeoutMicroSecondScan;
214  }
215
216  public int getRetriesNumber() {
217    return retries;
218  }
219
220  public int getMaxKeyValueSize() {
221    return maxKeyValueSize;
222  }
223
224  public int getBufferedMutatorMaxMutations() {
225    return bufferedMutatorMaxMutations;
226  }
227
228  public long getScannerMaxResultSize() {
229    return scannerMaxResultSize;
230  }
231
232  public boolean isClientScannerAsyncPrefetch() {
233    return clientScannerAsyncPrefetch;
234  }
235
236  public int getRpcTimeout() {
237    return rpcTimeout;
238  }
239}