View Javadoc

1   /*
2    * Copyright The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  package org.apache.hadoop.hbase.thrift;
21  
22  import java.io.IOException;
23  import java.util.Arrays;
24  import java.util.List;
25  import java.util.Set;
26  import java.util.concurrent.Callable;
27  import java.util.concurrent.ConcurrentHashMap;
28  import java.util.concurrent.ConcurrentMap;
29  import java.util.concurrent.LinkedBlockingQueue;
30  import java.util.concurrent.ThreadFactory;
31  import java.util.concurrent.ThreadPoolExecutor;
32  import java.util.concurrent.TimeUnit;
33  import java.util.concurrent.atomic.AtomicInteger;
34  import java.util.concurrent.atomic.AtomicLong;
35  
36  import org.apache.commons.logging.Log;
37  import org.apache.commons.logging.LogFactory;
38  import org.apache.hadoop.hbase.KeyValue;
39  import org.apache.hadoop.hbase.client.HTable;
40  import org.apache.hadoop.hbase.thrift.ThriftServerRunner.HBaseHandler;
41  import org.apache.hadoop.hbase.thrift.generated.TIncrement;
42  import org.apache.hadoop.hbase.util.Bytes;
43  import org.apache.hadoop.metrics.util.MBeanUtil;
44  import org.apache.thrift.TException;
45  
46  /**
47   * This class will coalesce increments from a thift server if
48   * hbase.regionserver.thrift.coalesceIncrement is set to true. Turning this
49   * config to true will cause the thrift server to queue increments into an
50   * instance of this class. The thread pool associated with this class will drain
51   * the coalesced increments as the thread is able. This can cause data loss if the
52   * thrift server dies or is shut down before everything in the queue is drained.
53   *
54   */
55  public class IncrementCoalescer implements IncrementCoalescerMBean {
56  
57    /**
58     * Used to identify a cell that will be incremented.
59     *
60     */
61    static class FullyQualifiedRow {
62      private byte[] table;
63      private byte[] rowKey;
64      private byte[] family;
65      private byte[] qualifier;
66  
67      public FullyQualifiedRow(byte[] table, byte[] rowKey, byte[] fam, byte[] qual) {
68        super();
69        this.table = table;
70        this.rowKey = rowKey;
71        this.family = fam;
72        this.qualifier = qual;
73      }
74  
75      public byte[] getTable() {
76        return table;
77      }
78  
79      public void setTable(byte[] table) {
80        this.table = table;
81      }
82  
83      public byte[] getRowKey() {
84        return rowKey;
85      }
86  
87      public void setRowKey(byte[] rowKey) {
88        this.rowKey = rowKey;
89      }
90  
91      public byte[] getFamily() {
92        return family;
93      }
94  
95      public void setFamily(byte[] fam) {
96        this.family = fam;
97      }
98  
99      public byte[] getQualifier() {
100       return qualifier;
101     }
102 
103     public void setQualifier(byte[] qual) {
104       this.qualifier = qual;
105     }
106 
107     @Override
108     public int hashCode() {
109       final int prime = 31;
110       int result = 1;
111       result = prime * result + Arrays.hashCode(family);
112       result = prime * result + Arrays.hashCode(qualifier);
113       result = prime * result + Arrays.hashCode(rowKey);
114       result = prime * result + Arrays.hashCode(table);
115       return result;
116     }
117 
118     @Override
119     public boolean equals(Object obj) {
120       if (this == obj) return true;
121       if (obj == null) return false;
122       if (getClass() != obj.getClass()) return false;
123       FullyQualifiedRow other = (FullyQualifiedRow) obj;
124       if (!Arrays.equals(family, other.family)) return false;
125       if (!Arrays.equals(qualifier, other.qualifier)) return false;
126       if (!Arrays.equals(rowKey, other.rowKey)) return false;
127       if (!Arrays.equals(table, other.table)) return false;
128       return true;
129     }
130 
131   }
132 
133   static class DaemonThreadFactory implements ThreadFactory {
134     static final AtomicInteger poolNumber = new AtomicInteger(1);
135     final ThreadGroup group;
136     final AtomicInteger threadNumber = new AtomicInteger(1);
137     final String namePrefix;
138 
139     DaemonThreadFactory() {
140       SecurityManager s = System.getSecurityManager();
141       group = (s != null) ? s.getThreadGroup() : Thread.currentThread().getThreadGroup();
142       namePrefix = "ICV-" + poolNumber.getAndIncrement() + "-thread-";
143     }
144 
145     public Thread newThread(Runnable r) {
146       Thread t = new Thread(group, r, namePrefix + threadNumber.getAndIncrement(), 0);
147       if (!t.isDaemon()) t.setDaemon(true);
148       if (t.getPriority() != Thread.NORM_PRIORITY) t.setPriority(Thread.NORM_PRIORITY);
149       return t;
150     }
151   }
152 
153   private final AtomicLong failedIncrements = new AtomicLong();
154   private final AtomicLong successfulCoalescings = new AtomicLong();
155   private final AtomicLong totalIncrements = new AtomicLong();
156   private final ConcurrentMap<FullyQualifiedRow, Long> countersMap =
157       new ConcurrentHashMap<FullyQualifiedRow, Long>(100000, 0.75f, 1500);
158   private final ThreadPoolExecutor pool;
159   private final HBaseHandler handler;
160 
161   private int maxQueueSize = 500000;
162   private static final int CORE_POOL_SIZE = 1;
163 
164   protected final Log LOG = LogFactory.getLog(this.getClass().getName());
165 
166   @SuppressWarnings("deprecation")
167   public IncrementCoalescer(HBaseHandler hand) {
168     this.handler = hand;
169     LinkedBlockingQueue<Runnable> queue = new LinkedBlockingQueue<Runnable>();
170     pool =
171         new ThreadPoolExecutor(CORE_POOL_SIZE, CORE_POOL_SIZE, 50, TimeUnit.MILLISECONDS, queue,
172             new DaemonThreadFactory());
173 
174     MBeanUtil.registerMBean("thrift", "Thrift", this);
175   }
176 
177   public boolean queueIncrement(TIncrement inc) throws TException {
178     if (!canQueue()) {
179       failedIncrements.incrementAndGet();
180       return false;
181     }
182     return internalQueueTincrement(inc);
183   }
184 
185   public boolean queueIncrements(List<TIncrement> incs) throws TException {
186     if (!canQueue()) {
187       failedIncrements.incrementAndGet();
188       return false;
189     }
190 
191     for (TIncrement tinc : incs) {
192       internalQueueTincrement(tinc);
193     }
194     return true;
195 
196   }
197 
198   private boolean internalQueueTincrement(TIncrement inc) throws TException {
199     byte[][] famAndQf = KeyValue.parseColumn(inc.getColumn());
200     if (famAndQf.length < 1) return false;
201     byte[] qual = famAndQf.length == 1 ? new byte[0] : famAndQf[1];
202 
203     return internalQueueIncrement(inc.getTable(), inc.getRow(), famAndQf[0], qual,
204       inc.getAmmount());
205 
206   }
207 
208   private boolean internalQueueIncrement(byte[] tableName, byte[] rowKey, byte[] fam,
209       byte[] qual, long ammount) throws TException {
210     int countersMapSize = countersMap.size();
211 
212 
213     //Make sure that the number of threads is scaled.
214     dynamicallySetCoreSize(countersMapSize);
215 
216     totalIncrements.incrementAndGet();
217 
218     FullyQualifiedRow key = new FullyQualifiedRow(tableName, rowKey, fam, qual);
219 
220     long currentAmount = ammount;
221     // Spin until able to insert the value back without collisions
222     while (true) {
223       Long value = countersMap.remove(key);
224       if (value == null) {
225         // There was nothing there, create a new value
226         value = new Long(currentAmount);
227       } else {
228         value += currentAmount;
229         successfulCoalescings.incrementAndGet();
230       }
231       // Try to put the value, only if there was none
232       Long oldValue = countersMap.putIfAbsent(key, value);
233       if (oldValue == null) {
234         // We were able to put it in, we're done
235         break;
236       }
237       // Someone else was able to put a value in, so let's remember our
238       // current value (plus what we picked up) and retry to add it in
239       currentAmount = value;
240     }
241 
242     // We limit the size of the queue simply because all we need is a
243     // notification that something needs to be incremented. No need
244     // for millions of callables that mean the same thing.
245     if (pool.getQueue().size() <= 1000) {
246       // queue it up
247       Callable<Integer> callable = createIncCallable();
248       pool.submit(callable);
249     }
250 
251     return true;
252   }
253 
254   public boolean canQueue() {
255     return countersMap.size() < maxQueueSize;
256   }
257 
258   private Callable<Integer> createIncCallable() {
259     return new Callable<Integer>() {
260       @Override
261       public Integer call() throws Exception {
262         int failures = 0;
263         Set<FullyQualifiedRow> keys = countersMap.keySet();
264         for (FullyQualifiedRow row : keys) {
265           Long counter = countersMap.remove(row);
266           if (counter == null) {
267             continue;
268           }
269           try {
270             HTable table = handler.getTable(row.getTable());
271             if (failures > 2) {
272               throw new IOException("Auto-Fail rest of ICVs");
273             }
274             table.incrementColumnValue(row.getRowKey(), row.getFamily(), row.getQualifier(),
275               counter);
276           } catch (IOException e) {
277             // log failure of increment
278             failures++;
279             LOG.error("FAILED_ICV: " + Bytes.toString(row.getTable()) + ", "
280                 + Bytes.toStringBinary(row.getRowKey()) + ", "
281                 + Bytes.toStringBinary(row.getFamily()) + ", "
282                 + Bytes.toStringBinary(row.getQualifier()) + ", " + counter);
283           }
284 
285         }
286         return failures;
287       }
288     };
289   }
290 
291   /**
292    * This method samples the incoming requests and, if selected, will check if
293    * the corePoolSize should be changed.
294    * @param countersMapSize
295    */
296   private void dynamicallySetCoreSize(int countersMapSize) {
297     // Here we are using countersMapSize as a random number, meaning this
298     // could be a Random object
299     if (countersMapSize % 10 != 0) {
300       return;
301     }
302     double currentRatio = (double) countersMapSize / (double) maxQueueSize;
303     int newValue = 1;
304     if (currentRatio < 0.1) {
305       // it's 1
306     } else if (currentRatio < 0.3) {
307       newValue = 2;
308     } else if (currentRatio < 0.5) {
309       newValue = 4;
310     } else if (currentRatio < 0.7) {
311       newValue = 8;
312     } else if (currentRatio < 0.9) {
313       newValue = 14;
314     } else {
315       newValue = 22;
316     }
317     if (pool.getCorePoolSize() != newValue) {
318       pool.setCorePoolSize(newValue);
319     }
320   }
321 
322   // MBean get/set methods
323   public int getQueueSize() {
324     return pool.getQueue().size();
325   }
326   public int getMaxQueueSize() {
327     return this.maxQueueSize;
328   }
329   public void setMaxQueueSize(int newSize) {
330     this.maxQueueSize = newSize;
331   }
332 
333   public long getPoolCompletedTaskCount() {
334     return pool.getCompletedTaskCount();
335   }
336   public long getPoolTaskCount() {
337     return pool.getTaskCount();
338   }
339   public int getPoolLargestPoolSize() {
340     return pool.getLargestPoolSize();
341   }
342   public int getCorePoolSize() {
343     return pool.getCorePoolSize();
344   }
345   public void setCorePoolSize(int newCoreSize) {
346     pool.setCorePoolSize(newCoreSize);
347   }
348   public int getMaxPoolSize() {
349     return pool.getMaximumPoolSize();
350   }
351   public void setMaxPoolSize(int newMaxSize) {
352     pool.setMaximumPoolSize(newMaxSize);
353   }
354   public long getFailedIncrements() {
355     return failedIncrements.get();
356   }
357 
358   public long getSuccessfulCoalescings() {
359     return successfulCoalescings.get();
360   }
361 
362   public long getTotalIncrements() {
363     return totalIncrements.get();
364   }
365 
366   public long getCountersMapSize() {
367     return countersMap.size();
368   }
369 
370 }