001/**
002 *
003 * Licensed to the Apache Software Foundation (ASF) under one
004 * or more contributor license agreements.  See the NOTICE file
005 * distributed with this work for additional information
006 * regarding copyright ownership.  The ASF licenses this file
007 * to you under the Apache License, Version 2.0 (the
008 * "License"); you may not use this file except in compliance
009 * with the License.  You may obtain a copy of the License at
010 *
011 *     http://www.apache.org/licenses/LICENSE-2.0
012 *
013 * Unless required by applicable law or agreed to in writing, software
014 * distributed under the License is distributed on an "AS IS" BASIS,
015 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
016 * See the License for the specific language governing permissions and
017 * limitations under the License.
018 */
019
020package org.apache.hadoop.hbase.client;
021
022import java.util.concurrent.ExecutorService;
023import org.apache.hadoop.hbase.TableName;
024import org.apache.yetus.audience.InterfaceAudience;
025
026/**
027 * Parameters for instantiating a {@link BufferedMutator}.
028 */
029@InterfaceAudience.Public
030public class BufferedMutatorParams implements Cloneable {
031
032  static final int UNSET = -1;
033
034  private final TableName tableName;
035  private long writeBufferSize = UNSET;
036  private long writeBufferPeriodicFlushTimeoutMs = UNSET;
037  private long writeBufferPeriodicFlushTimerTickMs = UNSET;
038  private int maxKeyValueSize = UNSET;
039  private ExecutorService pool = null;
040  private String implementationClassName = null;
041  private int rpcTimeout = UNSET;
042  private int operationTimeout = UNSET;
043  private BufferedMutator.ExceptionListener listener = new BufferedMutator.ExceptionListener() {
044    @Override
045    public void onException(RetriesExhaustedWithDetailsException exception,
046        BufferedMutator bufferedMutator)
047        throws RetriesExhaustedWithDetailsException {
048      throw exception;
049    }
050  };
051
052  public BufferedMutatorParams(TableName tableName) {
053    this.tableName = tableName;
054  }
055
056  public TableName getTableName() {
057    return tableName;
058  }
059
060  public long getWriteBufferSize() {
061    return writeBufferSize;
062  }
063
064  public BufferedMutatorParams rpcTimeout(final int rpcTimeout) {
065    this.rpcTimeout = rpcTimeout;
066    return this;
067  }
068
069  public int getRpcTimeout() {
070    return rpcTimeout;
071  }
072
073  public BufferedMutatorParams operationTimeout(final int operationTimeout) {
074    this.operationTimeout = operationTimeout;
075    return this;
076  }
077
078  /**
079   * @deprecated Since 2.3.0, will be removed in 4.0.0. Use {@link #operationTimeout(int)}
080   */
081  @Deprecated
082  public BufferedMutatorParams opertationTimeout(final int operationTimeout) {
083    this.operationTimeout = operationTimeout;
084    return this;
085  }
086
087  public int getOperationTimeout() {
088    return operationTimeout;
089  }
090
091  /**
092   * Override the write buffer size specified by the provided {@link Connection}'s
093   * {@link org.apache.hadoop.conf.Configuration} instance, via the configuration key
094   * {@code hbase.client.write.buffer}.
095   */
096  public BufferedMutatorParams writeBufferSize(long writeBufferSize) {
097    this.writeBufferSize = writeBufferSize;
098    return this;
099  }
100
101  public long getWriteBufferPeriodicFlushTimeoutMs() {
102    return writeBufferPeriodicFlushTimeoutMs;
103  }
104
105  /**
106   * Set the max timeout before the buffer is automatically flushed.
107   */
108  public BufferedMutatorParams setWriteBufferPeriodicFlushTimeoutMs(long timeoutMs) {
109    this.writeBufferPeriodicFlushTimeoutMs = timeoutMs;
110    return this;
111  }
112
113  /**
114   * @deprecated Since 3.0.0, will be removed in 4.0.0. We use a common timer in the whole client
115   *             implementation so you can not set it any more.
116   */
117  @Deprecated
118  public long getWriteBufferPeriodicFlushTimerTickMs() {
119    return writeBufferPeriodicFlushTimerTickMs;
120  }
121
122  /**
123   * Set the TimerTick how often the buffer timeout if checked.
124   * @deprecated Since 3.0.0, will be removed in 4.0.0. We use a common timer in the whole client
125   *             implementation so you can not set it any more.
126   */
127  @Deprecated
128  public BufferedMutatorParams setWriteBufferPeriodicFlushTimerTickMs(long timerTickMs) {
129    this.writeBufferPeriodicFlushTimerTickMs = timerTickMs;
130    return this;
131  }
132
133  public int getMaxKeyValueSize() {
134    return maxKeyValueSize;
135  }
136
137  /**
138   * Override the maximum key-value size specified by the provided {@link Connection}'s
139   * {@link org.apache.hadoop.conf.Configuration} instance, via the configuration key
140   * {@code hbase.client.keyvalue.maxsize}.
141   */
142  public BufferedMutatorParams maxKeyValueSize(int maxKeyValueSize) {
143    this.maxKeyValueSize = maxKeyValueSize;
144    return this;
145  }
146
147  public ExecutorService getPool() {
148    return pool;
149  }
150
151  /**
152   * Override the default executor pool defined by the {@code hbase.htable.threads.*}
153   * configuration values.
154   */
155  public BufferedMutatorParams pool(ExecutorService pool) {
156    this.pool = pool;
157    return this;
158  }
159
160  /**
161   * @return Name of the class we will use when we construct a {@link BufferedMutator} instance or
162   *         null if default implementation.
163   * @deprecated Since 3.0.0, will be removed in 4.0.0. You can not set it any more as the
164   *             implementation has to use too many internal stuffs in HBase.
165   */
166  @Deprecated
167  public String getImplementationClassName() {
168    return this.implementationClassName;
169  }
170
171  /**
172   * Specify a BufferedMutator implementation other than the default.
173   * @param implementationClassName Name of the BufferedMutator implementation class
174   * @deprecated Since 3.0.0, will be removed in 4.0.0. You can not set it any more as the
175   *             implementation has to use too many internal stuffs in HBase.
176   */
177  @Deprecated
178  public BufferedMutatorParams implementationClassName(String implementationClassName) {
179    this.implementationClassName = implementationClassName;
180    return this;
181  }
182
183  public BufferedMutator.ExceptionListener getListener() {
184    return listener;
185  }
186
187  /**
188   * Override the default error handler. Default handler simply rethrows the exception.
189   */
190  public BufferedMutatorParams listener(BufferedMutator.ExceptionListener listener) {
191    this.listener = listener;
192    return this;
193  }
194
195  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="CN_IDIOM_NO_SUPER_CALL",
196    justification="The clone below is complete")
197  @Override
198  public BufferedMutatorParams clone() {
199    BufferedMutatorParams clone = new BufferedMutatorParams(this.tableName);
200    clone.writeBufferSize                     = this.writeBufferSize;
201    clone.writeBufferPeriodicFlushTimeoutMs   = this.writeBufferPeriodicFlushTimeoutMs;
202    clone.writeBufferPeriodicFlushTimerTickMs = this.writeBufferPeriodicFlushTimerTickMs;
203    clone.maxKeyValueSize                     = this.maxKeyValueSize;
204    clone.pool                                = this.pool;
205    clone.listener                            = this.listener;
206    clone.implementationClassName             = this.implementationClassName;
207    return clone;
208  }
209}