001/*
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.client;
019
020import static org.hamcrest.CoreMatchers.containsString;
021import static org.hamcrest.CoreMatchers.instanceOf;
022import static org.hamcrest.MatcherAssert.assertThat;
023import static org.junit.Assert.assertArrayEquals;
024import static org.junit.Assert.assertEquals;
025import static org.junit.Assert.assertFalse;
026import static org.junit.Assert.assertNull;
027import static org.junit.Assert.assertTrue;
028import static org.junit.Assert.fail;
029
030import java.io.IOException;
031import java.io.UncheckedIOException;
032import java.util.ArrayList;
033import java.util.Arrays;
034import java.util.List;
035import java.util.Optional;
036import java.util.concurrent.CompletableFuture;
037import java.util.concurrent.ExecutionException;
038import java.util.concurrent.ForkJoinPool;
039import java.util.concurrent.Future;
040import java.util.concurrent.TimeUnit;
041import java.util.concurrent.TimeoutException;
042import java.util.function.Function;
043import java.util.stream.Collectors;
044import java.util.stream.IntStream;
045import org.apache.hadoop.hbase.Cell;
046import org.apache.hadoop.hbase.HBaseClassTestRule;
047import org.apache.hadoop.hbase.HBaseTestingUtility;
048import org.apache.hadoop.hbase.TableName;
049import org.apache.hadoop.hbase.coprocessor.ObserverContext;
050import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
051import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
052import org.apache.hadoop.hbase.coprocessor.RegionObserver;
053import org.apache.hadoop.hbase.regionserver.NoSuchColumnFamilyException;
054import org.apache.hadoop.hbase.testclassification.ClientTests;
055import org.apache.hadoop.hbase.testclassification.LargeTests;
056import org.apache.hadoop.hbase.util.Bytes;
057import org.junit.After;
058import org.junit.AfterClass;
059import org.junit.Before;
060import org.junit.BeforeClass;
061import org.junit.ClassRule;
062import org.junit.Test;
063import org.junit.experimental.categories.Category;
064import org.junit.runner.RunWith;
065import org.junit.runners.Parameterized;
066import org.junit.runners.Parameterized.Parameter;
067import org.junit.runners.Parameterized.Parameters;
068
069@RunWith(Parameterized.class)
070@Category({ LargeTests.class, ClientTests.class })
071public class TestAsyncTableBatch {
072
073  @ClassRule
074  public static final HBaseClassTestRule CLASS_RULE =
075    HBaseClassTestRule.forClass(TestAsyncTableBatch.class);
076
077  private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
078
079  private static TableName TABLE_NAME = TableName.valueOf("async");
080
081  private static byte[] FAMILY = Bytes.toBytes("cf");
082
083  private static byte[] CQ = Bytes.toBytes("cq");
084  private static byte[] CQ1 = Bytes.toBytes("cq1");
085
086  private static int COUNT = 1000;
087
088  private static AsyncConnection CONN;
089
090  private static byte[][] SPLIT_KEYS;
091
092  private static int MAX_KEY_VALUE_SIZE = 64 * 1024;
093
094  @Parameter(0)
095  public String tableType;
096
097  @Parameter(1)
098  public Function<TableName, AsyncTable<?>> tableGetter;
099
100  private static AsyncTable<?> getRawTable(TableName tableName) {
101    return CONN.getTable(tableName);
102  }
103
104  private static AsyncTable<?> getTable(TableName tableName) {
105    return CONN.getTable(tableName, ForkJoinPool.commonPool());
106  }
107
108  @Parameters(name = "{index}: type={0}")
109  public static List<Object[]> params() {
110    Function<TableName, AsyncTable<?>> rawTableGetter = TestAsyncTableBatch::getRawTable;
111    Function<TableName, AsyncTable<?>> tableGetter = TestAsyncTableBatch::getTable;
112    return Arrays.asList(new Object[] { "raw", rawTableGetter },
113      new Object[] { "normal", tableGetter });
114  }
115
116  @BeforeClass
117  public static void setUp() throws Exception {
118    TEST_UTIL.getConfiguration().setInt(ConnectionConfiguration.MAX_KEYVALUE_SIZE_KEY,
119      MAX_KEY_VALUE_SIZE);
120    TEST_UTIL.startMiniCluster(3);
121    SPLIT_KEYS = new byte[8][];
122    for (int i = 111; i < 999; i += 111) {
123      SPLIT_KEYS[i / 111 - 1] = Bytes.toBytes(String.format("%03d", i));
124    }
125    CONN = ConnectionFactory.createAsyncConnection(TEST_UTIL.getConfiguration()).get();
126  }
127
128  @AfterClass
129  public static void tearDown() throws Exception {
130    CONN.close();
131    TEST_UTIL.shutdownMiniCluster();
132  }
133
134  @Before
135  public void setUpBeforeTest() throws IOException, InterruptedException {
136    TEST_UTIL.createTable(TABLE_NAME, FAMILY, SPLIT_KEYS);
137    TEST_UTIL.waitTableAvailable(TABLE_NAME);
138  }
139
140  @After
141  public void tearDownAfterTest() throws IOException {
142    Admin admin = TEST_UTIL.getAdmin();
143    if (admin.isTableEnabled(TABLE_NAME)) {
144      admin.disableTable(TABLE_NAME);
145    }
146    admin.deleteTable(TABLE_NAME);
147  }
148
149  private byte[] getRow(int i) {
150    return Bytes.toBytes(String.format("%03d", i));
151  }
152
153  @Test
154  public void test()
155    throws InterruptedException, ExecutionException, IOException, TimeoutException {
156    AsyncTable<?> table = tableGetter.apply(TABLE_NAME);
157    table.putAll(IntStream.range(0, COUNT)
158      .mapToObj(i -> new Put(getRow(i)).addColumn(FAMILY, CQ, Bytes.toBytes(i)))
159      .collect(Collectors.toList())).get();
160    List<Result> results = table.getAll(IntStream.range(0, COUNT)
161      .mapToObj(i -> Arrays.asList(new Get(getRow(i)), new Get(Arrays.copyOf(getRow(i), 4))))
162      .flatMap(l -> l.stream()).collect(Collectors.toList())).get();
163    assertEquals(2 * COUNT, results.size());
164    for (int i = 0; i < COUNT; i++) {
165      assertEquals(i, Bytes.toInt(results.get(2 * i).getValue(FAMILY, CQ)));
166      assertTrue(results.get(2 * i + 1).isEmpty());
167    }
168    Admin admin = TEST_UTIL.getAdmin();
169    admin.flush(TABLE_NAME);
170    List<Future<?>> splitFutures =
171      TEST_UTIL.getHBaseCluster().getRegions(TABLE_NAME).stream().map(r -> {
172        byte[] startKey = r.getRegionInfo().getStartKey();
173        int number = startKey.length == 0 ? 55 : Integer.parseInt(Bytes.toString(startKey));
174        byte[] splitPoint = Bytes.toBytes(String.format("%03d", number + 55));
175        try {
176          return admin.splitRegionAsync(r.getRegionInfo().getRegionName(), splitPoint);
177        } catch (IOException e) {
178          throw new UncheckedIOException(e);
179        }
180      }).collect(Collectors.toList());
181    for (Future<?> future : splitFutures) {
182      future.get(30, TimeUnit.SECONDS);
183    }
184    table
185      .deleteAll(
186        IntStream.range(0, COUNT).mapToObj(i -> new Delete(getRow(i))).collect(Collectors.toList()))
187      .get();
188    results = table
189      .getAll(
190        IntStream.range(0, COUNT).mapToObj(i -> new Get(getRow(i))).collect(Collectors.toList()))
191      .get();
192    assertEquals(COUNT, results.size());
193    results.forEach(r -> assertTrue(r.isEmpty()));
194  }
195
196  @Test
197  public void testWithRegionServerFailover() throws Exception {
198    AsyncTable<?> table = tableGetter.apply(TABLE_NAME);
199    table.putAll(IntStream.range(0, COUNT)
200      .mapToObj(i -> new Put(getRow(i)).addColumn(FAMILY, CQ, Bytes.toBytes(i)))
201      .collect(Collectors.toList())).get();
202    TEST_UTIL.getMiniHBaseCluster().getRegionServer(0).abort("Aborting for tests");
203    Thread.sleep(100);
204    table.putAll(IntStream.range(COUNT, 2 * COUNT)
205      .mapToObj(i -> new Put(getRow(i)).addColumn(FAMILY, CQ, Bytes.toBytes(i)))
206      .collect(Collectors.toList())).get();
207    List<Result> results = table.getAll(
208      IntStream.range(0, 2 * COUNT).mapToObj(i -> new Get(getRow(i))).collect(Collectors.toList()))
209      .get();
210    assertEquals(2 * COUNT, results.size());
211    results.forEach(r -> assertFalse(r.isEmpty()));
212    table.deleteAll(IntStream.range(0, 2 * COUNT).mapToObj(i -> new Delete(getRow(i)))
213      .collect(Collectors.toList())).get();
214    results = table.getAll(
215      IntStream.range(0, 2 * COUNT).mapToObj(i -> new Get(getRow(i))).collect(Collectors.toList()))
216      .get();
217    assertEquals(2 * COUNT, results.size());
218    results.forEach(r -> assertTrue(r.isEmpty()));
219  }
220
221  @Test
222  public void testMixed() throws InterruptedException, ExecutionException, IOException {
223    AsyncTable<?> table = tableGetter.apply(TABLE_NAME);
224    table.putAll(IntStream.range(0, 7)
225      .mapToObj(i -> new Put(Bytes.toBytes(i)).addColumn(FAMILY, CQ, Bytes.toBytes((long) i)))
226      .collect(Collectors.toList())).get();
227    List<Row> actions = new ArrayList<>();
228    actions.add(new Get(Bytes.toBytes(0)));
229    actions.add(new Put(Bytes.toBytes(1)).addColumn(FAMILY, CQ, Bytes.toBytes(2L)));
230    actions.add(new Delete(Bytes.toBytes(2)));
231    actions.add(new Increment(Bytes.toBytes(3)).addColumn(FAMILY, CQ, 1));
232    actions.add(new Append(Bytes.toBytes(4)).addColumn(FAMILY, CQ, Bytes.toBytes(4)));
233    RowMutations rm = new RowMutations(Bytes.toBytes(5));
234    rm.add((Mutation) new Put(Bytes.toBytes(5)).addColumn(FAMILY, CQ, Bytes.toBytes(100L)));
235    rm.add((Mutation) new Put(Bytes.toBytes(5)).addColumn(FAMILY, CQ1, Bytes.toBytes(200L)));
236    actions.add(rm);
237    actions.add(new Get(Bytes.toBytes(6)));
238
239    List<Object> results = table.batchAll(actions).get();
240    assertEquals(7, results.size());
241    Result getResult = (Result) results.get(0);
242    assertEquals(0, Bytes.toLong(getResult.getValue(FAMILY, CQ)));
243    assertEquals(2, Bytes.toLong(table.get(new Get(Bytes.toBytes(1))).get().getValue(FAMILY, CQ)));
244    assertTrue(table.get(new Get(Bytes.toBytes(2))).get().isEmpty());
245    Result incrementResult = (Result) results.get(3);
246    assertEquals(4, Bytes.toLong(incrementResult.getValue(FAMILY, CQ)));
247    Result appendResult = (Result) results.get(4);
248    byte[] appendValue = appendResult.getValue(FAMILY, CQ);
249    assertEquals(12, appendValue.length);
250    assertEquals(4, Bytes.toLong(appendValue));
251    assertEquals(4, Bytes.toInt(appendValue, 8));
252    assertEquals(100,
253      Bytes.toLong(table.get(new Get(Bytes.toBytes(5))).get().getValue(FAMILY, CQ)));
254    assertEquals(200,
255      Bytes.toLong(table.get(new Get(Bytes.toBytes(5))).get().getValue(FAMILY, CQ1)));
256    getResult = (Result) results.get(6);
257    assertEquals(6, Bytes.toLong(getResult.getValue(FAMILY, CQ)));
258  }
259
260  public static final class ErrorInjectObserver implements RegionCoprocessor, RegionObserver {
261
262    @Override
263    public Optional<RegionObserver> getRegionObserver() {
264      return Optional.of(this);
265    }
266
267    @Override
268    public void preGetOp(ObserverContext<RegionCoprocessorEnvironment> e, Get get,
269      List<Cell> results) throws IOException {
270      if (e.getEnvironment().getRegionInfo().getEndKey().length == 0) {
271        throw new DoNotRetryRegionException("Inject Error");
272      }
273    }
274  }
275
276  @Test
277  public void testPartialSuccess() throws IOException, InterruptedException, ExecutionException {
278    Admin admin = TEST_UTIL.getAdmin();
279    TableDescriptor htd = TableDescriptorBuilder.newBuilder(admin.getDescriptor(TABLE_NAME))
280      .setCoprocessor(ErrorInjectObserver.class.getName()).build();
281    admin.modifyTable(htd);
282    AsyncTable<?> table = tableGetter.apply(TABLE_NAME);
283    table.putAll(Arrays.asList(SPLIT_KEYS).stream().map(k -> new Put(k).addColumn(FAMILY, CQ, k))
284      .collect(Collectors.toList())).get();
285    List<CompletableFuture<Result>> futures = table
286      .get(Arrays.asList(SPLIT_KEYS).stream().map(k -> new Get(k)).collect(Collectors.toList()));
287    for (int i = 0; i < SPLIT_KEYS.length - 1; i++) {
288      assertArrayEquals(SPLIT_KEYS[i], futures.get(i).get().getValue(FAMILY, CQ));
289    }
290    try {
291      futures.get(SPLIT_KEYS.length - 1).get();
292      fail();
293    } catch (ExecutionException e) {
294      assertThat(e.getCause(), instanceOf(RetriesExhaustedException.class));
295    }
296  }
297
298  @Test
299  public void testPartialSuccessOnSameRegion() throws InterruptedException, ExecutionException {
300    AsyncTable<?> table = tableGetter.apply(TABLE_NAME);
301    List<CompletableFuture<Object>> futures = table.batch(Arrays.asList(
302      new Put(Bytes.toBytes("put")).addColumn(Bytes.toBytes("not-exists"), CQ,
303        Bytes.toBytes("bad")),
304      new Increment(Bytes.toBytes("inc")).addColumn(FAMILY, CQ, 1),
305      new Put(Bytes.toBytes("put")).addColumn(FAMILY, CQ, Bytes.toBytes("good"))));
306    try {
307      futures.get(0).get();
308      fail();
309    } catch (ExecutionException e) {
310      assertThat(e.getCause(), instanceOf(RetriesExhaustedException.class));
311      assertThat(e.getCause().getCause(), instanceOf(NoSuchColumnFamilyException.class));
312    }
313    assertEquals(1, Bytes.toLong(((Result) futures.get(1).get()).getValue(FAMILY, CQ)));
314    assertTrue(((Result) futures.get(2).get()).isEmpty());
315    assertEquals("good",
316      Bytes.toString(table.get(new Get(Bytes.toBytes("put"))).get().getValue(FAMILY, CQ)));
317  }
318
319  @Test
320  public void testInvalidPut() {
321    AsyncTable<?> table = tableGetter.apply(TABLE_NAME);
322    try {
323      table.batch(Arrays.asList(new Delete(Bytes.toBytes(0)), new Put(Bytes.toBytes(0))));
324      fail("Should fail since the put does not contain any cells");
325    } catch (IllegalArgumentException e) {
326      assertThat(e.getMessage(), containsString("No columns to insert"));
327    }
328
329    try {
330      table.batch(
331        Arrays.asList(new Put(Bytes.toBytes(0)).addColumn(FAMILY, CQ, new byte[MAX_KEY_VALUE_SIZE]),
332          new Delete(Bytes.toBytes(0))));
333      fail("Should fail since the put exceeds the max key value size");
334    } catch (IllegalArgumentException e) {
335      assertThat(e.getMessage(), containsString("KeyValue size too large"));
336    }
337  }
338
339  @Test
340  public void testInvalidPutInRowMutations() throws IOException {
341    final byte[] row = Bytes.toBytes(0);
342
343    AsyncTable<?> table = tableGetter.apply(TABLE_NAME);
344    try {
345      table
346        .batch(Arrays.asList(new Delete(row), new RowMutations(row).add((Mutation) new Put(row))));
347      fail("Should fail since the put does not contain any cells");
348    } catch (IllegalArgumentException e) {
349      assertThat(e.getMessage(), containsString("No columns to insert"));
350    }
351
352    try {
353      table.batch(Arrays.asList(
354        new RowMutations(row)
355          .add((Mutation) new Put(row).addColumn(FAMILY, CQ, new byte[MAX_KEY_VALUE_SIZE])),
356        new Delete(row)));
357      fail("Should fail since the put exceeds the max key value size");
358    } catch (IllegalArgumentException e) {
359      assertThat(e.getMessage(), containsString("KeyValue size too large"));
360    }
361  }
362
363  @Test
364  public void testInvalidPutInRowMutationsInCheckAndMutate() throws IOException {
365    final byte[] row = Bytes.toBytes(0);
366
367    AsyncTable<?> table = tableGetter.apply(TABLE_NAME);
368    try {
369      table.batch(Arrays.asList(new Delete(row), CheckAndMutate.newBuilder(row)
370        .ifNotExists(FAMILY, CQ).build(new RowMutations(row).add((Mutation) new Put(row)))));
371      fail("Should fail since the put does not contain any cells");
372    } catch (IllegalArgumentException e) {
373      assertThat(e.getMessage(), containsString("No columns to insert"));
374    }
375
376    try {
377      table.batch(Arrays.asList(
378        CheckAndMutate.newBuilder(row).ifNotExists(FAMILY, CQ)
379          .build(new RowMutations(row)
380            .add((Mutation) new Put(row).addColumn(FAMILY, CQ, new byte[MAX_KEY_VALUE_SIZE]))),
381        new Delete(row)));
382      fail("Should fail since the put exceeds the max key value size");
383    } catch (IllegalArgumentException e) {
384      assertThat(e.getMessage(), containsString("KeyValue size too large"));
385    }
386  }
387
388  @Test
389  public void testWithCheckAndMutate() throws Exception {
390    AsyncTable<?> table = tableGetter.apply(TABLE_NAME);
391
392    byte[] row1 = Bytes.toBytes("row1");
393    byte[] row2 = Bytes.toBytes("row2");
394    byte[] row3 = Bytes.toBytes("row3");
395    byte[] row4 = Bytes.toBytes("row4");
396    byte[] row5 = Bytes.toBytes("row5");
397    byte[] row6 = Bytes.toBytes("row6");
398    byte[] row7 = Bytes.toBytes("row7");
399
400    table
401      .putAll(Arrays.asList(new Put(row1).addColumn(FAMILY, Bytes.toBytes("A"), Bytes.toBytes("a")),
402        new Put(row2).addColumn(FAMILY, Bytes.toBytes("B"), Bytes.toBytes("b")),
403        new Put(row3).addColumn(FAMILY, Bytes.toBytes("C"), Bytes.toBytes("c")),
404        new Put(row4).addColumn(FAMILY, Bytes.toBytes("D"), Bytes.toBytes("d")),
405        new Put(row5).addColumn(FAMILY, Bytes.toBytes("E"), Bytes.toBytes("e")),
406        new Put(row6).addColumn(FAMILY, Bytes.toBytes("F"), Bytes.toBytes(10L)),
407        new Put(row7).addColumn(FAMILY, Bytes.toBytes("G"), Bytes.toBytes("g"))))
408      .get();
409
410    CheckAndMutate checkAndMutate1 =
411      CheckAndMutate.newBuilder(row1).ifEquals(FAMILY, Bytes.toBytes("A"), Bytes.toBytes("a"))
412        .build(new RowMutations(row1)
413          .add((Mutation) new Put(row1).addColumn(FAMILY, Bytes.toBytes("B"), Bytes.toBytes("g")))
414          .add((Mutation) new Delete(row1).addColumns(FAMILY, Bytes.toBytes("A")))
415          .add(new Increment(row1).addColumn(FAMILY, Bytes.toBytes("C"), 3L))
416          .add(new Append(row1).addColumn(FAMILY, Bytes.toBytes("D"), Bytes.toBytes("d"))));
417    Get get = new Get(row2).addColumn(FAMILY, Bytes.toBytes("B"));
418    RowMutations mutations =
419      new RowMutations(row3).add((Mutation) new Delete(row3).addColumns(FAMILY, Bytes.toBytes("C")))
420        .add((Mutation) new Put(row3).addColumn(FAMILY, Bytes.toBytes("F"), Bytes.toBytes("f")))
421        .add(new Increment(row3).addColumn(FAMILY, Bytes.toBytes("A"), 5L))
422        .add(new Append(row3).addColumn(FAMILY, Bytes.toBytes("B"), Bytes.toBytes("b")));
423    CheckAndMutate checkAndMutate2 =
424      CheckAndMutate.newBuilder(row4).ifEquals(FAMILY, Bytes.toBytes("D"), Bytes.toBytes("a"))
425        .build(new Put(row4).addColumn(FAMILY, Bytes.toBytes("E"), Bytes.toBytes("h")));
426    Put put = new Put(row5).addColumn(FAMILY, Bytes.toBytes("E"), Bytes.toBytes("f"));
427    CheckAndMutate checkAndMutate3 =
428      CheckAndMutate.newBuilder(row6).ifEquals(FAMILY, Bytes.toBytes("F"), Bytes.toBytes(10L))
429        .build(new Increment(row6).addColumn(FAMILY, Bytes.toBytes("F"), 1));
430    CheckAndMutate checkAndMutate4 =
431      CheckAndMutate.newBuilder(row7).ifEquals(FAMILY, Bytes.toBytes("G"), Bytes.toBytes("g"))
432        .build(new Append(row7).addColumn(FAMILY, Bytes.toBytes("G"), Bytes.toBytes("g")));
433
434    List<Row> actions = Arrays.asList(checkAndMutate1, get, mutations, checkAndMutate2, put,
435      checkAndMutate3, checkAndMutate4);
436    List<Object> results = table.batchAll(actions).get();
437
438    CheckAndMutateResult checkAndMutateResult = (CheckAndMutateResult) results.get(0);
439    assertTrue(checkAndMutateResult.isSuccess());
440    assertEquals(3L,
441      Bytes.toLong(checkAndMutateResult.getResult().getValue(FAMILY, Bytes.toBytes("C"))));
442    assertEquals("d",
443      Bytes.toString(checkAndMutateResult.getResult().getValue(FAMILY, Bytes.toBytes("D"))));
444
445    assertEquals("b",
446      Bytes.toString(((Result) results.get(1)).getValue(FAMILY, Bytes.toBytes("B"))));
447
448    Result result = (Result) results.get(2);
449    assertTrue(result.getExists());
450    assertEquals(5L, Bytes.toLong(result.getValue(FAMILY, Bytes.toBytes("A"))));
451    assertEquals("b", Bytes.toString(result.getValue(FAMILY, Bytes.toBytes("B"))));
452
453    checkAndMutateResult = (CheckAndMutateResult) results.get(3);
454    assertFalse(checkAndMutateResult.isSuccess());
455    assertNull(checkAndMutateResult.getResult());
456
457    assertTrue(((Result) results.get(4)).isEmpty());
458
459    checkAndMutateResult = (CheckAndMutateResult) results.get(5);
460    assertTrue(checkAndMutateResult.isSuccess());
461    assertEquals(11,
462      Bytes.toLong(checkAndMutateResult.getResult().getValue(FAMILY, Bytes.toBytes("F"))));
463
464    checkAndMutateResult = (CheckAndMutateResult) results.get(6);
465    assertTrue(checkAndMutateResult.isSuccess());
466    assertEquals("gg",
467      Bytes.toString(checkAndMutateResult.getResult().getValue(FAMILY, Bytes.toBytes("G"))));
468
469    result = table.get(new Get(row1)).get();
470    assertEquals("g", Bytes.toString(result.getValue(FAMILY, Bytes.toBytes("B"))));
471    assertNull(result.getValue(FAMILY, Bytes.toBytes("A")));
472    assertEquals(3L, Bytes.toLong(result.getValue(FAMILY, Bytes.toBytes("C"))));
473    assertEquals("d", Bytes.toString(result.getValue(FAMILY, Bytes.toBytes("D"))));
474
475    result = table.get(new Get(row3)).get();
476    assertNull(result.getValue(FAMILY, Bytes.toBytes("C")));
477    assertEquals("f", Bytes.toString(result.getValue(FAMILY, Bytes.toBytes("F"))));
478    assertNull(Bytes.toString(result.getValue(FAMILY, Bytes.toBytes("C"))));
479    assertEquals(5L, Bytes.toLong(result.getValue(FAMILY, Bytes.toBytes("A"))));
480    assertEquals("b", Bytes.toString(result.getValue(FAMILY, Bytes.toBytes("B"))));
481
482    result = table.get(new Get(row4)).get();
483    assertEquals("d", Bytes.toString(result.getValue(FAMILY, Bytes.toBytes("D"))));
484
485    result = table.get(new Get(row5)).get();
486    assertEquals("f", Bytes.toString(result.getValue(FAMILY, Bytes.toBytes("E"))));
487
488    result = table.get(new Get(row6)).get();
489    assertEquals(11, Bytes.toLong(result.getValue(FAMILY, Bytes.toBytes("F"))));
490
491    result = table.get(new Get(row7)).get();
492    assertEquals("gg", Bytes.toString(result.getValue(FAMILY, Bytes.toBytes("G"))));
493  }
494}