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 static org.junit.Assert.assertEquals;
22  import static org.junit.Assert.assertFalse;
23  import static org.junit.Assert.assertTrue;
24  import static org.junit.Assert.fail;
25  
26  import java.io.IOException;
27  import java.util.ArrayList;
28  import java.util.HashSet;
29  import java.util.List;
30  import java.util.concurrent.CountDownLatch;
31  import java.util.concurrent.ThreadPoolExecutor;
32  
33  import org.apache.commons.logging.Log;
34  import org.apache.commons.logging.LogFactory;
35  import org.apache.hadoop.hbase.Cell;
36  import org.apache.hadoop.hbase.CellUtil;
37  import org.apache.hadoop.hbase.HBaseTestingUtility;
38  import org.apache.hadoop.hbase.HRegionLocation;
39  import org.apache.hadoop.hbase.testclassification.MediumTests;
40  import org.apache.hadoop.hbase.ServerName;
41  import org.apache.hadoop.hbase.TableName;
42  import org.apache.hadoop.hbase.Waiter;
43  import org.apache.hadoop.hbase.exceptions.OperationConflictException;
44  import org.apache.hadoop.hbase.util.Bytes;
45  import org.apache.hadoop.hbase.util.JVMClusterUtil;
46  import org.apache.hadoop.hbase.util.Threads;
47  import org.junit.AfterClass;
48  import org.junit.Assert;
49  import org.junit.Before;
50  import org.junit.BeforeClass;
51  import org.junit.Test;
52  import org.junit.experimental.categories.Category;
53  
54  @Category(MediumTests.class)
55  public class TestMultiParallel {
56    private static final Log LOG = LogFactory.getLog(TestMultiParallel.class);
57  
58    private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
59    private static final byte[] VALUE = Bytes.toBytes("value");
60    private static final byte[] QUALIFIER = Bytes.toBytes("qual");
61    private static final String FAMILY = "family";
62    private static final TableName TEST_TABLE = TableName.valueOf("multi_test_table");
63    private static final byte[] BYTES_FAMILY = Bytes.toBytes(FAMILY);
64    private static final byte[] ONE_ROW = Bytes.toBytes("xxx");
65    private static final byte [][] KEYS = makeKeys();
66  
67    private static final int slaves = 5; // also used for testing HTable pool size
68    private static Connection CONNECTION;
69  
70    @BeforeClass public static void beforeClass() throws Exception {
71      // Uncomment the following lines if more verbosity is needed for
72      // debugging (see HBASE-12285 for details).
73      //((Log4JLogger)RpcServer.LOG).getLogger().setLevel(Level.ALL);
74      //((Log4JLogger)RpcClient.LOG).getLogger().setLevel(Level.ALL);
75      //((Log4JLogger)ScannerCallable.LOG).getLogger().setLevel(Level.ALL);
76      UTIL.startMiniCluster(slaves);
77      HTable t = UTIL.createMultiRegionTable(TEST_TABLE, Bytes.toBytes(FAMILY));
78      UTIL.waitTableEnabled(TEST_TABLE);
79      t.close();
80      CONNECTION = ConnectionFactory.createConnection(UTIL.getConfiguration());
81    }
82  
83    @AfterClass public static void afterClass() throws Exception {
84      CONNECTION.close();
85      UTIL.shutdownMiniCluster();
86    }
87  
88    @Before public void before() throws Exception {
89      LOG.info("before");
90      if (UTIL.ensureSomeRegionServersAvailable(slaves)) {
91        // Distribute regions
92        UTIL.getMiniHBaseCluster().getMaster().balance();
93  
94        // Wait until completing balance
95        UTIL.waitFor(15 * 1000, UTIL.predicateNoRegionsInTransition());
96      }
97      LOG.info("before done");
98    }
99  
100   private static byte[][] makeKeys() {
101     byte [][] starterKeys = HBaseTestingUtility.KEYS;
102     // Create a "non-uniform" test set with the following characteristics:
103     // a) Unequal number of keys per region
104 
105     // Don't use integer as a multiple, so that we have a number of keys that is
106     // not a multiple of the number of regions
107     int numKeys = (int) ((float) starterKeys.length * 10.33F);
108 
109     List<byte[]> keys = new ArrayList<byte[]>();
110     for (int i = 0; i < numKeys; i++) {
111       int kIdx = i % starterKeys.length;
112       byte[] k = starterKeys[kIdx];
113       byte[] cp = new byte[k.length + 1];
114       System.arraycopy(k, 0, cp, 0, k.length);
115       cp[k.length] = new Integer(i % 256).byteValue();
116       keys.add(cp);
117     }
118 
119     // b) Same duplicate keys (showing multiple Gets/Puts to the same row, which
120     // should work)
121     // c) keys are not in sorted order (within a region), to ensure that the
122     // sorting code and index mapping doesn't break the functionality
123     for (int i = 0; i < 100; i++) {
124       int kIdx = i % starterKeys.length;
125       byte[] k = starterKeys[kIdx];
126       byte[] cp = new byte[k.length + 1];
127       System.arraycopy(k, 0, cp, 0, k.length);
128       cp[k.length] = new Integer(i % 256).byteValue();
129       keys.add(cp);
130     }
131     return keys.toArray(new byte [][] {new byte [] {}});
132   }
133 
134 
135   /**
136    * This is for testing the active number of threads that were used while
137    * doing a batch operation. It inserts one row per region via the batch
138    * operation, and then checks the number of active threads.
139    * For HBASE-3553
140    * @throws IOException
141    * @throws InterruptedException
142    * @throws NoSuchFieldException
143    * @throws SecurityException
144    */
145   @Test(timeout=300000)
146   public void testActiveThreadsCount() throws Exception {
147     try (Connection connection = ConnectionFactory.createConnection(UTIL.getConfiguration())) {
148       ThreadPoolExecutor executor = HTable.getDefaultExecutor(UTIL.getConfiguration());
149       try {
150         try (Table t = connection.getTable(TEST_TABLE, executor)) {
151           List<Put> puts = constructPutRequests(); // creates a Put for every region
152           t.batch(puts);
153           HashSet<ServerName> regionservers = new HashSet<ServerName>();
154           try (RegionLocator locator = connection.getRegionLocator(TEST_TABLE)) {
155             for (Row r : puts) {
156               HRegionLocation location = locator.getRegionLocation(r.getRow());
157               regionservers.add(location.getServerName());
158             }
159           }
160           assertEquals(regionservers.size(), executor.getLargestPoolSize());
161         }
162       } finally {
163         executor.shutdownNow();
164       }
165     }
166   }
167 
168   @Test(timeout=300000)
169   public void testBatchWithGet() throws Exception {
170     LOG.info("test=testBatchWithGet");
171     Table table = new HTable(UTIL.getConfiguration(), TEST_TABLE);
172 
173     // load test data
174     List<Put> puts = constructPutRequests();
175     table.batch(puts);
176 
177     // create a list of gets and run it
178     List<Row> gets = new ArrayList<Row>();
179     for (byte[] k : KEYS) {
180       Get get = new Get(k);
181       get.addColumn(BYTES_FAMILY, QUALIFIER);
182       gets.add(get);
183     }
184     Result[] multiRes = new Result[gets.size()];
185     table.batch(gets, multiRes);
186 
187     // Same gets using individual call API
188     List<Result> singleRes = new ArrayList<Result>();
189     for (Row get : gets) {
190       singleRes.add(table.get((Get) get));
191     }
192     // Compare results
193     Assert.assertEquals(singleRes.size(), multiRes.length);
194     for (int i = 0; i < singleRes.size(); i++) {
195       Assert.assertTrue(singleRes.get(i).containsColumn(BYTES_FAMILY, QUALIFIER));
196       Cell[] singleKvs = singleRes.get(i).rawCells();
197       Cell[] multiKvs = multiRes[i].rawCells();
198       for (int j = 0; j < singleKvs.length; j++) {
199         Assert.assertEquals(singleKvs[j], multiKvs[j]);
200         Assert.assertEquals(0, Bytes.compareTo(CellUtil.cloneValue(singleKvs[j]),
201             CellUtil.cloneValue(multiKvs[j])));
202       }
203     }
204     table.close();
205   }
206 
207   @Test
208   public void testBadFam() throws Exception {
209     LOG.info("test=testBadFam");
210     Table table = new HTable(UTIL.getConfiguration(), TEST_TABLE);
211 
212     List<Row> actions = new ArrayList<Row>();
213     Put p = new Put(Bytes.toBytes("row1"));
214     p.add(Bytes.toBytes("bad_family"), Bytes.toBytes("qual"), Bytes.toBytes("value"));
215     actions.add(p);
216     p = new Put(Bytes.toBytes("row2"));
217     p.add(BYTES_FAMILY, Bytes.toBytes("qual"), Bytes.toBytes("value"));
218     actions.add(p);
219 
220     // row1 and row2 should be in the same region.
221 
222     Object [] r = new Object[actions.size()];
223     try {
224       table.batch(actions, r);
225       fail();
226     } catch (RetriesExhaustedWithDetailsException ex) {
227       LOG.debug(ex);
228       // good!
229       assertFalse(ex.mayHaveClusterIssues());
230     }
231     assertEquals(2, r.length);
232     assertTrue(r[0] instanceof Throwable);
233     assertTrue(r[1] instanceof Result);
234     table.close();
235   }
236 
237   @Test (timeout=300000)
238   public void testFlushCommitsNoAbort() throws Exception {
239     LOG.info("test=testFlushCommitsNoAbort");
240     doTestFlushCommits(false);
241   }
242 
243   /**
244    * Only run one Multi test with a forced RegionServer abort. Otherwise, the
245    * unit tests will take an unnecessarily long time to run.
246    *
247    * @throws Exception
248    */
249   @Test (timeout=360000)
250   public void testFlushCommitsWithAbort() throws Exception {
251     LOG.info("test=testFlushCommitsWithAbort");
252     doTestFlushCommits(true);
253   }
254 
255   /**
256    * Set table auto flush to false and test flushing commits
257    * @param doAbort true if abort one regionserver in the testing
258    * @throws Exception
259    */
260   private void doTestFlushCommits(boolean doAbort) throws Exception {
261     // Load the data
262     LOG.info("get new table");
263     Table table = UTIL.getConnection().getTable(TEST_TABLE);
264     table.setWriteBufferSize(10 * 1024 * 1024);
265 
266     LOG.info("constructPutRequests");
267     List<Put> puts = constructPutRequests();
268     table.put(puts);
269     LOG.info("puts");
270     final int liveRScount = UTIL.getMiniHBaseCluster().getLiveRegionServerThreads()
271         .size();
272     assert liveRScount > 0;
273     JVMClusterUtil.RegionServerThread liveRS = UTIL.getMiniHBaseCluster()
274         .getLiveRegionServerThreads().get(0);
275     if (doAbort) {
276       liveRS.getRegionServer().abort("Aborting for tests",
277           new Exception("doTestFlushCommits"));
278       // If we wait for no regions being online after we abort the server, we
279       // could ensure the master has re-assigned the regions on killed server
280       // after writing successfully. It means the server we aborted is dead
281       // and detected by matser
282       while (liveRS.getRegionServer().getNumberOfOnlineRegions() != 0) {
283         Thread.sleep(100);
284       }
285       // try putting more keys after the abort. same key/qual... just validating
286       // no exceptions thrown
287       puts = constructPutRequests();
288       table.put(puts);
289     }
290 
291     LOG.info("validating loaded data");
292     validateLoadedData(table);
293 
294     // Validate server and region count
295     List<JVMClusterUtil.RegionServerThread> liveRSs = UTIL.getMiniHBaseCluster().getLiveRegionServerThreads();
296     int count = 0;
297     for (JVMClusterUtil.RegionServerThread t: liveRSs) {
298       count++;
299       LOG.info("Count=" + count + ", Alive=" + t.getRegionServer());
300     }
301     LOG.info("Count=" + count);
302     Assert.assertEquals("Server count=" + count + ", abort=" + doAbort,
303         (doAbort ? (liveRScount - 1) : liveRScount), count);
304     if (doAbort) {
305       UTIL.getMiniHBaseCluster().waitOnRegionServer(0);
306       UTIL.waitFor(15 * 1000, new Waiter.Predicate<Exception>() {
307         @Override
308         public boolean evaluate() throws Exception {
309           return UTIL.getMiniHBaseCluster().getMaster()
310               .getClusterStatus().getServersSize() == (liveRScount - 1);
311         }
312       });
313       UTIL.waitFor(15 * 1000, UTIL.predicateNoRegionsInTransition());
314     }
315 
316     table.close();
317     LOG.info("done");
318   }
319 
320   @Test (timeout=300000)
321   public void testBatchWithPut() throws Exception {
322     LOG.info("test=testBatchWithPut");
323     Table table = CONNECTION.getTable(TEST_TABLE);
324     // put multiple rows using a batch
325     List<Put> puts = constructPutRequests();
326 
327     Object[] results = table.batch(puts);
328     validateSizeAndEmpty(results, KEYS.length);
329 
330     if (true) {
331       int liveRScount = UTIL.getMiniHBaseCluster().getLiveRegionServerThreads().size();
332       assert liveRScount > 0;
333       JVMClusterUtil.RegionServerThread liveRS =
334         UTIL.getMiniHBaseCluster().getLiveRegionServerThreads().get(0);
335       liveRS.getRegionServer().abort("Aborting for tests", new Exception("testBatchWithPut"));
336       puts = constructPutRequests();
337       try {
338         results = table.batch(puts);
339       } catch (RetriesExhaustedWithDetailsException ree) {
340         LOG.info(ree.getExhaustiveDescription());
341         table.close();
342         throw ree;
343       }
344       validateSizeAndEmpty(results, KEYS.length);
345     }
346 
347     validateLoadedData(table);
348     table.close();
349   }
350 
351   @Test(timeout=300000)
352   public void testBatchWithDelete() throws Exception {
353     LOG.info("test=testBatchWithDelete");
354     Table table = new HTable(UTIL.getConfiguration(), TEST_TABLE);
355 
356     // Load some data
357     List<Put> puts = constructPutRequests();
358     Object[] results = table.batch(puts);
359     validateSizeAndEmpty(results, KEYS.length);
360 
361     // Deletes
362     List<Row> deletes = new ArrayList<Row>();
363     for (int i = 0; i < KEYS.length; i++) {
364       Delete delete = new Delete(KEYS[i]);
365       delete.addFamily(BYTES_FAMILY);
366       deletes.add(delete);
367     }
368     results = table.batch(deletes);
369     validateSizeAndEmpty(results, KEYS.length);
370 
371     // Get to make sure ...
372     for (byte[] k : KEYS) {
373       Get get = new Get(k);
374       get.addColumn(BYTES_FAMILY, QUALIFIER);
375       Assert.assertFalse(table.exists(get));
376     }
377     table.close();
378   }
379 
380   @Test(timeout=300000)
381   public void testHTableDeleteWithList() throws Exception {
382     LOG.info("test=testHTableDeleteWithList");
383     Table table = new HTable(UTIL.getConfiguration(), TEST_TABLE);
384 
385     // Load some data
386     List<Put> puts = constructPutRequests();
387     Object[] results = table.batch(puts);
388     validateSizeAndEmpty(results, KEYS.length);
389 
390     // Deletes
391     ArrayList<Delete> deletes = new ArrayList<Delete>();
392     for (int i = 0; i < KEYS.length; i++) {
393       Delete delete = new Delete(KEYS[i]);
394       delete.deleteFamily(BYTES_FAMILY);
395       deletes.add(delete);
396     }
397     table.delete(deletes);
398     Assert.assertTrue(deletes.isEmpty());
399 
400     // Get to make sure ...
401     for (byte[] k : KEYS) {
402       Get get = new Get(k);
403       get.addColumn(BYTES_FAMILY, QUALIFIER);
404       Assert.assertFalse(table.exists(get));
405     }
406     table.close();
407   }
408 
409   @Test(timeout=300000)
410   public void testBatchWithManyColsInOneRowGetAndPut() throws Exception {
411     LOG.info("test=testBatchWithManyColsInOneRowGetAndPut");
412     Table table = new HTable(UTIL.getConfiguration(), TEST_TABLE);
413 
414     List<Row> puts = new ArrayList<Row>();
415     for (int i = 0; i < 100; i++) {
416       Put put = new Put(ONE_ROW);
417       byte[] qual = Bytes.toBytes("column" + i);
418       put.add(BYTES_FAMILY, qual, VALUE);
419       puts.add(put);
420     }
421     Object[] results = table.batch(puts);
422 
423     // validate
424     validateSizeAndEmpty(results, 100);
425 
426     // get the data back and validate that it is correct
427     List<Row> gets = new ArrayList<Row>();
428     for (int i = 0; i < 100; i++) {
429       Get get = new Get(ONE_ROW);
430       byte[] qual = Bytes.toBytes("column" + i);
431       get.addColumn(BYTES_FAMILY, qual);
432       gets.add(get);
433     }
434 
435     Object[] multiRes = table.batch(gets);
436 
437     int idx = 0;
438     for (Object r : multiRes) {
439       byte[] qual = Bytes.toBytes("column" + idx);
440       validateResult(r, qual, VALUE);
441       idx++;
442     }
443     table.close();
444   }
445 
446   @Test(timeout=300000)
447   public void testBatchWithIncrementAndAppend() throws Exception {
448     LOG.info("test=testBatchWithIncrementAndAppend");
449     final byte[] QUAL1 = Bytes.toBytes("qual1");
450     final byte[] QUAL2 = Bytes.toBytes("qual2");
451     final byte[] QUAL3 = Bytes.toBytes("qual3");
452     final byte[] QUAL4 = Bytes.toBytes("qual4");
453     Table table = new HTable(UTIL.getConfiguration(), TEST_TABLE);
454     Delete d = new Delete(ONE_ROW);
455     table.delete(d);
456     Put put = new Put(ONE_ROW);
457     put.add(BYTES_FAMILY, QUAL1, Bytes.toBytes("abc"));
458     put.add(BYTES_FAMILY, QUAL2, Bytes.toBytes(1L));
459     table.put(put);
460 
461     Increment inc = new Increment(ONE_ROW);
462     inc.addColumn(BYTES_FAMILY, QUAL2, 1);
463     inc.addColumn(BYTES_FAMILY, QUAL3, 1);
464 
465     Append a = new Append(ONE_ROW);
466     a.add(BYTES_FAMILY, QUAL1, Bytes.toBytes("def"));
467     a.add(BYTES_FAMILY, QUAL4, Bytes.toBytes("xyz"));
468     List<Row> actions = new ArrayList<Row>();
469     actions.add(inc);
470     actions.add(a);
471 
472     Object[] multiRes = table.batch(actions);
473     validateResult(multiRes[1], QUAL1, Bytes.toBytes("abcdef"));
474     validateResult(multiRes[1], QUAL4, Bytes.toBytes("xyz"));
475     validateResult(multiRes[0], QUAL2, Bytes.toBytes(2L));
476     validateResult(multiRes[0], QUAL3, Bytes.toBytes(1L));
477     table.close();
478   }
479 
480   @Test(timeout=300000)
481   public void testNonceCollision() throws Exception {
482     LOG.info("test=testNonceCollision");
483     final Connection connection = ConnectionFactory.createConnection(UTIL.getConfiguration());
484     Table table = connection.getTable(TEST_TABLE);
485     Put put = new Put(ONE_ROW);
486     put.add(BYTES_FAMILY, QUALIFIER, Bytes.toBytes(0L));
487 
488     // Replace nonce manager with the one that returns each nonce twice.
489     NonceGenerator cnm = new PerClientRandomNonceGenerator() {
490       long lastNonce = -1;
491       @Override
492       public synchronized long newNonce() {
493         long nonce = 0;
494         if (lastNonce == -1) {
495           lastNonce = nonce = super.newNonce();
496         } else {
497           nonce = lastNonce;
498           lastNonce = -1L;
499         }
500         return nonce;
501       }
502     };
503 
504     NonceGenerator oldCnm =
505       ConnectionUtils.injectNonceGeneratorForTesting((ClusterConnection)connection, cnm);
506 
507     // First test sequential requests.
508     try {
509       Increment inc = new Increment(ONE_ROW);
510       inc.addColumn(BYTES_FAMILY, QUALIFIER, 1L);
511       table.increment(inc);
512       inc = new Increment(ONE_ROW);
513       inc.addColumn(BYTES_FAMILY, QUALIFIER, 1L);
514       try {
515         table.increment(inc);
516         fail("Should have thrown an exception");
517       } catch (OperationConflictException ex) {
518       }
519       Get get = new Get(ONE_ROW);
520       get.addColumn(BYTES_FAMILY, QUALIFIER);
521       Result result = table.get(get);
522       validateResult(result, QUALIFIER, Bytes.toBytes(1L));
523 
524       // Now run a bunch of requests in parallel, exactly half should succeed.
525       int numRequests = 40;
526       final CountDownLatch startedLatch = new CountDownLatch(numRequests);
527       final CountDownLatch startLatch = new CountDownLatch(1);
528       final CountDownLatch doneLatch = new CountDownLatch(numRequests);
529       for (int i = 0; i < numRequests; ++i) {
530         Runnable r = new Runnable() {
531           @Override
532           public void run() {
533             Table table = null;
534             try {
535               table = connection.getTable(TEST_TABLE);
536             } catch (IOException e) {
537               fail("Not expected");
538             }
539             Increment inc = new Increment(ONE_ROW);
540             inc.addColumn(BYTES_FAMILY, QUALIFIER, 1L);
541             startedLatch.countDown();
542             try {
543               startLatch.await();
544             } catch (InterruptedException e) {
545               fail("Not expected");
546             }
547             try {
548               table.increment(inc);
549             } catch (OperationConflictException ex) { // Some threads are expected to fail.
550             } catch (IOException ioEx) {
551               fail("Not expected");
552             }
553             doneLatch.countDown();
554           }
555         };
556         Threads.setDaemonThreadRunning(new Thread(r));
557       }
558       startedLatch.await(); // Wait until all threads are ready...
559       startLatch.countDown(); // ...and unleash the herd!
560       doneLatch.await();
561       // Now verify
562       get = new Get(ONE_ROW);
563       get.addColumn(BYTES_FAMILY, QUALIFIER);
564       result = table.get(get);
565       validateResult(result, QUALIFIER, Bytes.toBytes((numRequests / 2) + 1L));
566       table.close();
567     } finally {
568       ConnectionManager.injectNonceGeneratorForTesting((ClusterConnection)connection, oldCnm);
569     }
570   }
571 
572   @Test(timeout=300000)
573   public void testBatchWithMixedActions() throws Exception {
574     LOG.info("test=testBatchWithMixedActions");
575     Table table = new HTable(UTIL.getConfiguration(), TEST_TABLE);
576 
577     // Load some data to start
578     Object[] results = table.batch(constructPutRequests());
579     validateSizeAndEmpty(results, KEYS.length);
580 
581     // Batch: get, get, put(new col), delete, get, get of put, get of deleted,
582     // put
583     List<Row> actions = new ArrayList<Row>();
584 
585     byte[] qual2 = Bytes.toBytes("qual2");
586     byte[] val2 = Bytes.toBytes("putvalue2");
587 
588     // 0 get
589     Get get = new Get(KEYS[10]);
590     get.addColumn(BYTES_FAMILY, QUALIFIER);
591     actions.add(get);
592 
593     // 1 get
594     get = new Get(KEYS[11]);
595     get.addColumn(BYTES_FAMILY, QUALIFIER);
596     actions.add(get);
597 
598     // 2 put of new column
599     Put put = new Put(KEYS[10]);
600     put.add(BYTES_FAMILY, qual2, val2);
601     actions.add(put);
602 
603     // 3 delete
604     Delete delete = new Delete(KEYS[20]);
605     delete.deleteFamily(BYTES_FAMILY);
606     actions.add(delete);
607 
608     // 4 get
609     get = new Get(KEYS[30]);
610     get.addColumn(BYTES_FAMILY, QUALIFIER);
611     actions.add(get);
612 
613     // There used to be a 'get' of a previous put here, but removed
614     // since this API really cannot guarantee order in terms of mixed
615     // get/puts.
616 
617     // 5 put of new column
618     put = new Put(KEYS[40]);
619     put.add(BYTES_FAMILY, qual2, val2);
620     actions.add(put);
621 
622     results = table.batch(actions);
623 
624     // Validation
625 
626     validateResult(results[0]);
627     validateResult(results[1]);
628     validateEmpty(results[2]);
629     validateEmpty(results[3]);
630     validateResult(results[4]);
631     validateEmpty(results[5]);
632 
633     // validate last put, externally from the batch
634     get = new Get(KEYS[40]);
635     get.addColumn(BYTES_FAMILY, qual2);
636     Result r = table.get(get);
637     validateResult(r, qual2, val2);
638 
639     table.close();
640   }
641 
642   // // Helper methods ////
643 
644   private void validateResult(Object r) {
645     validateResult(r, QUALIFIER, VALUE);
646   }
647 
648   private void validateResult(Object r1, byte[] qual, byte[] val) {
649     Result r = (Result)r1;
650     Assert.assertTrue(r.containsColumn(BYTES_FAMILY, qual));
651     byte[] value = r.getValue(BYTES_FAMILY, qual);
652     if (0 != Bytes.compareTo(val, value)) {
653       fail("Expected [" + Bytes.toStringBinary(val)
654           + "] but got [" + Bytes.toStringBinary(value) + "]");
655     }
656   }
657 
658   private List<Put> constructPutRequests() {
659     List<Put> puts = new ArrayList<>();
660     for (byte[] k : KEYS) {
661       Put put = new Put(k);
662       put.add(BYTES_FAMILY, QUALIFIER, VALUE);
663       puts.add(put);
664     }
665     return puts;
666   }
667 
668   private void validateLoadedData(Table table) throws IOException {
669     // get the data back and validate that it is correct
670     LOG.info("Validating data on " + table);
671     for (byte[] k : KEYS) {
672       Get get = new Get(k);
673       get.addColumn(BYTES_FAMILY, QUALIFIER);
674       Result r = table.get(get);
675       Assert.assertTrue(r.containsColumn(BYTES_FAMILY, QUALIFIER));
676       Assert.assertEquals(0, Bytes.compareTo(VALUE, r
677           .getValue(BYTES_FAMILY, QUALIFIER)));
678     }
679   }
680 
681   private void validateEmpty(Object r1) {
682     Result result = (Result)r1;
683     Assert.assertTrue(result != null);
684     Assert.assertTrue(result.getRow() == null);
685     Assert.assertEquals(0, result.rawCells().length);
686   }
687 
688   private void validateSizeAndEmpty(Object[] results, int expectedSize) {
689     // Validate got back the same number of Result objects, all empty
690     Assert.assertEquals(expectedSize, results.length);
691     for (Object result : results) {
692       validateEmpty(result);
693     }
694   }
695 }