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