1919
2020import static org .junit .Assert .assertEquals ;
2121
22+ import java .io .IOException ;
2223import java .util .ArrayList ;
2324import java .util .List ;
2425import org .apache .hadoop .hbase .Cell ;
2526import org .apache .hadoop .hbase .CellBuilderFactory ;
2627import org .apache .hadoop .hbase .CellBuilderType ;
28+ import org .apache .hadoop .hbase .CompareOperator ;
2729import org .apache .hadoop .hbase .CompatibilityFactory ;
2830import org .apache .hadoop .hbase .HBaseClassTestRule ;
2931import org .apache .hadoop .hbase .HBaseTestingUtil ;
3032import org .apache .hadoop .hbase .HConstants ;
3133import org .apache .hadoop .hbase .TableName ;
3234import org .apache .hadoop .hbase .Waiter ;
35+ import org .apache .hadoop .hbase .filter .BinaryComparator ;
36+ import org .apache .hadoop .hbase .filter .QualifierFilter ;
3337import org .apache .hadoop .hbase .io .encoding .DataBlockEncoding ;
3438import org .apache .hadoop .hbase .ipc .RpcServerInterface ;
3539import org .apache .hadoop .hbase .logging .Log4jUtils ;
3640import org .apache .hadoop .hbase .metrics .BaseSource ;
41+ import org .apache .hadoop .hbase .regionserver .HRegion ;
3742import org .apache .hadoop .hbase .regionserver .HRegionServer ;
3843import org .apache .hadoop .hbase .test .MetricsAssertHelper ;
3944import org .apache .hadoop .hbase .testclassification .ClientTests ;
@@ -62,7 +67,7 @@ public class TestMultiRespectsLimits {
6267 private static final MetricsAssertHelper METRICS_ASSERT =
6368 CompatibilityFactory .getInstance (MetricsAssertHelper .class );
6469 private final static byte [] FAMILY = Bytes .toBytes ("D" );
65- public static final int MAX_SIZE = 50 ;
70+ public static final int MAX_SIZE = 90 ;
6671 private static String LOG_LEVEL ;
6772
6873 @ Rule
@@ -148,6 +153,10 @@ public void testBlockMultiLimits() throws Exception {
148153 Bytes .toBytes ("3" ), // Get This
149154 Bytes .toBytes ("4" ), // Buffer
150155 Bytes .toBytes ("5" ), // Buffer
156+ Bytes .toBytes ("6" ), // Buffer
157+ Bytes .toBytes ("7" ), // Get This
158+ Bytes .toBytes ("8" ), // Buffer
159+ Bytes .toBytes ("9" ), // Buffer
151160 };
152161
153162 // Set the value size so that one result will be less than the MAX_SIZE
@@ -156,7 +165,12 @@ public void testBlockMultiLimits() throws Exception {
156165 byte [] value = new byte [1 ];
157166 Bytes .random (value );
158167
159- for (byte [] col : cols ) {
168+ for (int i = 0 ; i < cols .length ; i ++) {
169+ if (i == 6 ) {
170+ // do a flush here so we end up with 2 blocks, 55 and 45 bytes
171+ flush (regionServer , tableName );
172+ }
173+ byte [] col = cols [i ];
160174 Put p = new Put (row );
161175 p .add (CellBuilderFactory .create (CellBuilderType .SHALLOW_COPY ).setRow (row ).setFamily (FAMILY )
162176 .setQualifier (col ).setTimestamp (p .getTimestamp ()).setType (Cell .Type .Put ).setValue (value )
@@ -165,28 +179,43 @@ public void testBlockMultiLimits() throws Exception {
165179 }
166180
167181 // Make sure that a flush happens
168- try (final Admin admin = TEST_UTIL .getAdmin ()) {
169- admin .flush (tableName );
170- TEST_UTIL .waitFor (60000 , new Waiter .Predicate <Exception >() {
171- @ Override
172- public boolean evaluate () throws Exception {
173- return regionServer .getRegions (tableName ).get (0 ).getMaxFlushedSeqId () > 3 ;
174- }
175- });
176- }
177-
178- List <Get > gets = new ArrayList <>(2 );
179- Get g0 = new Get (row );
180- g0 .addColumn (FAMILY , cols [0 ]);
182+ flush (regionServer , tableName );
183+
184+ List <Get > gets = new ArrayList <>(4 );
185+ // This get returns nothing since the filter doesn't match. Filtered cells still retain
186+ // blocks, and this is a full row scan of both blocks. This equals 100 bytes so we should
187+ // throw a multiResponseTooLarge after this get if we are counting filtered cells correctly.
188+ Get g0 = new Get (row ).addFamily (FAMILY ).setFilter (
189+ new QualifierFilter (CompareOperator .EQUAL , new BinaryComparator (Bytes .toBytes ("sdf" ))));
181190 gets .add (g0 );
182191
192+ // g1 and g2 each count the first 55 byte block, so we end up with block size of 110
193+ // after g2 and throw a multiResponseTooLarge before g3
194+ Get g1 = new Get (row );
195+ g1 .addColumn (FAMILY , cols [0 ]);
196+ gets .add (g1 );
197+
183198 Get g2 = new Get (row );
184199 g2 .addColumn (FAMILY , cols [3 ]);
185200 gets .add (g2 );
186201
202+ Get g3 = new Get (row );
203+ g3 .addColumn (FAMILY , cols [7 ]);
204+ gets .add (g3 );
205+
187206 Result [] results = t .get (gets );
188- assertEquals (2 , results .length );
189- METRICS_ASSERT .assertCounterGt ("exceptions" , startingExceptions , s );
190- METRICS_ASSERT .assertCounterGt ("exceptions.multiResponseTooLarge" , startingMultiExceptions , s );
207+ assertEquals (4 , results .length );
208+ // Expect 2 exceptions (thus 3 rpcs) -- one for g0, then another for g1 + g2, final rpc for g3.
209+ // If we tracked lastBlock we could squeeze g3 into the second rpc because g2 would be "free"
210+ // since it's in the same block as g1.
211+ METRICS_ASSERT .assertCounterGt ("exceptions" , startingExceptions + 1 , s );
212+ METRICS_ASSERT .assertCounterGt ("exceptions.multiResponseTooLarge" , startingMultiExceptions + 1 ,
213+ s );
214+ }
215+
216+ private void flush (HRegionServer regionServer , TableName tableName ) throws IOException {
217+ for (HRegion region : regionServer .getRegions (tableName )) {
218+ region .flush (true );
219+ }
191220 }
192221}
0 commit comments