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.example;
20  
21  import org.apache.commons.logging.Log;
22  import org.apache.commons.logging.LogFactory;
23  import org.apache.hadoop.conf.Configured;
24  import org.apache.hadoop.hbase.TableName;
25  import org.apache.hadoop.hbase.client.BufferedMutator;
26  import org.apache.hadoop.hbase.client.BufferedMutatorParams;
27  import org.apache.hadoop.hbase.client.Connection;
28  import org.apache.hadoop.hbase.client.ConnectionFactory;
29  import org.apache.hadoop.hbase.client.Put;
30  import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException;
31  import org.apache.hadoop.hbase.util.Bytes;
32  import org.apache.hadoop.util.Tool;
33  import org.apache.hadoop.util.ToolRunner;
34  
35  import java.io.IOException;
36  import java.util.ArrayList;
37  import java.util.List;
38  import java.util.concurrent.Callable;
39  import java.util.concurrent.ExecutionException;
40  import java.util.concurrent.ExecutorService;
41  import java.util.concurrent.Executors;
42  import java.util.concurrent.Future;
43  import java.util.concurrent.TimeUnit;
44  import java.util.concurrent.TimeoutException;
45  
46  /**
47   * An example of using the {@link BufferedMutator} interface.
48   */
49  public class BufferedMutatorExample extends Configured implements Tool {
50  
51    private static final Log LOG = LogFactory.getLog(BufferedMutatorExample.class);
52  
53    private static final int POOL_SIZE = 10;
54    private static final int TASK_COUNT = 100;
55    private static final TableName TABLE = TableName.valueOf("foo");
56    private static final byte[] FAMILY = Bytes.toBytes("f");
57  
58    @Override
59    public int run(String[] args) throws InterruptedException, ExecutionException, TimeoutException {
60  
61      /** a callback invoked when an asynchronous write fails. */
62      final BufferedMutator.ExceptionListener listener = new BufferedMutator.ExceptionListener() {
63        @Override
64        public void onException(RetriesExhaustedWithDetailsException e, BufferedMutator mutator) {
65          for (int i = 0; i < e.getNumExceptions(); i++) {
66            LOG.info("Failed to sent put " + e.getRow(i) + ".");
67          }
68        }
69      };
70      BufferedMutatorParams params = new BufferedMutatorParams(TABLE)
71          .listener(listener);
72  
73      //
74      // step 1: create a single Connection and a BufferedMutator, shared by all worker threads.
75      //
76      try (final Connection conn = ConnectionFactory.createConnection(getConf());
77           final BufferedMutator mutator = conn.getBufferedMutator(params)) {
78  
79        /** worker pool that operates on BufferedTable instances */
80        final ExecutorService workerPool = Executors.newFixedThreadPool(POOL_SIZE);
81        List<Future<Void>> futures = new ArrayList<>(TASK_COUNT);
82  
83        for (int i = 0; i < TASK_COUNT; i++) {
84          futures.add(workerPool.submit(new Callable<Void>() {
85            @Override
86            public Void call() throws Exception {
87              //
88              // step 2: each worker sends edits to the shared BufferedMutator instance. They all use
89              // the same backing buffer, call-back "listener", and RPC executor pool.
90              //
91              Put p = new Put(Bytes.toBytes("someRow"));
92              p.addColumn(FAMILY, Bytes.toBytes("someQualifier"), Bytes.toBytes("some value"));
93              mutator.mutate(p);
94              // do work... maybe you want to call mutator.flush() after many edits to ensure any of
95              // this worker's edits are sent before exiting the Callable
96              return null;
97            }
98          }));
99        }
100 
101       //
102       // step 3: clean up the worker pool, shut down.
103       //
104       for (Future<Void> f : futures) {
105         f.get(5, TimeUnit.MINUTES);
106       }
107       workerPool.shutdown();
108     } catch (IOException e) {
109       // exception while creating/destroying Connection or BufferedMutator
110       LOG.info("exception while creating/destroying Connection or BufferedMutator", e);
111     } // BufferedMutator.close() ensures all work is flushed. Could be the custom listener is
112       // invoked from here.
113     return 0;
114   }
115 
116   public static void main(String[] args) throws Exception {
117     ToolRunner.run(new BufferedMutatorExample(), args);
118   }
119 }