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}