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.thrift; 019 020import java.io.IOException; 021import java.util.Arrays; 022import java.util.List; 023import java.util.Set; 024import java.util.concurrent.Callable; 025import java.util.concurrent.ConcurrentHashMap; 026import java.util.concurrent.ConcurrentMap; 027import java.util.concurrent.LinkedBlockingQueue; 028import java.util.concurrent.ThreadPoolExecutor; 029import java.util.concurrent.TimeUnit; 030import java.util.concurrent.atomic.LongAdder; 031import org.apache.hadoop.hbase.CellUtil; 032import org.apache.hadoop.hbase.client.Table; 033import org.apache.hadoop.hbase.thrift.generated.TIncrement; 034import org.apache.hadoop.hbase.util.Bytes; 035import org.apache.hadoop.hbase.util.Threads; 036import org.apache.hadoop.metrics2.util.MBeans; 037import org.apache.yetus.audience.InterfaceAudience; 038import org.slf4j.Logger; 039import org.slf4j.LoggerFactory; 040 041/** 042 * This class will coalesce increments from a thift server if 043 * hbase.regionserver.thrift.coalesceIncrement is set to true. Turning this 044 * config to true will cause the thrift server to queue increments into an 045 * instance of this class. The thread pool associated with this class will drain 046 * the coalesced increments as the thread is able. This can cause data loss if the 047 * thrift server dies or is shut down before everything in the queue is drained. 048 * 049 */ 050@InterfaceAudience.Private 051public class IncrementCoalescer implements IncrementCoalescerMBean { 052 /** 053 * Used to identify a cell that will be incremented. 054 * 055 */ 056 static class FullyQualifiedRow { 057 private byte[] table; 058 private byte[] rowKey; 059 private byte[] family; 060 private byte[] qualifier; 061 062 public FullyQualifiedRow(byte[] table, byte[] rowKey, byte[] fam, byte[] qual) { 063 super(); 064 this.table = table; 065 this.rowKey = rowKey; 066 this.family = fam; 067 this.qualifier = qual; 068 } 069 070 public byte[] getTable() { 071 return table; 072 } 073 074 public void setTable(byte[] table) { 075 this.table = table; 076 } 077 078 public byte[] getRowKey() { 079 return rowKey; 080 } 081 082 public byte[] getFamily() { 083 return family; 084 } 085 086 public void setFamily(byte[] fam) { 087 this.family = fam; 088 } 089 090 public byte[] getQualifier() { 091 return qualifier; 092 } 093 094 public void setQualifier(byte[] qual) { 095 this.qualifier = qual; 096 } 097 098 @Override 099 public int hashCode() { 100 final int prime = 31; 101 int result = 1; 102 result = prime * result + Arrays.hashCode(family); 103 result = prime * result + Arrays.hashCode(qualifier); 104 result = prime * result + Arrays.hashCode(rowKey); 105 result = prime * result + Arrays.hashCode(table); 106 return result; 107 } 108 109 @Override 110 public boolean equals(Object obj) { 111 if (this == obj) { 112 return true; 113 } 114 if (obj == null) { 115 return false; 116 } 117 if (getClass() != obj.getClass()) { 118 return false; 119 } 120 121 FullyQualifiedRow other = (FullyQualifiedRow) obj; 122 123 if (!Arrays.equals(family, other.family)) { 124 return false; 125 } 126 if (!Arrays.equals(qualifier, other.qualifier)) { 127 return false; 128 } 129 if (!Arrays.equals(rowKey, other.rowKey)) { 130 return false; 131 } 132 133 return Arrays.equals(table, other.table); 134 } 135 } 136 137 private final LongAdder failedIncrements = new LongAdder(); 138 private final LongAdder successfulCoalescings = new LongAdder(); 139 private final LongAdder totalIncrements = new LongAdder(); 140 private final ConcurrentMap<FullyQualifiedRow, Long> countersMap = 141 new ConcurrentHashMap<>(100000, 0.75f, 1500); 142 private final ThreadPoolExecutor pool; 143 private final ThriftHBaseServiceHandler handler; 144 145 private int maxQueueSize = 500000; 146 private static final int CORE_POOL_SIZE = 1; 147 148 private static final Logger LOG = LoggerFactory.getLogger(IncrementCoalescer.class); 149 150 public IncrementCoalescer(ThriftHBaseServiceHandler hand) { 151 this.handler = hand; 152 LinkedBlockingQueue<Runnable> queue = new LinkedBlockingQueue<>(); 153 pool = new ThreadPoolExecutor(CORE_POOL_SIZE, CORE_POOL_SIZE, 50, 154 TimeUnit.MILLISECONDS, queue, 155 Threads.newDaemonThreadFactory("IncrementCoalescer")); 156 MBeans.register("thrift", "Thrift", this); 157 } 158 159 public boolean queueIncrement(TIncrement inc) { 160 if (!canQueue()) { 161 failedIncrements.increment(); 162 return false; 163 } 164 return internalQueueTincrement(inc); 165 } 166 167 public boolean queueIncrements(List<TIncrement> incs) { 168 if (!canQueue()) { 169 failedIncrements.increment(); 170 return false; 171 } 172 173 for (TIncrement tinc : incs) { 174 internalQueueTincrement(tinc); 175 } 176 return true; 177 } 178 179 private boolean internalQueueTincrement(TIncrement inc) { 180 byte[][] famAndQf = CellUtil.parseColumn(inc.getColumn()); 181 182 if (famAndQf.length != 2) { 183 return false; 184 } 185 186 return internalQueueIncrement(inc.getTable(), inc.getRow(), famAndQf[0], famAndQf[1], 187 inc.getAmmount()); 188 } 189 190 @SuppressWarnings("FutureReturnValueIgnored") 191 private boolean internalQueueIncrement(byte[] tableName, byte[] rowKey, byte[] fam, 192 byte[] qual, long ammount) { 193 int countersMapSize = countersMap.size(); 194 195 //Make sure that the number of threads is scaled. 196 dynamicallySetCoreSize(countersMapSize); 197 198 totalIncrements.increment(); 199 200 FullyQualifiedRow key = new FullyQualifiedRow(tableName, rowKey, fam, qual); 201 202 long currentAmount = ammount; 203 // Spin until able to insert the value back without collisions 204 while (true) { 205 Long value = countersMap.remove(key); 206 if (value == null) { 207 // There was nothing there, create a new value 208 value = currentAmount; 209 } else { 210 value += currentAmount; 211 successfulCoalescings.increment(); 212 } 213 // Try to put the value, only if there was none 214 Long oldValue = countersMap.putIfAbsent(key, value); 215 if (oldValue == null) { 216 // We were able to put it in, we're done 217 break; 218 } 219 // Someone else was able to put a value in, so let's remember our 220 // current value (plus what we picked up) and retry to add it in 221 currentAmount = value; 222 } 223 224 // We limit the size of the queue simply because all we need is a 225 // notification that something needs to be incremented. No need 226 // for millions of callables that mean the same thing. 227 if (pool.getQueue().size() <= 1000) { 228 // queue it up 229 Callable<Integer> callable = createIncCallable(); 230 pool.submit(callable); 231 } 232 233 return true; 234 } 235 236 public boolean canQueue() { 237 return countersMap.size() < maxQueueSize; 238 } 239 240 private Callable<Integer> createIncCallable() { 241 return () -> { 242 int failures = 0; 243 Set<FullyQualifiedRow> keys = countersMap.keySet(); 244 for (FullyQualifiedRow row : keys) { 245 Long counter = countersMap.remove(row); 246 if (counter == null) { 247 continue; 248 } 249 Table table = null; 250 try { 251 table = handler.getTable(row.getTable()); 252 if (failures > 2) { 253 throw new IOException("Auto-Fail rest of ICVs"); 254 } 255 table.incrementColumnValue(row.getRowKey(), row.getFamily(), row.getQualifier(), 256 counter); 257 } catch (IOException e) { 258 // log failure of increment 259 failures++; 260 LOG.error("FAILED_ICV: " + Bytes.toString(row.getTable()) + ", " 261 + Bytes.toStringBinary(row.getRowKey()) + ", " 262 + Bytes.toStringBinary(row.getFamily()) + ", " 263 + Bytes.toStringBinary(row.getQualifier()) + ", " + counter, e); 264 } finally{ 265 if(table != null){ 266 table.close(); 267 } 268 } 269 } 270 return failures; 271 }; 272 } 273 274 /** 275 * This method samples the incoming requests and, if selected, will check if 276 * the corePoolSize should be changed. 277 * @param countersMapSize 278 */ 279 private void dynamicallySetCoreSize(int countersMapSize) { 280 // Here we are using countersMapSize as a random number, meaning this 281 // could be a Random object 282 if (countersMapSize % 10 != 0) { 283 return; 284 } 285 double currentRatio = (double) countersMapSize / (double) maxQueueSize; 286 int newValue; 287 288 if (currentRatio < 0.1) { 289 newValue = 1; 290 } else if (currentRatio < 0.3) { 291 newValue = 2; 292 } else if (currentRatio < 0.5) { 293 newValue = 4; 294 } else if (currentRatio < 0.7) { 295 newValue = 8; 296 } else if (currentRatio < 0.9) { 297 newValue = 14; 298 } else { 299 newValue = 22; 300 } 301 302 if (pool.getCorePoolSize() != newValue) { 303 pool.setCorePoolSize(newValue); 304 } 305 } 306 307 // MBean get/set methods 308 @Override 309 public int getQueueSize() { 310 return pool.getQueue().size(); 311 } 312 313 @Override 314 public int getMaxQueueSize() { 315 return this.maxQueueSize; 316 } 317 318 @Override 319 public void setMaxQueueSize(int newSize) { 320 this.maxQueueSize = newSize; 321 } 322 323 @Override 324 public long getPoolCompletedTaskCount() { 325 return pool.getCompletedTaskCount(); 326 } 327 328 @Override 329 public long getPoolTaskCount() { 330 return pool.getTaskCount(); 331 } 332 333 @Override 334 public int getPoolLargestPoolSize() { 335 return pool.getLargestPoolSize(); 336 } 337 338 @Override 339 public int getCorePoolSize() { 340 return pool.getCorePoolSize(); 341 } 342 343 @Override 344 public void setCorePoolSize(int newCoreSize) { 345 pool.setCorePoolSize(newCoreSize); 346 } 347 348 @Override 349 public int getMaxPoolSize() { 350 return pool.getMaximumPoolSize(); 351 } 352 353 @Override 354 public void setMaxPoolSize(int newMaxSize) { 355 pool.setMaximumPoolSize(newMaxSize); 356 } 357 358 @Override 359 public long getFailedIncrements() { 360 return failedIncrements.sum(); 361 } 362 363 @Override 364 public long getSuccessfulCoalescings() { 365 return successfulCoalescings.sum(); 366 } 367 368 @Override 369 public long getTotalIncrements() { 370 return totalIncrements.sum(); 371 } 372 373 @Override 374 public long getCountersMapSize() { 375 return countersMap.size(); 376 } 377}