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. n
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}