@@ -300,38 +300,33 @@ boolean passesBloomFilter(Scan scan, final SortedSet<byte[]> columns) {
300300 }
301301
302302 public boolean passesDeleteFamilyBloomFilter (byte [] row , int rowOffset , int rowLen ) {
303- if (reader instanceof MultiTenantBloomSupport ) {
304- try {
305- return ((MultiTenantBloomSupport ) reader ).passesDeleteFamilyBloomFilter (row , rowOffset ,
306- rowLen );
307- } catch (IOException e ) {
308- LOG .warn ("Failed multi-tenant delete-family bloom check, proceeding without" , e );
309- return true ;
310- }
311- }
312- // Cache Bloom filter as a local variable in case it is set to null by
313- // another thread on an IO error.
314303 BloomFilter bloomFilter = this .deleteFamilyBloomFilter ;
315304
316- // Empty file or there is no delete family at all
317305 if (reader .getTrailer ().getEntryCount () == 0 || deleteFamilyCnt == 0 ) {
318306 return false ;
319307 }
320308
321- if (bloomFilter == null ) {
309+ if (bloomFilter != null ) {
310+ try {
311+ if (!bloomFilter .supportsAutoLoading ()) {
312+ return true ;
313+ }
314+ return bloomFilter .contains (row , rowOffset , rowLen , null );
315+ } catch (IllegalArgumentException e ) {
316+ LOG .error ("Bad Delete Family bloom filter data -- proceeding without" , e );
317+ setDeleteFamilyBloomFilterFaulty ();
318+ }
322319 return true ;
323320 }
324-
325- try {
326- if (!bloomFilter .supportsAutoLoading ()) {
321+ if (reader instanceof MultiTenantBloomSupport ) {
322+ try {
323+ return ((MultiTenantBloomSupport ) reader ).passesDeleteFamilyBloomFilter (row , rowOffset ,
324+ rowLen );
325+ } catch (IOException e ) {
326+ LOG .warn ("Failed multi-tenant delete-family bloom check, proceeding without" , e );
327327 return true ;
328328 }
329- return bloomFilter .contains (row , rowOffset , rowLen , null );
330- } catch (IllegalArgumentException e ) {
331- LOG .error ("Bad Delete Family bloom filter data -- proceeding without" , e );
332- setDeleteFamilyBloomFilterFaulty ();
333329 }
334-
335330 return true ;
336331 }
337332
@@ -341,6 +336,13 @@ public boolean passesDeleteFamilyBloomFilter(byte[] row, int rowOffset, int rowL
341336 * @return True if passes
342337 */
343338 private boolean passesGeneralRowBloomFilter (byte [] row , int rowOffset , int rowLen ) {
339+ BloomFilter bloomFilter = this .generalBloomFilter ;
340+ if (bloomFilter != null ) {
341+ if (rowOffset != 0 || rowLen != row .length ) {
342+ throw new AssertionError ("For row-only Bloom filters the row must occupy the whole array" );
343+ }
344+ return checkGeneralBloomFilter (row , null , bloomFilter );
345+ }
344346 if (reader instanceof MultiTenantBloomSupport ) {
345347 try {
346348 boolean passed =
@@ -352,19 +354,8 @@ private boolean passesGeneralRowBloomFilter(byte[] row, int rowOffset, int rowLe
352354 return true ;
353355 }
354356 }
355- BloomFilter bloomFilter = this .generalBloomFilter ;
356- if (bloomFilter == null ) {
357- incrementBloomEligible ();
358- return true ;
359- }
360-
361- // Used in ROW bloom
362- byte [] key = null ;
363- if (rowOffset != 0 || rowLen != row .length ) {
364- throw new AssertionError ("For row-only Bloom filters the row must occupy the whole array" );
365- }
366- key = row ;
367- return checkGeneralBloomFilter (key , null , bloomFilter );
357+ incrementBloomEligible ();
358+ return true ;
368359 }
369360
370361 /**
@@ -373,6 +364,16 @@ private boolean passesGeneralRowBloomFilter(byte[] row, int rowOffset, int rowLe
373364 * @return True if passes
374365 */
375366 public boolean passesGeneralRowColBloomFilter (ExtendedCell cell ) {
367+ BloomFilter bloomFilter = this .generalBloomFilter ;
368+ if (bloomFilter != null ) {
369+ ExtendedCell kvKey ;
370+ if (cell .getTypeByte () == KeyValue .Type .Maximum .getCode () && cell .getFamilyLength () == 0 ) {
371+ kvKey = cell ;
372+ } else {
373+ kvKey = PrivateCellUtil .createFirstOnRowCol (cell );
374+ }
375+ return checkGeneralBloomFilter (null , kvKey , bloomFilter );
376+ }
376377 if (reader instanceof MultiTenantBloomSupport ) {
377378 try {
378379 boolean passed = ((MultiTenantBloomSupport ) reader ).passesGeneralRowColBloomFilter (cell );
@@ -383,20 +384,8 @@ public boolean passesGeneralRowColBloomFilter(ExtendedCell cell) {
383384 return true ;
384385 }
385386 }
386- BloomFilter bloomFilter = this .generalBloomFilter ;
387- if (bloomFilter == null ) {
388- incrementBloomEligible ();
389- return true ;
390- }
391- // Used in ROW_COL bloom
392- ExtendedCell kvKey = null ;
393- // Already if the incoming key is a fake rowcol key then use it as it is
394- if (cell .getTypeByte () == KeyValue .Type .Maximum .getCode () && cell .getFamilyLength () == 0 ) {
395- kvKey = cell ;
396- } else {
397- kvKey = PrivateCellUtil .createFirstOnRowCol (cell );
398- }
399- return checkGeneralBloomFilter (null , kvKey , bloomFilter );
387+ incrementBloomEligible ();
388+ return true ;
400389 }
401390
402391 /**
@@ -418,6 +407,10 @@ private boolean passesGeneralRowPrefixBloomFilter(Scan scan) {
418407 rowPrefix = Bytes .copy (row , 0 , prefixLength );
419408 }
420409
410+ BloomFilter bloomFilter = this .generalBloomFilter ;
411+ if (bloomFilter != null ) {
412+ return checkGeneralBloomFilter (rowPrefix , null , bloomFilter );
413+ }
421414 if (reader instanceof MultiTenantBloomSupport ) {
422415 try {
423416 boolean passed = ((MultiTenantBloomSupport ) reader )
@@ -429,13 +422,8 @@ private boolean passesGeneralRowPrefixBloomFilter(Scan scan) {
429422 return true ;
430423 }
431424 }
432-
433- BloomFilter bloomFilter = this .generalBloomFilter ;
434- if (bloomFilter == null ) {
435- incrementBloomEligible ();
436- return true ;
437- }
438- return checkGeneralBloomFilter (rowPrefix , null , bloomFilter );
425+ incrementBloomEligible ();
426+ return true ;
439427 }
440428
441429 private boolean checkGeneralBloomFilter (byte [] key , Cell kvKey , BloomFilter bloomFilter ) {
0 commit comments