Skip to content

Commit

Permalink
extend test to include filtered case and show lastBlock implications
Browse files Browse the repository at this point in the history
  • Loading branch information
bbeaudreault committed Feb 3, 2023
1 parent 786f2b4 commit 91e02cf
Showing 1 changed file with 47 additions and 18 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -19,21 +19,26 @@

import static org.junit.Assert.assertEquals;

import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellBuilderFactory;
import org.apache.hadoop.hbase.CellBuilderType;
import org.apache.hadoop.hbase.CompareOperator;
import org.apache.hadoop.hbase.CompatibilityFactory;
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseTestingUtil;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.Waiter;
import org.apache.hadoop.hbase.filter.BinaryComparator;
import org.apache.hadoop.hbase.filter.QualifierFilter;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
import org.apache.hadoop.hbase.ipc.RpcServerInterface;
import org.apache.hadoop.hbase.logging.Log4jUtils;
import org.apache.hadoop.hbase.metrics.BaseSource;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.test.MetricsAssertHelper;
import org.apache.hadoop.hbase.testclassification.ClientTests;
Expand Down Expand Up @@ -62,7 +67,7 @@ public class TestMultiRespectsLimits {
private static final MetricsAssertHelper METRICS_ASSERT =
CompatibilityFactory.getInstance(MetricsAssertHelper.class);
private final static byte[] FAMILY = Bytes.toBytes("D");
public static final int MAX_SIZE = 50;
public static final int MAX_SIZE = 90;
private static String LOG_LEVEL;

@Rule
Expand Down Expand Up @@ -148,6 +153,10 @@ public void testBlockMultiLimits() throws Exception {
Bytes.toBytes("3"), // Get This
Bytes.toBytes("4"), // Buffer
Bytes.toBytes("5"), // Buffer
Bytes.toBytes("6"), // Buffer
Bytes.toBytes("7"), // Get This
Bytes.toBytes("8"), // Buffer
Bytes.toBytes("9"), // Buffer
};

// Set the value size so that one result will be less than the MAX_SIZE
Expand All @@ -156,7 +165,12 @@ public void testBlockMultiLimits() throws Exception {
byte[] value = new byte[1];
Bytes.random(value);

for (byte[] col : cols) {
for (int i = 0; i < cols.length; i++) {
if (i == 6) {
// do a flush here so we end up with 2 blocks, 55 and 45 bytes
flush(regionServer, tableName);
}
byte[] col = cols[i];
Put p = new Put(row);
p.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY).setRow(row).setFamily(FAMILY)
.setQualifier(col).setTimestamp(p.getTimestamp()).setType(Cell.Type.Put).setValue(value)
Expand All @@ -165,28 +179,43 @@ public void testBlockMultiLimits() throws Exception {
}

// Make sure that a flush happens
try (final Admin admin = TEST_UTIL.getAdmin()) {
admin.flush(tableName);
TEST_UTIL.waitFor(60000, new Waiter.Predicate<Exception>() {
@Override
public boolean evaluate() throws Exception {
return regionServer.getRegions(tableName).get(0).getMaxFlushedSeqId() > 3;
}
});
}

List<Get> gets = new ArrayList<>(2);
Get g0 = new Get(row);
g0.addColumn(FAMILY, cols[0]);
flush(regionServer, tableName);

List<Get> gets = new ArrayList<>(4);
// This get returns nothing since the filter doesn't match. Filtered cells still retain
// blocks, and this is a full row scan of both blocks. This equals 100 bytes so we should
// throw a multiResponseTooLarge after this get if we are counting filtered cells correctly.
Get g0 = new Get(row).addFamily(FAMILY).setFilter(
new QualifierFilter(CompareOperator.EQUAL, new BinaryComparator(Bytes.toBytes("sdf"))));
gets.add(g0);

// g1 and g2 each count the first 55 byte block, so we end up with block size of 110
// after g2 and throw a multiResponseTooLarge before g3
Get g1 = new Get(row);
g1.addColumn(FAMILY, cols[0]);
gets.add(g1);

Get g2 = new Get(row);
g2.addColumn(FAMILY, cols[3]);
gets.add(g2);

Get g3 = new Get(row);
g3.addColumn(FAMILY, cols[7]);
gets.add(g3);

Result[] results = t.get(gets);
assertEquals(2, results.length);
METRICS_ASSERT.assertCounterGt("exceptions", startingExceptions, s);
METRICS_ASSERT.assertCounterGt("exceptions.multiResponseTooLarge", startingMultiExceptions, s);
assertEquals(4, results.length);
// Expect 2 exceptions (thus 3 rpcs) -- one for g0, then another for g1 + g2, final rpc for g3.
// If we tracked lastBlock we could squeeze g3 into the second rpc because g2 would be "free"
// since it's in the same block as g1.
METRICS_ASSERT.assertCounterGt("exceptions", startingExceptions + 1, s);
METRICS_ASSERT.assertCounterGt("exceptions.multiResponseTooLarge", startingMultiExceptions + 1,
s);
}

private void flush(HRegionServer regionServer, TableName tableName) throws IOException {
for (HRegion region : regionServer.getRegions(tableName)) {
region.flush(true);
}
}
}

0 comments on commit 91e02cf

Please sign in to comment.