View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one or more contributor license
4    * agreements. See the NOTICE file distributed with this work for additional information regarding
5    * copyright ownership. The ASF licenses this file to you under the Apache License, Version 2.0 (the
6    * "License"); you may not use this file except in compliance with the License. You may obtain a
7    * copy of the License at
8    *
9    *  http://www.apache.org/licenses/LICENSE-2.0
10   *
11   *  Unless required by applicable law or agreed to in writing, software distributed under the
12   * License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
13   * express or implied. See the License for the specific language governing permissions and
14   * limitations under the License.
15   */
16  package org.apache.hadoop.hbase.client;
17  
18  import static org.apache.hadoop.hbase.client.BufferedMutatorParams.UNSET;
19  import java.io.Closeable;
20  import java.io.IOException;
21  import java.io.InterruptedIOException;
22  import java.util.Arrays;
23  import java.util.Iterator;
24  import java.util.List;
25  import java.util.NoSuchElementException;
26  import java.util.Timer;
27  import java.util.TimerTask;
28  import java.util.concurrent.ConcurrentLinkedQueue;
29  import java.util.concurrent.ExecutorService;
30  import java.util.concurrent.TimeUnit;
31  import java.util.concurrent.atomic.AtomicInteger;
32  import java.util.concurrent.atomic.AtomicLong;
33  import org.apache.commons.logging.Log;
34  import org.apache.commons.logging.LogFactory;
35  import org.apache.hadoop.conf.Configuration;
36  import org.apache.hadoop.hbase.TableName;
37  import org.apache.hadoop.hbase.classification.InterfaceAudience;
38  import org.apache.hadoop.hbase.classification.InterfaceStability;
39  import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
40  
41  /**
42   * <p>
43   * Used to communicate with a single HBase table similar to {@link HTable}
44   * but meant for batched, potentially asynchronous puts. Obtain an instance from
45   * a {@link Connection} and call {@link #close()} afterwards.
46   * </p>
47   *
48   * <p>
49   * While this can be used accross threads, great care should be used when doing so.
50   * Errors are global to the buffered mutator and the Exceptions can be thrown on any
51   * thread that causes the flush for requests.
52   * </p>
53   *
54   * @see ConnectionFactory
55   * @see Connection
56   * @since 1.0.0
57   */
58  @InterfaceAudience.Private
59  @InterfaceStability.Evolving
60  public class BufferedMutatorImpl implements BufferedMutator {
61  
62    private static final Log LOG = LogFactory.getLog(BufferedMutatorImpl.class);
63  
64    private final ExceptionListener listener;
65  
66    protected ClusterConnection connection; // non-final so can be overridden in test
67    private final TableName tableName;
68    private volatile Configuration conf;
69    final ConcurrentLinkedQueue<Mutation> writeAsyncBuffer = new ConcurrentLinkedQueue<Mutation>();
70    AtomicLong currentWriteBufferSize = new AtomicLong(0);
71  
72    /**
73     * Count the size of {@link BufferedMutatorImpl#writeAsyncBuffer}.
74     * The {@link ConcurrentLinkedQueue#size()} is NOT a constant-time operation.
75     */
76    final AtomicInteger undealtMutationCount = new AtomicInteger(0);
77    private long writeBufferSize;
78    /**
79     * Having the timer tick run more often that once every 100ms is needless and will
80     * probably cause too many timer events firing having a negative impact on performance.
81     */
82    public static final long MIN_WRITE_BUFFER_PERIODIC_FLUSH_TIMERTICK_MS = 100;
83  
84    private final AtomicLong writeBufferPeriodicFlushTimeoutMs = new AtomicLong(0);
85    private final AtomicLong writeBufferPeriodicFlushTimerTickMs =
86            new AtomicLong(MIN_WRITE_BUFFER_PERIODIC_FLUSH_TIMERTICK_MS);
87    private Timer writeBufferPeriodicFlushTimer = null;
88  
89    private final int maxKeyValueSize;
90    private boolean closed = false;
91    private final ExecutorService pool;
92    private int writeRpcTimeout; // needed to pass in through AsyncProcess constructor
93    private int operationTimeout;
94  
95    protected AsyncProcess ap; // non-final so can be overridden in test
96  
97    BufferedMutatorImpl(ClusterConnection conn, RpcRetryingCallerFactory rpcCallerFactory,
98        RpcControllerFactory rpcFactory, BufferedMutatorParams params) {
99      if (conn == null || conn.isClosed()) {
100       throw new IllegalArgumentException("Connection is null or closed.");
101     }
102 
103     this.tableName = params.getTableName();
104     this.connection = conn;
105     this.conf = connection.getConfiguration();
106     this.pool = params.getPool();
107     this.listener = params.getListener();
108 
109     ConnectionConfiguration connConf = conn.getConnectionConfiguration();
110     if (connConf == null) {
111       // Slow: parse conf in ConnectionConfiguration constructor
112       connConf = new ConnectionConfiguration(conf);
113     }
114     this.writeBufferSize = params.getWriteBufferSize() != BufferedMutatorParams.UNSET ?
115         params.getWriteBufferSize() : connConf.getWriteBufferSize();
116 
117     // Set via the setter because it does value validation and starts/stops the TimerTask
118     long newWriteBufferPeriodicFlushTimeoutMs =
119             params.getWriteBufferPeriodicFlushTimeoutMs() != UNSET
120                     ? params.getWriteBufferPeriodicFlushTimeoutMs()
121                     : connConf.getWriteBufferPeriodicFlushTimeoutMs();
122     long newWriteBufferPeriodicFlushTimerTickMs =
123             params.getWriteBufferPeriodicFlushTimerTickMs() != UNSET
124                     ? params.getWriteBufferPeriodicFlushTimerTickMs()
125                     : connConf.getWriteBufferPeriodicFlushTimerTickMs();
126     this.setWriteBufferPeriodicFlush(
127             newWriteBufferPeriodicFlushTimeoutMs,
128             newWriteBufferPeriodicFlushTimerTickMs);
129 
130     this.maxKeyValueSize = params.getMaxKeyValueSize() != BufferedMutatorParams.UNSET ?
131         params.getMaxKeyValueSize() : connConf.getMaxKeyValueSize();
132 
133     this.writeRpcTimeout = connConf.getWriteRpcTimeout();
134     this.operationTimeout = connConf.getOperationTimeout();
135     // puts need to track errors globally due to how the APIs currently work.
136     ap = new AsyncProcess(connection, conf, pool, rpcCallerFactory, true, rpcFactory, writeRpcTimeout);
137   }
138 
139   @Override
140   public TableName getName() {
141     return tableName;
142   }
143 
144   @Override
145   public Configuration getConfiguration() {
146     return conf;
147   }
148 
149   @Override
150   public void mutate(Mutation m) throws InterruptedIOException,
151       RetriesExhaustedWithDetailsException {
152     mutate(Arrays.asList(m));
153   }
154 
155   @Override
156   public void mutate(List<? extends Mutation> ms) throws InterruptedIOException,
157       RetriesExhaustedWithDetailsException {
158 
159     if (closed) {
160       throw new IllegalStateException("Cannot put when the BufferedMutator is closed.");
161     }
162 
163     long toAddSize = 0;
164     int toAddCount = 0;
165     for (Mutation m : ms) {
166       if (m instanceof Put) {
167         validatePut((Put) m);
168       }
169       toAddSize += m.heapSize();
170       ++toAddCount;
171     }
172 
173     if (currentWriteBufferSize.get() == 0) {
174       firstRecordInBufferTimestamp.set(System.currentTimeMillis());
175     }
176 
177     // This behavior is highly non-intuitive... it does not protect us against
178     // 94-incompatible behavior, which is a timing issue because hasError, the below code
179     // and setter of hasError are not synchronized. Perhaps it should be removed.
180     if (ap.hasError()) {
181       currentWriteBufferSize.addAndGet(toAddSize);
182       writeAsyncBuffer.addAll(ms);
183       undealtMutationCount.addAndGet(toAddCount);
184       backgroundFlushCommits(true);
185     } else {
186       currentWriteBufferSize.addAndGet(toAddSize);
187       writeAsyncBuffer.addAll(ms);
188       undealtMutationCount.addAndGet(toAddCount);
189     }
190 
191     // Now try and queue what needs to be queued.
192     while (undealtMutationCount.get() != 0
193         && currentWriteBufferSize.get() > writeBufferSize) {
194       backgroundFlushCommits(false);
195     }
196   }
197 
198   protected long getExecutedWriteBufferPeriodicFlushes() {
199     return executedWriteBufferPeriodicFlushes.get();
200   }
201 
202   private final AtomicLong firstRecordInBufferTimestamp = new AtomicLong(0);
203   private final AtomicLong executedWriteBufferPeriodicFlushes = new AtomicLong(0);
204 
205   private void timerCallbackForWriteBufferPeriodicFlush() {
206     if (currentWriteBufferSize.get() == 0) {
207       return; // Nothing to flush
208     }
209     long now = System.currentTimeMillis();
210     if (firstRecordInBufferTimestamp.get() + writeBufferPeriodicFlushTimeoutMs.get() > now) {
211       return; // No need to flush yet
212     }
213     // The first record in the writebuffer has been in there too long --> flush
214     try {
215       executedWriteBufferPeriodicFlushes.incrementAndGet();
216       flush();
217     } catch (InterruptedIOException | RetriesExhaustedWithDetailsException e) {
218       LOG.error("Exception during timerCallbackForWriteBufferPeriodicFlush --> " + e.getMessage());
219     }
220   }
221 
222   // validate for well-formedness
223   public void validatePut(final Put put) throws IllegalArgumentException {
224     HTable.validatePut(put, maxKeyValueSize);
225   }
226 
227   @Override
228   public synchronized void close() throws IOException {
229     try {
230       if (this.closed) {
231         return;
232       }
233 
234       // Stop any running Periodic Flush timer.
235       disableWriteBufferPeriodicFlush();
236 
237       // As we can have an operation in progress even if the buffer is empty, we call
238       // backgroundFlushCommits at least one time.
239       backgroundFlushCommits(true);
240       this.pool.shutdown();
241       boolean terminated;
242       int loopCnt = 0;
243       do {
244         // wait until the pool has terminated
245         terminated = this.pool.awaitTermination(60, TimeUnit.SECONDS);
246         loopCnt += 1;
247         if (loopCnt >= 10) {
248           LOG.warn("close() failed to terminate pool after 10 minutes. Abandoning pool.");
249           break;
250         }
251       } while (!terminated);
252 
253     } catch (InterruptedException e) {
254       LOG.warn("waitForTermination interrupted");
255 
256     } finally {
257       this.closed = true;
258     }
259   }
260 
261   @Override
262   public synchronized void flush() throws InterruptedIOException,
263       RetriesExhaustedWithDetailsException {
264     // As we can have an operation in progress even if the buffer is empty, we call
265     // backgroundFlushCommits at least one time.
266     backgroundFlushCommits(true);
267   }
268 
269   /**
270    * Send the operations in the buffer to the servers. Does not wait for the server's answer. If
271    * the is an error (max retried reach from a previous flush or bad operation), it tries to send
272    * all operations in the buffer and sends an exception.
273    *
274    * @param synchronous - if true, sends all the writes and wait for all of them to finish before
275    *        returning.
276    */
277   private void backgroundFlushCommits(boolean synchronous) throws
278       InterruptedIOException,
279       RetriesExhaustedWithDetailsException {
280     if (!synchronous && writeAsyncBuffer.isEmpty()) {
281       return;
282     }
283 
284     if (!synchronous) {
285       try (QueueRowAccess taker = createQueueRowAccess()){
286         ap.submit(tableName, taker, true, null, false);
287         if (ap.hasError()) {
288           LOG.debug(tableName + ": One or more of the operations have failed -"
289               + " waiting for all operation in progress to finish (successfully or not)");
290         }
291       }
292     }
293     if (synchronous || ap.hasError()) {
294       while (true) {
295         try (QueueRowAccess taker = createQueueRowAccess()){
296           if (taker.isEmpty()) {
297             break;
298           }
299           ap.submit(tableName, taker, true, null, false);
300         }
301       }
302 
303       RetriesExhaustedWithDetailsException error =
304           ap.waitForAllPreviousOpsAndReset(null, tableName.getNameAsString());
305       if (error != null) {
306         if (listener == null) {
307           throw error;
308         } else {
309           this.listener.onException(error, this);
310         }
311       }
312     }
313   }
314 
315   /**
316    * This is used for legacy purposes in {@link HTable#setWriteBufferSize(long)} only. This ought
317    * not be called for production uses.
318    * @deprecated Going away when we drop public support for {@link HTableInterface}.
319    */
320   @Deprecated
321   public void setWriteBufferSize(long writeBufferSize) throws RetriesExhaustedWithDetailsException,
322       InterruptedIOException {
323     this.writeBufferSize = writeBufferSize;
324     if (currentWriteBufferSize.get() > writeBufferSize) {
325       flush();
326     }
327   }
328 
329   /**
330    * {@inheritDoc}
331    */
332   @Override
333   public long getWriteBufferSize() {
334     return this.writeBufferSize;
335   }
336 
337   /**
338    * Sets the maximum time before the buffer is automatically flushed.
339    * @param timeoutMs    The maximum number of milliseconds how long records may be buffered
340    *                     before they are flushed. Set to 0 to disable.
341    * @param timerTickMs  The number of milliseconds between each check if the
342    *                     timeout has been exceeded. Must be 100ms (as defined in
343    *                     {@link #MIN_WRITE_BUFFER_PERIODIC_FLUSH_TIMERTICK_MS})
344    *                     or larger to avoid performance problems.
345    */
346   public synchronized void setWriteBufferPeriodicFlush(long timeoutMs, long timerTickMs) {
347     long originalTimeoutMs   = this.writeBufferPeriodicFlushTimeoutMs.get();
348     long originalTimerTickMs = this.writeBufferPeriodicFlushTimerTickMs.get();
349 
350     // Both parameters have minimal values.
351     writeBufferPeriodicFlushTimeoutMs.set(Math.max(0, timeoutMs));
352     writeBufferPeriodicFlushTimerTickMs.set(
353             Math.max(MIN_WRITE_BUFFER_PERIODIC_FLUSH_TIMERTICK_MS, timerTickMs));
354 
355     // If something changed we stop the old Timer.
356     if (writeBufferPeriodicFlushTimeoutMs.get() != originalTimeoutMs ||
357         writeBufferPeriodicFlushTimerTickMs.get() != originalTimerTickMs) {
358       if (writeBufferPeriodicFlushTimer != null) {
359         writeBufferPeriodicFlushTimer.cancel();
360         writeBufferPeriodicFlushTimer = null;
361       }
362     }
363 
364     // If we have the need for a timer and there is none we start it
365     if (writeBufferPeriodicFlushTimer == null &&
366         writeBufferPeriodicFlushTimeoutMs.get() > 0) {
367       writeBufferPeriodicFlushTimer = new Timer(true); // Create Timer running as Daemon.
368       writeBufferPeriodicFlushTimer.schedule(new TimerTask() {
369         @Override
370         public void run() {
371           BufferedMutatorImpl.this.timerCallbackForWriteBufferPeriodicFlush();
372         }
373       }, writeBufferPeriodicFlushTimerTickMs.get(),
374          writeBufferPeriodicFlushTimerTickMs.get());
375     }
376   }
377 
378   /**
379    * Disable periodic flushing of the write buffer.
380    */
381   public void disableWriteBufferPeriodicFlush() {
382     setWriteBufferPeriodicFlush(0, MIN_WRITE_BUFFER_PERIODIC_FLUSH_TIMERTICK_MS);
383   }
384 
385   /**
386    * Sets the maximum time before the buffer is automatically flushed checking once per second.
387    * @param timeoutMs    The maximum number of milliseconds how long records may be buffered
388    *                     before they are flushed. Set to 0 to disable.
389    */
390   public void setWriteBufferPeriodicFlush(long timeoutMs) {
391     setWriteBufferPeriodicFlush(timeoutMs, 1000L);
392   }
393 
394   /**
395    * Returns the current periodic flush timeout value in milliseconds.
396    * @return The maximum number of milliseconds how long records may be buffered before they
397    *   are flushed. The value 0 means this is disabled.
398    */
399   public long getWriteBufferPeriodicFlushTimeoutMs() {
400     return writeBufferPeriodicFlushTimeoutMs.get();
401   }
402 
403   /**
404    * Returns the current periodic flush timertick interval in milliseconds.
405    * @return The number of milliseconds between each check if the timeout has been exceeded.
406    *   This value only has a real meaning if the timeout has been set to > 0
407    */
408   public long getWriteBufferPeriodicFlushTimerTickMs() {
409     return writeBufferPeriodicFlushTimerTickMs.get();
410   }
411 
412   public void setRpcTimeout(int writeRpcTimeout) {
413     this.writeRpcTimeout = writeRpcTimeout;
414     this.ap.setRpcTimeout(writeRpcTimeout);
415   }
416 
417   public void setOperationTimeout(int operationTimeout) {
418     this.operationTimeout = operationTimeout;
419     this.ap.setOperationTimeout(operationTimeout);
420   }
421 
422   long getCurrentWriteBufferSize() {
423     return currentWriteBufferSize.get();
424   }
425 
426   /**
427    * This is used for legacy purposes in {@link HTable#getWriteBuffer()} only. This should not beÓ
428    * called from production uses.
429    * @deprecated Going away when we drop public support for {@link HTableInterface}.
430 Ó   */
431   @Deprecated
432   public List<Row> getWriteBuffer() {
433     return Arrays.asList(writeAsyncBuffer.toArray(new Row[0]));
434   }
435 
436   QueueRowAccess createQueueRowAccess() {
437     return new QueueRowAccess();
438   }
439 
440   class QueueRowAccess implements RowAccess<Row>, Closeable {
441     private int remainder = undealtMutationCount.getAndSet(0);
442     private Mutation last = null;
443 
444     @Override
445     public Iterator<Row> iterator() {
446       return new Iterator<Row>() {
447         private int countDown = remainder;
448         @Override
449         public boolean hasNext() {
450           return countDown > 0;
451         }
452         @Override
453         public Row next() {
454           restoreLastMutation();
455           if (!hasNext()) {
456             throw new NoSuchElementException();
457           }
458           last = writeAsyncBuffer.poll();
459           if (last == null) {
460             throw new NoSuchElementException();
461           }
462           currentWriteBufferSize.addAndGet(-last.heapSize());
463           --countDown;
464           return last;
465         }
466         @Override
467         public void remove() {
468           if (last == null) {
469             throw new IllegalStateException();
470           }
471           --remainder;
472           last = null;
473         }
474       };
475     }
476 
477     private void restoreLastMutation() {
478       // restore the last mutation since it isn't submitted
479       if (last != null) {
480         writeAsyncBuffer.add(last);
481         currentWriteBufferSize.addAndGet(last.heapSize());
482         last = null;
483       }
484     }
485 
486     @Override
487     public int size() {
488       return remainder;
489     }
490 
491     @Override
492     public boolean isEmpty() {
493       return remainder <= 0;
494     }
495     @Override
496     public void close() {
497       restoreLastMutation();
498       if (remainder > 0) {
499         undealtMutationCount.addAndGet(remainder);
500         remainder = 0;
501       }
502     }
503   }
504 }