103103import  org .apache .hadoop .hbase .regionserver .compactions .DefaultCompactor ;
104104import  org .apache .hadoop .hbase .regionserver .querymatcher .ScanQueryMatcher ;
105105import  org .apache .hadoop .hbase .regionserver .throttle .NoLimitThroughputController ;
106+ import  org .apache .hadoop .hbase .regionserver .throttle .ThroughputController ;
106107import  org .apache .hadoop .hbase .security .User ;
107108import  org .apache .hadoop .hbase .testclassification .MediumTests ;
108109import  org .apache .hadoop .hbase .testclassification .RegionServerTests ;
@@ -784,11 +785,12 @@ private void injectFault() throws IOException {
784785    }
785786  }
786787
787-   private  static  void  flushStore (HStore  store , long  id ) throws  IOException  {
788+   private  static  StoreFlushContext  flushStore (HStore  store , long  id ) throws  IOException  {
788789    StoreFlushContext  storeFlushCtx  = store .createFlushContext (id , FlushLifeCycleTracker .DUMMY );
789790    storeFlushCtx .prepare ();
790791    storeFlushCtx .flushCache (Mockito .mock (MonitoredTask .class ));
791792    storeFlushCtx .commit (Mockito .mock (MonitoredTask .class ));
793+     return  storeFlushCtx ;
792794  }
793795
794796  /** 
@@ -2236,7 +2238,7 @@ public void testClearSnapshotGetScannerConcurrently() throws Exception {
22362238      flushThread .join ();
22372239
22382240      if  (myDefaultMemStore .shouldWait ) {
2239-         SegmentScanner  segmentScanner  = getSegmentScanner (storeScanner );
2241+         SegmentScanner  segmentScanner  = getTypeKeyValueScanner (storeScanner ,  SegmentScanner . class );
22402242        MemStoreLABImpl  memStoreLAB  = (MemStoreLABImpl ) (segmentScanner .segment .getMemStoreLAB ());
22412243        assertTrue (memStoreLAB .isClosed ());
22422244        assertTrue (!memStoreLAB .chunks .isEmpty ());
@@ -2263,16 +2265,16 @@ public void testClearSnapshotGetScannerConcurrently() throws Exception {
22632265    }
22642266  }
22652267
2266-   private  SegmentScanner  getSegmentScanner (StoreScanner  storeScanner ) {
2267-     List <SegmentScanner > segmentScanners  = new  ArrayList <SegmentScanner >();
2268+   @ SuppressWarnings ("unchecked" )
2269+   private  <T > T  getTypeKeyValueScanner (StoreScanner  storeScanner , Class <T > keyValueScannerClass ) {
2270+     List <T > resultScanners  = new  ArrayList <T >();
22682271    for  (KeyValueScanner  keyValueScanner  : storeScanner .currentScanners ) {
2269-       if  (keyValueScanner   instanceof   SegmentScanner ) {
2270-         segmentScanners .add ((SegmentScanner ) keyValueScanner );
2272+       if  (keyValueScannerClass . isInstance ( keyValueScanner ) ) {
2273+         resultScanners .add ((T ) keyValueScanner );
22712274      }
22722275    }
2273- 
2274-     assertTrue (segmentScanners .size () == 1 );
2275-     return  segmentScanners .get (0 );
2276+     assertTrue (resultScanners .size () == 1 );
2277+     return  resultScanners .get (0 );
22762278  }
22772279
22782280  @ Test  
@@ -2324,6 +2326,116 @@ public CustomDefaultMemStore(Configuration conf, CellComparator c,
23242326
23252327  }
23262328
2329+   /** 
2330+    * This test is for HBASE-26488 
2331+    */ 
2332+   @ Test 
2333+   public  void  testMemoryLeakWhenFlushMemStoreRetrying () throws  Exception  {
2334+ 
2335+     Configuration  conf  = HBaseConfiguration .create ();
2336+ 
2337+     byte [] smallValue  = new  byte [3 ];
2338+     byte [] largeValue  = new  byte [9 ];
2339+     final  long  timestamp  = EnvironmentEdgeManager .currentTime ();
2340+     final  long  seqId  = 100 ;
2341+     final  Cell  smallCell  = createCell (qf1 , timestamp , seqId , smallValue );
2342+     final  Cell  largeCell  = createCell (qf2 , timestamp , seqId , largeValue );
2343+     TreeSet <byte []> quals  = new  TreeSet <>(Bytes .BYTES_COMPARATOR );
2344+     quals .add (qf1 );
2345+     quals .add (qf2 );
2346+ 
2347+     conf .set (HStore .MEMSTORE_CLASS_NAME , MyDefaultMemStore1 .class .getName ());
2348+     conf .setBoolean (WALFactory .WAL_ENABLED , false );
2349+     conf .set (DefaultStoreEngine .DEFAULT_STORE_FLUSHER_CLASS_KEY ,
2350+       MyDefaultStoreFlusher .class .getName ());
2351+ 
2352+     init (name .getMethodName (), conf , ColumnFamilyDescriptorBuilder .newBuilder (family ).build ());
2353+     MyDefaultMemStore1  myDefaultMemStore  = (MyDefaultMemStore1 ) (store .memstore );
2354+     assertTrue ((store .storeEngine .getStoreFlusher ()) instanceof  MyDefaultStoreFlusher );
2355+ 
2356+     MemStoreSizing  memStoreSizing  = new  NonThreadSafeMemStoreSizing ();
2357+     store .add (smallCell , memStoreSizing );
2358+     store .add (largeCell , memStoreSizing );
2359+     flushStore (store , id ++);
2360+ 
2361+     MemStoreLABImpl  memStoreLAB  =
2362+         (MemStoreLABImpl ) (myDefaultMemStore .snapshotImmutableSegment .getMemStoreLAB ());
2363+     assertTrue (memStoreLAB .isClosed ());
2364+     assertTrue (memStoreLAB .getOpenScannerCount () == 0 );
2365+     assertTrue (memStoreLAB .isReclaimed ());
2366+     assertTrue (memStoreLAB .chunks .isEmpty ());
2367+     StoreScanner  storeScanner  = null ;
2368+     try  {
2369+       storeScanner  =
2370+           (StoreScanner ) store .getScanner (new  Scan (new  Get (row )), quals , seqId  + 1 );
2371+       assertTrue (store .storeEngine .getStoreFileManager ().getStorefileCount () == 1 );
2372+       assertTrue (store .memstore .size ().getCellsCount () == 0 );
2373+       assertTrue (store .memstore .getSnapshotSize ().getCellsCount () == 0 );
2374+       assertTrue (storeScanner .currentScanners .size () == 1 );
2375+       assertTrue (storeScanner .currentScanners .get (0 ) instanceof  StoreFileScanner );
2376+ 
2377+       List <Cell > results  = new  ArrayList <>();
2378+       storeScanner .next (results );
2379+       assertEquals (2 , results .size ());
2380+       CellUtil .equals (smallCell , results .get (0 ));
2381+       CellUtil .equals (largeCell , results .get (1 ));
2382+     } finally  {
2383+       if  (storeScanner  != null ) {
2384+         storeScanner .close ();
2385+       }
2386+     }
2387+   }
2388+ 
2389+ 
2390+   static  class  MyDefaultMemStore1  extends  DefaultMemStore  {
2391+ 
2392+     private  ImmutableSegment  snapshotImmutableSegment ;
2393+ 
2394+     public  MyDefaultMemStore1 (Configuration  conf , CellComparator  c ,
2395+         RegionServicesForStores  regionServices ) {
2396+       super (conf , c , regionServices );
2397+     }
2398+ 
2399+     @ Override 
2400+     public  MemStoreSnapshot  snapshot () {
2401+       MemStoreSnapshot  result  = super .snapshot ();
2402+       this .snapshotImmutableSegment  = snapshot ;
2403+       return  result ;
2404+     }
2405+ 
2406+   }
2407+ 
2408+   public  static  class  MyDefaultStoreFlusher  extends  DefaultStoreFlusher  {
2409+     private  static  final  AtomicInteger  failCounter  = new  AtomicInteger (1 );
2410+     private  static  final  AtomicInteger  counter  = new  AtomicInteger (0 );
2411+ 
2412+     public  MyDefaultStoreFlusher (Configuration  conf , HStore  store ) {
2413+       super (conf , store );
2414+     }
2415+ 
2416+     @ Override 
2417+     public  List <Path > flushSnapshot (MemStoreSnapshot  snapshot , long  cacheFlushId ,
2418+         MonitoredTask  status , ThroughputController  throughputController ,
2419+         FlushLifeCycleTracker  tracker ) throws  IOException  {
2420+       counter .incrementAndGet ();
2421+       return  super .flushSnapshot (snapshot , cacheFlushId , status , throughputController , tracker );
2422+     }
2423+ 
2424+     @ Override 
2425+     protected  void  performFlush (InternalScanner  scanner , final  CellSink  sink ,
2426+         ThroughputController  throughputController ) throws  IOException  {
2427+ 
2428+       final  int  currentCount  = counter .get ();
2429+       CellSink  newCellSink  = (cell ) -> {
2430+         if  (currentCount  <= failCounter .get ()) {
2431+           throw  new  IOException ("Simulated exception by tests" );
2432+         }
2433+         sink .append (cell );
2434+       };
2435+       super .performFlush (scanner , newCellSink , throughputController );
2436+     }
2437+   }
2438+ 
23272439  private  HStoreFile  mockStoreFileWithLength (long  length ) {
23282440    HStoreFile  sf  = mock (HStoreFile .class );
23292441    StoreFileReader  sfr  = mock (StoreFileReader .class );
@@ -3107,7 +3219,5 @@ protected void doClearSnapShot() {
31073219        }
31083220      }
31093221    }
3110- 
3111- 
31123222  }
31133223}
0 commit comments