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.io.IOException; 023import java.util.ArrayList; 024import java.util.List; 025import org.apache.hadoop.hbase.Cell; 026import org.apache.hadoop.hbase.CellBuilderFactory; 027import org.apache.hadoop.hbase.CellBuilderType; 028import org.apache.hadoop.hbase.CompareOperator; 029import org.apache.hadoop.hbase.CompatibilityFactory; 030import org.apache.hadoop.hbase.HBaseTestingUtil; 031import org.apache.hadoop.hbase.HConstants; 032import org.apache.hadoop.hbase.TableName; 033import org.apache.hadoop.hbase.Waiter; 034import org.apache.hadoop.hbase.filter.BinaryComparator; 035import org.apache.hadoop.hbase.filter.QualifierFilter; 036import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; 037import org.apache.hadoop.hbase.ipc.RpcServerInterface; 038import org.apache.hadoop.hbase.logging.Log4jUtils; 039import org.apache.hadoop.hbase.metrics.BaseSource; 040import org.apache.hadoop.hbase.regionserver.HRegion; 041import org.apache.hadoop.hbase.regionserver.HRegionServer; 042import org.apache.hadoop.hbase.test.MetricsAssertHelper; 043import org.apache.hadoop.hbase.testclassification.ClientTests; 044import org.apache.hadoop.hbase.testclassification.MediumTests; 045import org.apache.hadoop.hbase.util.Bytes; 046import org.junit.jupiter.api.AfterAll; 047import org.junit.jupiter.api.BeforeAll; 048import org.junit.jupiter.api.BeforeEach; 049import org.junit.jupiter.api.Tag; 050import org.junit.jupiter.api.Test; 051import org.junit.jupiter.api.TestInfo; 052 053/** 054 * This test sets the multi size WAAAAAY low and then checks to make sure that gets will still make 055 * progress. 056 */ 057@Tag(MediumTests.TAG) 058@Tag(ClientTests.TAG) 059public class TestMultiRespectsLimits { 060 061 private final static HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil(); 062 private static final MetricsAssertHelper METRICS_ASSERT = 063 CompatibilityFactory.getInstance(MetricsAssertHelper.class); 064 private final static byte[] FAMILY = Bytes.toBytes("D"); 065 public static final int MAX_SIZE = 90; 066 private static String LOG_LEVEL; 067 068 private String methodName; 069 070 @BeforeEach 071 public void setUp(TestInfo testInfo) { 072 methodName = testInfo.getTestMethod().get().getName(); 073 } 074 075 @BeforeAll 076 public static void setUpBeforeClass() throws Exception { 077 // disable the debug log to avoid flooding the output 078 LOG_LEVEL = Log4jUtils.getEffectiveLevel(AsyncRegionLocatorHelper.class.getName()); 079 Log4jUtils.setLogLevel(AsyncRegionLocatorHelper.class.getName(), "INFO"); 080 TEST_UTIL.getConfiguration().setLong(HConstants.HBASE_SERVER_SCANNER_MAX_RESULT_SIZE_KEY, 081 MAX_SIZE); 082 083 // Only start on regionserver so that all regions are on the same server. 084 TEST_UTIL.startMiniCluster(1); 085 } 086 087 @AfterAll 088 public static void tearDownAfterClass() throws Exception { 089 if (LOG_LEVEL != null) { 090 Log4jUtils.setLogLevel(AsyncRegionLocatorHelper.class.getName(), LOG_LEVEL); 091 } 092 TEST_UTIL.shutdownMiniCluster(); 093 } 094 095 @Test 096 public void testMultiLimits() throws Exception { 097 final TableName tableName = TableName.valueOf(methodName); 098 Table t = TEST_UTIL.createTable(tableName, FAMILY); 099 TEST_UTIL.loadTable(t, FAMILY, false); 100 101 // Split the table to make sure that the chunking happens accross regions. 102 try (final Admin admin = TEST_UTIL.getAdmin()) { 103 admin.split(tableName); 104 TEST_UTIL.waitFor(60000, new Waiter.Predicate<Exception>() { 105 @Override 106 public boolean evaluate() throws Exception { 107 return admin.getRegions(tableName).size() > 1; 108 } 109 }); 110 } 111 List<Get> gets = new ArrayList<>(MAX_SIZE); 112 113 for (int i = 0; i < MAX_SIZE; i++) { 114 gets.add(new Get(HBaseTestingUtil.ROWS[i])); 115 } 116 117 RpcServerInterface rpcServer = TEST_UTIL.getHBaseCluster().getRegionServer(0).getRpcServer(); 118 BaseSource s = rpcServer.getMetrics().getMetricsSource(); 119 long startingExceptions = METRICS_ASSERT.getCounter("exceptions", s); 120 long startingMultiExceptions = METRICS_ASSERT.getCounter("exceptions.multiResponseTooLarge", s); 121 122 Result[] results = t.get(gets); 123 assertEquals(MAX_SIZE, results.length); 124 125 // Cells from TEST_UTIL.loadTable have a length of 27. 126 // Multiplying by less than that gives an easy lower bound on size. 127 // However in reality each kv is being reported as much higher than that. 128 METRICS_ASSERT.assertCounterGt("exceptions", startingExceptions + ((MAX_SIZE * 25) / MAX_SIZE), 129 s); 130 METRICS_ASSERT.assertCounterGt("exceptions.multiResponseTooLarge", 131 startingMultiExceptions + ((MAX_SIZE * 25) / MAX_SIZE), s); 132 } 133 134 @Test 135 public void testBlockMultiLimits() throws Exception { 136 final TableName tableName = TableName.valueOf(methodName); 137 TEST_UTIL.getAdmin().createTable( 138 TableDescriptorBuilder.newBuilder(tableName).setColumnFamily(ColumnFamilyDescriptorBuilder 139 .newBuilder(FAMILY).setDataBlockEncoding(DataBlockEncoding.FAST_DIFF).build()).build()); 140 Table t = TEST_UTIL.getConnection().getTable(tableName); 141 142 final HRegionServer regionServer = TEST_UTIL.getHBaseCluster().getRegionServer(0); 143 RpcServerInterface rpcServer = regionServer.getRpcServer(); 144 BaseSource s = rpcServer.getMetrics().getMetricsSource(); 145 long startingExceptions = METRICS_ASSERT.getCounter("exceptions", s); 146 long startingMultiExceptions = METRICS_ASSERT.getCounter("exceptions.multiResponseTooLarge", s); 147 148 byte[] row = Bytes.toBytes("TEST"); 149 byte[][] cols = new byte[][] { Bytes.toBytes("0"), // Get this 150 Bytes.toBytes("1"), // Buffer 151 Bytes.toBytes("2"), // Buffer 152 Bytes.toBytes("3"), // Get This 153 Bytes.toBytes("4"), // Buffer 154 Bytes.toBytes("5"), // Buffer 155 Bytes.toBytes("6"), // Buffer 156 Bytes.toBytes("7"), // Get This 157 Bytes.toBytes("8"), // Buffer 158 Bytes.toBytes("9"), // Buffer 159 }; 160 161 // Set the value size so that one result will be less than the MAX_SIZE 162 // however the block being reference will be larger than MAX_SIZE. 163 // This should cause the regionserver to try and send a result immediately. 164 byte[] value = new byte[1]; 165 Bytes.random(value); 166 167 for (int i = 0; i < cols.length; i++) { 168 if (i == 6) { 169 // do a flush here so we end up with 2 blocks, 55 and 45 bytes 170 flush(regionServer, tableName); 171 } 172 byte[] col = cols[i]; 173 Put p = new Put(row); 174 p.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY).setRow(row).setFamily(FAMILY) 175 .setQualifier(col).setTimestamp(p.getTimestamp()).setType(Cell.Type.Put).setValue(value) 176 .build()); 177 t.put(p); 178 } 179 180 // Make sure that a flush happens 181 flush(regionServer, tableName); 182 183 List<Get> gets = new ArrayList<>(4); 184 // This get returns nothing since the filter doesn't match. Filtered cells still retain 185 // blocks, and this is a full row scan of both blocks. This equals 100 bytes so we should 186 // throw a multiResponseTooLarge after this get if we are counting filtered cells correctly. 187 Get g0 = new Get(row).addFamily(FAMILY).setFilter( 188 new QualifierFilter(CompareOperator.EQUAL, new BinaryComparator(Bytes.toBytes("sdf")))); 189 gets.add(g0); 190 191 // g1 and g2 each count the first 55 byte block, so we end up with block size of 110 192 // after g2 and throw a multiResponseTooLarge before g3 193 Get g1 = new Get(row); 194 g1.addColumn(FAMILY, cols[0]); 195 gets.add(g1); 196 197 Get g2 = new Get(row); 198 g2.addColumn(FAMILY, cols[3]); 199 gets.add(g2); 200 201 Get g3 = new Get(row); 202 g3.addColumn(FAMILY, cols[7]); 203 gets.add(g3); 204 205 Result[] results = t.get(gets); 206 assertEquals(4, results.length); 207 // Expect 2 exceptions (thus 3 rpcs) -- one for g0, then another for g1 + g2, final rpc for g3. 208 // If we tracked lastBlock we could squeeze g3 into the second rpc because g2 would be "free" 209 // since it's in the same block as g1. 210 METRICS_ASSERT.assertCounterGt("exceptions", startingExceptions + 1, s); 211 METRICS_ASSERT.assertCounterGt("exceptions.multiResponseTooLarge", startingMultiExceptions + 1, 212 s); 213 } 214 215 private void flush(HRegionServer regionServer, TableName tableName) throws IOException { 216 for (HRegion region : regionServer.getRegions(tableName)) { 217 region.flush(true); 218 } 219 } 220}