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 java.util.Collections; 021import java.util.HashMap; 022import java.util.Map; 023import java.util.concurrent.ExecutorService; 024import org.apache.hadoop.hbase.TableName; 025import org.apache.yetus.audience.InterfaceAudience; 026 027/** 028 * Parameters for instantiating a {@link BufferedMutator}. 029 */ 030@InterfaceAudience.Public 031public class BufferedMutatorParams implements Cloneable { 032 033 static final int UNSET = -1; 034 035 private final TableName tableName; 036 private long writeBufferSize = UNSET; 037 private long writeBufferPeriodicFlushTimeoutMs = UNSET; 038 private long writeBufferPeriodicFlushTimerTickMs = UNSET; 039 private int maxKeyValueSize = UNSET; 040 private ExecutorService pool = null; 041 private String implementationClassName = null; 042 private int rpcTimeout = UNSET; 043 private int operationTimeout = UNSET; 044 protected Map<String, byte[]> requestAttributes = Collections.emptyMap(); 045 private BufferedMutator.ExceptionListener listener = new BufferedMutator.ExceptionListener() { 046 @Override 047 public void onException(RetriesExhaustedWithDetailsException exception, 048 BufferedMutator bufferedMutator) throws RetriesExhaustedWithDetailsException { 049 throw exception; 050 } 051 }; 052 053 public BufferedMutatorParams(TableName tableName) { 054 this.tableName = tableName; 055 } 056 057 public TableName getTableName() { 058 return tableName; 059 } 060 061 public long getWriteBufferSize() { 062 return writeBufferSize; 063 } 064 065 public BufferedMutatorParams rpcTimeout(final int rpcTimeout) { 066 this.rpcTimeout = rpcTimeout; 067 return this; 068 } 069 070 public int getRpcTimeout() { 071 return rpcTimeout; 072 } 073 074 public BufferedMutatorParams operationTimeout(final int operationTimeout) { 075 this.operationTimeout = operationTimeout; 076 return this; 077 } 078 079 /** 080 * @deprecated Since 2.3.0, will be removed in 4.0.0. Use {@link #operationTimeout(int)} 081 */ 082 @Deprecated 083 public BufferedMutatorParams opertationTimeout(final int operationTimeout) { 084 this.operationTimeout = operationTimeout; 085 return this; 086 } 087 088 public int getOperationTimeout() { 089 return operationTimeout; 090 } 091 092 public BufferedMutatorParams setRequestAttribute(String key, byte[] value) { 093 if (requestAttributes.isEmpty()) { 094 requestAttributes = new HashMap<>(); 095 } 096 requestAttributes.put(key, value); 097 return this; 098 } 099 100 public Map<String, byte[]> getRequestAttributes() { 101 return requestAttributes; 102 } 103 104 /** 105 * Override the write buffer size specified by the provided {@link Connection}'s 106 * {@link org.apache.hadoop.conf.Configuration} instance, via the configuration key 107 * {@code hbase.client.write.buffer}. 108 */ 109 public BufferedMutatorParams writeBufferSize(long writeBufferSize) { 110 this.writeBufferSize = writeBufferSize; 111 return this; 112 } 113 114 public long getWriteBufferPeriodicFlushTimeoutMs() { 115 return writeBufferPeriodicFlushTimeoutMs; 116 } 117 118 /** 119 * Set the max timeout before the buffer is automatically flushed. 120 */ 121 public BufferedMutatorParams setWriteBufferPeriodicFlushTimeoutMs(long timeoutMs) { 122 this.writeBufferPeriodicFlushTimeoutMs = timeoutMs; 123 return this; 124 } 125 126 /** 127 * @deprecated Since 3.0.0, will be removed in 4.0.0. We use a common timer in the whole client 128 * implementation so you can not set it any more. 129 */ 130 @Deprecated 131 public long getWriteBufferPeriodicFlushTimerTickMs() { 132 return writeBufferPeriodicFlushTimerTickMs; 133 } 134 135 /** 136 * Set the TimerTick how often the buffer timeout if checked. 137 * @deprecated Since 3.0.0, will be removed in 4.0.0. We use a common timer in the whole client 138 * implementation so you can not set it any more. 139 */ 140 @Deprecated 141 public BufferedMutatorParams setWriteBufferPeriodicFlushTimerTickMs(long timerTickMs) { 142 this.writeBufferPeriodicFlushTimerTickMs = timerTickMs; 143 return this; 144 } 145 146 public int getMaxKeyValueSize() { 147 return maxKeyValueSize; 148 } 149 150 /** 151 * Override the maximum key-value size specified by the provided {@link Connection}'s 152 * {@link org.apache.hadoop.conf.Configuration} instance, via the configuration key 153 * {@code hbase.client.keyvalue.maxsize}. 154 */ 155 public BufferedMutatorParams maxKeyValueSize(int maxKeyValueSize) { 156 this.maxKeyValueSize = maxKeyValueSize; 157 return this; 158 } 159 160 /** 161 * @deprecated Since 3.0.0-alpha-2, will be removed in 4.0.0. You can not set it anymore. 162 * BufferedMutator will use Connection's ExecutorService. 163 */ 164 @Deprecated 165 public ExecutorService getPool() { 166 return pool; 167 } 168 169 /** 170 * Override the default executor pool defined by the {@code hbase.htable.threads.*} configuration 171 * values. 172 * @deprecated Since 3.0.0-alpha-2, will be removed in 4.0.0. You can not set it anymore. 173 * BufferedMutator will use Connection's ExecutorService. 174 */ 175 @Deprecated 176 public BufferedMutatorParams pool(ExecutorService pool) { 177 this.pool = pool; 178 return this; 179 } 180 181 /** 182 * @return Name of the class we will use when we construct a {@link BufferedMutator} instance or 183 * null if default implementation. 184 * @deprecated Since 3.0.0, will be removed in 4.0.0. You can not set it any more as the 185 * implementation has to use too many internal stuffs in HBase. 186 */ 187 @Deprecated 188 public String getImplementationClassName() { 189 return this.implementationClassName; 190 } 191 192 /** 193 * Specify a BufferedMutator implementation other than the default. 194 * @param implementationClassName Name of the BufferedMutator implementation class 195 * @deprecated Since 3.0.0, will be removed in 4.0.0. You can not set it any more as the 196 * implementation has to use too many internal stuffs in HBase. 197 */ 198 @Deprecated 199 public BufferedMutatorParams implementationClassName(String implementationClassName) { 200 this.implementationClassName = implementationClassName; 201 return this; 202 } 203 204 public BufferedMutator.ExceptionListener getListener() { 205 return listener; 206 } 207 208 /** 209 * Override the default error handler. Default handler simply rethrows the exception. 210 */ 211 public BufferedMutatorParams listener(BufferedMutator.ExceptionListener listener) { 212 this.listener = listener; 213 return this; 214 } 215 216 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "CN_IDIOM_NO_SUPER_CALL", 217 justification = "The clone below is complete") 218 @Override 219 public BufferedMutatorParams clone() { 220 BufferedMutatorParams clone = new BufferedMutatorParams(this.tableName); 221 clone.writeBufferSize = this.writeBufferSize; 222 clone.writeBufferPeriodicFlushTimeoutMs = this.writeBufferPeriodicFlushTimeoutMs; 223 clone.writeBufferPeriodicFlushTimerTickMs = this.writeBufferPeriodicFlushTimerTickMs; 224 clone.maxKeyValueSize = this.maxKeyValueSize; 225 clone.pool = this.pool; 226 clone.listener = this.listener; 227 clone.implementationClassName = this.implementationClassName; 228 return clone; 229 } 230}