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 public long getWriteBufferPeriodicFlushTimerTickMs() { 127 return writeBufferPeriodicFlushTimerTickMs; 128 } 129 130 /** 131 * Set the TimerTick how often the buffer timeout if checked. 132 */ 133 public BufferedMutatorParams setWriteBufferPeriodicFlushTimerTickMs(long timerTickMs) { 134 this.writeBufferPeriodicFlushTimerTickMs = timerTickMs; 135 return this; 136 } 137 138 public int getMaxKeyValueSize() { 139 return maxKeyValueSize; 140 } 141 142 /** 143 * Override the maximum key-value size specified by the provided {@link Connection}'s 144 * {@link org.apache.hadoop.conf.Configuration} instance, via the configuration key 145 * {@code hbase.client.keyvalue.maxsize}. 146 */ 147 public BufferedMutatorParams maxKeyValueSize(int maxKeyValueSize) { 148 this.maxKeyValueSize = maxKeyValueSize; 149 return this; 150 } 151 152 public ExecutorService getPool() { 153 return pool; 154 } 155 156 /** 157 * Override the default executor pool defined by the {@code hbase.htable.threads.*} configuration 158 * values. 159 */ 160 public BufferedMutatorParams pool(ExecutorService pool) { 161 this.pool = pool; 162 return this; 163 } 164 165 /** 166 * Returns Name of the class we will use when we construct a {@link BufferedMutator} instance or 167 * null if default implementation. 168 */ 169 public String getImplementationClassName() { 170 return this.implementationClassName; 171 } 172 173 /** 174 * Specify a BufferedMutator implementation other than the default. 175 * @param implementationClassName Name of the BufferedMutator implementation class 176 */ 177 public BufferedMutatorParams implementationClassName(String implementationClassName) { 178 this.implementationClassName = implementationClassName; 179 return this; 180 } 181 182 public BufferedMutator.ExceptionListener getListener() { 183 return listener; 184 } 185 186 /** 187 * Override the default error handler. Default handler simply rethrows the exception. 188 */ 189 public BufferedMutatorParams listener(BufferedMutator.ExceptionListener listener) { 190 this.listener = listener; 191 return this; 192 } 193 194 /* 195 * (non-Javadoc) 196 * @see java.lang.Object#clone() 197 */ 198 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "CN_IDIOM_NO_SUPER_CALL", 199 justification = "The clone below is complete") 200 @Override 201 public BufferedMutatorParams clone() { 202 BufferedMutatorParams clone = new BufferedMutatorParams(this.tableName); 203 clone.writeBufferSize = this.writeBufferSize; 204 clone.writeBufferPeriodicFlushTimeoutMs = this.writeBufferPeriodicFlushTimeoutMs; 205 clone.writeBufferPeriodicFlushTimerTickMs = this.writeBufferPeriodicFlushTimerTickMs; 206 clone.maxKeyValueSize = this.maxKeyValueSize; 207 clone.pool = this.pool; 208 clone.listener = this.listener; 209 clone.implementationClassName = this.implementationClassName; 210 return clone; 211 } 212}