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.junit.jupiter.api.Assertions.assertEquals;
021
022import java.util.ArrayList;
023import java.util.LinkedList;
024import java.util.List;
025import java.util.Map.Entry;
026import org.apache.hadoop.hbase.HBaseTestingUtil;
027import org.apache.hadoop.hbase.TableName;
028import org.apache.hadoop.hbase.client.Scan.ReadType;
029import org.apache.hadoop.hbase.testclassification.LargeTests;
030import org.apache.hadoop.hbase.util.Bytes;
031import org.junit.jupiter.api.AfterAll;
032import org.junit.jupiter.api.BeforeAll;
033import org.junit.jupiter.api.Tag;
034import org.junit.jupiter.api.Test;
035import org.slf4j.Logger;
036import org.slf4j.LoggerFactory;
037
038import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
039
040@Tag(LargeTests.TAG)
041public class TestSizeFailures {
042
043  private static final Logger LOG = LoggerFactory.getLogger(TestSizeFailures.class);
044  protected final static HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil();
045  private static byte[] FAMILY = Bytes.toBytes("testFamily");
046  protected static int SLAVES = 1;
047  private static TableName TABLENAME;
048  private static final int NUM_ROWS = 1000 * 1000, NUM_COLS = 9;
049
050  @BeforeAll
051  public static void setUpBeforeClass() throws Exception {
052    // Uncomment the following lines if more verbosity is needed for
053    // debugging (see HBASE-12285 for details).
054    // ((Log4JLogger)RpcServer.LOG).getLogger().setLevel(Level.ALL);
055    // ((Log4JLogger)RpcClient.LOG).getLogger().setLevel(Level.ALL);
056    // ((Log4JLogger)ScannerCallable.LOG).getLogger().setLevel(Level.ALL);
057    TEST_UTIL.startMiniCluster(SLAVES);
058
059    // Write a bunch of data
060    TABLENAME = TableName.valueOf("testSizeFailures");
061    List<byte[]> qualifiers = new ArrayList<>();
062    for (int i = 1; i <= 10; i++) {
063      qualifiers.add(Bytes.toBytes(Integer.toString(i)));
064    }
065
066    TableDescriptor tableDescriptor = TableDescriptorBuilder.newBuilder(TABLENAME)
067      .setColumnFamily(ColumnFamilyDescriptorBuilder.of(FAMILY)).build();
068    byte[][] splits = new byte[9][2];
069    for (int i = 1; i < 10; i++) {
070      int split = 48 + i;
071      splits[i - 1][0] = (byte) (split >>> 8);
072      splits[i - 1][0] = (byte) (split);
073    }
074    TEST_UTIL.getAdmin().createTable(tableDescriptor, splits);
075    Connection conn = TEST_UTIL.getConnection();
076
077    try (Table table = conn.getTable(TABLENAME)) {
078      List<Put> puts = new LinkedList<>();
079      for (int i = 0; i < NUM_ROWS; i++) {
080        Put p = new Put(Bytes.toBytes(Integer.toString(i)));
081        for (int j = 0; j < NUM_COLS; j++) {
082          byte[] value = new byte[50];
083          Bytes.random(value);
084          p.addColumn(FAMILY, Bytes.toBytes(Integer.toString(j)), value);
085        }
086        puts.add(p);
087
088        if (puts.size() == 1000) {
089          table.batch(puts, null);
090          puts.clear();
091        }
092      }
093
094      if (puts.size() > 0) {
095        table.batch(puts, null);
096      }
097    }
098  }
099
100  @AfterAll
101  public static void tearDownAfterClass() throws Exception {
102    TEST_UTIL.shutdownMiniCluster();
103  }
104
105  /**
106   * Basic client side validation of HBASE-13262
107   */
108  @Test
109  public void testScannerSeesAllRecords() throws Exception {
110    Connection conn = TEST_UTIL.getConnection();
111    try (Table table = conn.getTable(TABLENAME)) {
112      Scan s = new Scan();
113      s.addFamily(FAMILY);
114      s.setMaxResultSize(-1);
115      s.setBatch(-1);
116      s.setCaching(500);
117      Entry<Long, Long> entry = sumTable(table.getScanner(s));
118      long rowsObserved = entry.getKey();
119      long entriesObserved = entry.getValue();
120
121      // Verify that we see 1M rows and 9M cells
122      assertEquals(NUM_ROWS, rowsObserved);
123      assertEquals(NUM_ROWS * NUM_COLS, entriesObserved);
124    }
125  }
126
127  /**
128   * Basic client side validation of HBASE-13262
129   */
130  @Test
131  public void testSmallScannerSeesAllRecords() throws Exception {
132    Connection conn = TEST_UTIL.getConnection();
133    try (Table table = conn.getTable(TABLENAME)) {
134      Scan s = new Scan();
135      s.setReadType(ReadType.PREAD);
136      s.addFamily(FAMILY);
137      s.setMaxResultSize(-1);
138      s.setBatch(-1);
139      s.setCaching(500);
140      Entry<Long, Long> entry = sumTable(table.getScanner(s));
141      long rowsObserved = entry.getKey();
142      long entriesObserved = entry.getValue();
143
144      // Verify that we see 1M rows and 9M cells
145      assertEquals(NUM_ROWS, rowsObserved);
146      assertEquals(NUM_ROWS * NUM_COLS, entriesObserved);
147    }
148  }
149
150  /**
151   * Count the number of rows and the number of entries from a scanner The Scanner
152   * @return An entry where the first item is rows observed and the second is entries observed.
153   */
154  private Entry<Long, Long> sumTable(ResultScanner scanner) {
155    long rowsObserved = 0L;
156    long entriesObserved = 0L;
157
158    // Read all the records in the table
159    for (Result result : scanner) {
160      rowsObserved++;
161      while (result.advance()) {
162        entriesObserved++;
163      }
164    }
165    return Maps.immutableEntry(rowsObserved, entriesObserved);
166  }
167}