View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  package org.apache.hadoop.hbase.client;
20  
21  import org.apache.hadoop.conf.Configuration;
22  import org.apache.hadoop.hbase.TableName;
23  import org.apache.hadoop.hbase.classification.InterfaceAudience;
24  import org.apache.hadoop.hbase.classification.InterfaceStability;
25  
26  import java.io.Closeable;
27  import java.io.IOException;
28  import java.util.List;
29  
30  /**
31   * <p>Used to communicate with a single HBase table similar to {@link Table} but meant for
32   * batched, asynchronous puts. Obtain an instance from a {@link Connection} and call
33   * {@link #close()} afterwards. Customizations can be applied to the {@code BufferedMutator} via
34   * the {@link BufferedMutatorParams}.
35   * </p>
36   *
37   * <p>Exception handling with asynchronously via the {@link BufferedMutator.ExceptionListener}.
38   * The default implementation is to throw the exception upon receipt. This behavior can be
39   * overridden with a custom implementation, provided as a parameter with
40   * {@link BufferedMutatorParams#listener(BufferedMutator.ExceptionListener)}.</p>
41   *
42   * <p>Map/Reduce jobs are good use cases for using {@code BufferedMutator}. Map/reduce jobs
43   * benefit from batching, but have no natural flush point. {@code BufferedMutator} receives the
44   * puts from the M/R job and will batch puts based on some heuristic, such as the accumulated size
45   * of the puts, and submit batches of puts asynchronously so that the M/R logic can continue
46   * without interruption.
47   * </p>
48   *
49   * <p>{@code BufferedMutator} can also be used on more exotic circumstances. Map/Reduce batch jobs
50   * will have a single {@code BufferedMutator} per thread. A single {@code BufferedMutator} can
51   * also be effectively used in high volume online systems to batch puts, with the caveat that
52   * extreme circumstances, such as JVM or machine failure, may cause some data loss.</p>
53   *
54   * <p>NOTE: This class replaces the functionality that used to be available via
55   * {@link HTableInterface#setAutoFlush(boolean)} set to {@code false}.
56   * </p>
57   *
58   * <p>See also the {@code BufferedMutatorExample} in the hbase-examples module.</p>
59   * @see ConnectionFactory
60   * @see Connection
61   * @since 1.0.0
62   */
63  @InterfaceAudience.Public
64  @InterfaceStability.Evolving
65  public interface BufferedMutator extends Closeable {
66    /**
67     * Gets the fully qualified table name instance of the table that this BufferedMutator writes to.
68     */
69    TableName getName();
70  
71    /**
72     * Returns the {@link org.apache.hadoop.conf.Configuration} object used by this instance.
73     * <p>
74     * The reference returned is not a copy, so any change made to it will
75     * affect this instance.
76     */
77    Configuration getConfiguration();
78  
79    /**
80     * Sends a {@link Mutation} to the table. The mutations will be buffered and sent over the
81     * wire as part of a batch. Currently only supports {@link Put} and {@link Delete} mutations.
82     *
83     * @param mutation The data to send.
84     * @throws IOException if a remote or network exception occurs.
85     */
86    void mutate(Mutation mutation) throws IOException;
87  
88    /**
89     * Send some {@link Mutation}s to the table. The mutations will be buffered and sent over the
90     * wire as part of a batch. There is no guarantee of sending entire content of {@code mutations}
91     * in a single batch; it will be broken up according to the write buffer capacity.
92     *
93     * @param mutations The data to send.
94     * @throws IOException if a remote or network exception occurs.
95     */
96    void mutate(List<? extends Mutation> mutations) throws IOException;
97  
98    /**
99     * Performs a {@link #flush()} and releases any resources held.
100    *
101    * @throws IOException if a remote or network exception occurs.
102    */
103   @Override
104   void close() throws IOException;
105 
106   /**
107    * Executes all the buffered, asynchronous {@link Mutation} operations and waits until they
108    * are done.
109    *
110    * @throws IOException if a remote or network exception occurs.
111    */
112   void flush() throws IOException;
113 
114   /**
115    * Returns the maximum size in bytes of the write buffer for this HTable.
116    * <p>
117    * The default value comes from the configuration parameter {@code hbase.client.write.buffer}.
118    * @return The size of the write buffer in bytes.
119    */
120   long getWriteBufferSize();
121 
122   /**
123    * Listens for asynchronous exceptions on a {@link BufferedMutator}.
124    */
125   @InterfaceAudience.Public
126   @InterfaceStability.Evolving
127   interface ExceptionListener {
128     public void onException(RetriesExhaustedWithDetailsException exception,
129         BufferedMutator mutator) throws RetriesExhaustedWithDetailsException;
130   }
131 }