You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2020/02/17 16:32:52 UTC

[hbase] branch branch-2 updated: HBASE-23853 [Flakey Test] TestBlockEvictionFromClient#testBlockRefCountAfterSplits ADDENDUM: make set evict on close volatile

This is an automated email from the ASF dual-hosted git repository.

stack pushed a commit to branch branch-2
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/branch-2 by this push:
     new f20e07b  HBASE-23853 [Flakey Test] TestBlockEvictionFromClient#testBlockRefCountAfterSplits ADDENDUM: make set evict on close volatile
f20e07b is described below

commit f20e07b72b429310f594f7b5fe91d4f5fa73b0cd
Author: stack <st...@apache.org>
AuthorDate: Mon Feb 17 08:32:08 2020 -0800

    HBASE-23853 [Flakey Test] TestBlockEvictionFromClient#testBlockRefCountAfterSplits
    ADDENDUM: make set evict on close volatile
---
 .../apache/hadoop/hbase/io/hfile/CacheConfig.java  |  3 +-
 .../hbase/client/TestBlockEvictionFromClient.java  | 43 +++-------------------
 2 files changed, 7 insertions(+), 39 deletions(-)

diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheConfig.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheConfig.java
index dfce791..be4c31f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheConfig.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/CacheConfig.java
@@ -131,7 +131,7 @@ public class CacheConfig {
   private boolean cacheBloomsOnWrite;
 
   /** Whether blocks of a file should be evicted when the file is closed */
-  private boolean evictOnClose;
+  private volatile boolean evictOnClose;
 
   /** Whether data blocks should be stored in compressed and/or encrypted form in the cache */
   private final boolean cacheDataCompressed;
@@ -210,7 +210,6 @@ public class CacheConfig {
 
   /**
    * Constructs a cache configuration copied from the specified configuration.
-   * @param cacheConf
    */
   public CacheConfig(CacheConfig cacheConf) {
     this.cacheDataOnRead = cacheConf.cacheDataOnRead;
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java
index 3f7418e..03b566e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestBlockEvictionFromClient.java
@@ -1,4 +1,4 @@
-/**
+/*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
  * distributed with this work for additional information
@@ -20,7 +20,6 @@ package org.apache.hadoop.hbase.client;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
-
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Iterator;
@@ -58,7 +57,6 @@ import org.apache.hadoop.hbase.regionserver.ScannerContext;
 import org.apache.hadoop.hbase.testclassification.ClientTests;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.hadoop.hbase.util.Threads;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
@@ -70,7 +68,6 @@ import org.junit.experimental.categories.Category;
 import org.junit.rules.TestName;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-
 import org.apache.hbase.thirdparty.com.google.common.collect.Iterables;
 
 @Category({ LargeTests.class, ClientTests.class })
@@ -104,9 +101,6 @@ public class TestBlockEvictionFromClient {
   @Rule
   public TestName name = new TestName();
 
-  /**
-   * @throws java.lang.Exception
-   */
   @BeforeClass
   public static void setUpBeforeClass() throws Exception {
     ROWS[0] = ROW;
@@ -125,17 +119,11 @@ public class TestBlockEvictionFromClient {
     TEST_UTIL.startMiniCluster(SLAVES);
   }
 
-  /**
-   * @throws java.lang.Exception
-   */
   @AfterClass
   public static void tearDownAfterClass() throws Exception {
     TEST_UTIL.shutdownMiniCluster();
   }
 
-  /**
-   * @throws java.lang.Exception
-   */
   @Before
   public void setUp() throws Exception {
     CustomInnerRegionObserver.waitForGets.set(false);
@@ -143,9 +131,6 @@ public class TestBlockEvictionFromClient {
     CustomInnerRegionObserver.countOfGets.set(0);
   }
 
-  /**
-   * @throws java.lang.Exception
-   */
   @After
   public void tearDown() throws Exception {
     if (latch != null) {
@@ -571,18 +556,15 @@ public class TestBlockEvictionFromClient {
   @Test
   public void testBlockRefCountAfterSplits() throws IOException, InterruptedException {
     final TableName tableName = TableName.valueOf(name.getMethodName());
-    try (Table table =
-        TEST_UTIL.createTable(tableName, FAMILIES_1, 1, 1024)) {
+    try (Table table = TEST_UTIL.createTable(tableName, FAMILIES_1, 1, 1024)) {
       // get the block cache and region
       RegionLocator locator = TEST_UTIL.getConnection().getRegionLocator(tableName);
       String regionName = locator.getAllRegionLocations().get(0).getRegion().getEncodedName();
-      HRegion region =
-          TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName);
+      HRegion region = TEST_UTIL.getRSForFirstRegionInTable(tableName).getRegion(regionName);
       HStore store = region.getStores().iterator().next();
       CacheConfig cacheConf = store.getCacheConfig();
       cacheConf.setEvictOnClose(true);
       BlockCache cache = cacheConf.getBlockCache().get();
-
       Put put = new Put(ROW);
       put.addColumn(FAMILY, QUALIFIER, data);
       table.put(put);
@@ -604,21 +586,8 @@ public class TestBlockEvictionFromClient {
       LOG.info("About to SPLIT on {}", Bytes.toString(ROW1));
       TEST_UTIL.getAdmin().split(tableName, ROW1);
       // Wait for splits
-      TEST_UTIL.waitFor(60000,
-        () -> TEST_UTIL.getAdmin().getRegions(rs).size() > regionCount);
-      LOG.info("Split finished");
-      LOG.info("Compaction state {} {}", region, region.getCompactionState());
-      while (region.getCompactionState().compareTo(CompactionState.NONE) != 0) {
-        Threads.sleep(10);
-      }
-      region.waitForFlushesAndCompactions();
-      region.compactStores();
-      region.compact(true);
-      LOG.info("Compaction after {} {}", region, region.getCompactionState());
-      while (region.getCompactionState().compareTo(CompactionState.NONE) != 0) {
-        Threads.sleep(10);
-      }
-      LOG.info("Compaction finished {} {}", region, region.getCompactionState());
+      TEST_UTIL.waitFor(60000, () -> TEST_UTIL.getAdmin().getRegions(rs).size() > regionCount);
+      LOG.info("Split finished, is region closed {}", region.isClosed());
       Iterator<CachedBlock> iterator = cache.iterator();
       // Though the split had created the HalfStorefileReader - the firstkey and lastkey scanners
       // should be closed inorder to return those blocks
@@ -1319,7 +1288,7 @@ public class TestBlockEvictionFromClient {
             // If get has closed only the scan's blocks would be available
             assertEquals(refCount, CustomInnerRegionObserver.countOfGets.get());
           } else {
-              assertEquals(refCount, CustomInnerRegionObserver.countOfGets.get() + (NO_OF_THREADS));
+            assertEquals(refCount, CustomInnerRegionObserver.countOfGets.get() + (NO_OF_THREADS));
           }
         }
       } else {