wchevreuil commented on code in PR #5605:
URL: https://github.com/apache/hbase/pull/5605#discussion_r1504055111


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/PrefetchExecutor.java:
##########
@@ -78,7 +85,10 @@ public Thread newThread(Runnable r) {
       + Path.SEPARATOR_CHAR + ")|(" + Path.SEPARATOR_CHAR
       + HConstants.HREGION_COMPACTIONDIR_NAME.replace(".", "\\.") + 
Path.SEPARATOR_CHAR + ")");
 
-  public static void request(Path path, Runnable runnable) {
+  // For tests. Contains computed prefetch delay
+  private static long computedPrefetchDelay;
+
+  public static void request(Path path, boolean isInterrupted, Runnable 
runnable) {

Review Comment:
   Why do we need the additional parameter? Just remove the future from the 
collection in the "interrupt" method.



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/PrefetchExecutor.java:
##########
@@ -127,11 +153,44 @@ public static boolean isCompleted(Path path) {
     return true;
   }
 
-  private PrefetchExecutor() {

Review Comment:
   Why are we making this public now?



##########
hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetch.java:
##########
@@ -106,10 +115,19 @@ public class TestPrefetch {
   @Before
   public void setUp() throws IOException {
     conf = TEST_UTIL.getConfiguration();
+    long var = conf.getInt(PREFETCH_DELAY, 1000);

Review Comment:
   nit: useless line, please remove it.



##########
hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetch.java:
##########
@@ -106,10 +115,19 @@ public class TestPrefetch {
   @Before
   public void setUp() throws IOException {
     conf = TEST_UTIL.getConfiguration();
+    long var = conf.getInt(PREFETCH_DELAY, 1000);
     conf.setBoolean(CacheConfig.PREFETCH_BLOCKS_ON_OPEN_KEY, true);
     fs = HFileSystem.get(conf);
     blockCache = BlockCacheFactory.createBlockCache(conf);
     cacheConf = new CacheConfig(conf, blockCache);
+    prefetchExecutorNotifier = new PrefetchExecutorNotifier(conf);
+    resetTiming();

Review Comment:
   Is this irrelevant for non delay related test? If so, remove it from here 
and put at the end of the related tests only.



##########
hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetch.java:
##########
@@ -84,6 +87,7 @@
 @Category({ IOTests.class, MediumTests.class })
 public class TestPrefetch {
   private static final Logger LOG = 
LoggerFactory.getLogger(TestPrefetch.class);
+  protected PrefetchExecutorNotifier prefetchExecutorNotifier;

Review Comment:
   Why is this global? It should be created within the context of each 
individual test, so better make it local on each test method.



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/PrefetchExecutor.java:
##########
@@ -78,7 +85,10 @@ public Thread newThread(Runnable r) {
       + Path.SEPARATOR_CHAR + ")|(" + Path.SEPARATOR_CHAR
       + HConstants.HREGION_COMPACTIONDIR_NAME.replace(".", "\\.") + 
Path.SEPARATOR_CHAR + ")");
 
-  public static void request(Path path, Runnable runnable) {
+  // For tests. Contains computed prefetch delay
+  private static long computedPrefetchDelay;

Review Comment:
   Do we really need to calculate and check this on the tests? I guess the 
configured delay is a guaranteed minimum. so we could just check 1) before that 
minimum no prefetch is running, 2) we check that once prefetch has completed it 
took at least the minimum time (specified in the delay).



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/PrefetchExecutorNotifier.java:
##########
@@ -0,0 +1,87 @@
+/*
+ * 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
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.regionserver;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.conf.ConfigurationManager;
+import org.apache.hadoop.hbase.conf.PropagatingConfigurationObserver;
+import org.apache.hadoop.hbase.io.hfile.PrefetchExecutor;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Class to submit requests for PrefetchExecutor depending on configuration 
change
+ */
+@InterfaceAudience.Private
+public final class PrefetchExecutorNotifier implements 
PropagatingConfigurationObserver {
+  private static final Logger LOG = 
LoggerFactory.getLogger(CompactSplit.class);
+
+  /** Wait time in miliseconds before executing prefetch */
+  public static final String PREFETCH_DELAY = "hbase.hfile.prefetch.delay";
+
+
+  private final HRegionServer server;
+  private final Configuration conf;
+
+  PrefetchExecutorNotifier(Configuration conf, HRegionServer server) {
+    this.server = server;
+    this.conf = server.getConfiguration();
+  }
+
+  // only for test
+  public PrefetchExecutorNotifier(Configuration conf) {
+    this.server = null;
+    this.conf = conf;
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public void onConfigurationChange(Configuration newConf) {
+    // Update prefetch delay in the prefetch executor class
+    // interrupt and restart threads which have not started executing
+
+    PrefetchExecutor.loadConfiguration(conf);
+    LOG.info("Config hbase.hfile.prefetch.delay is changed to {}",
+      conf.getInt(PREFETCH_DELAY, 1000));
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public void registerChildren(ConfigurationManager manager) {
+    // No children to register.
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public void deregisterChildren(ConfigurationManager manager) {
+    // No children to register
+  }
+
+  public int getPrefetchDelay() {
+    return PrefetchExecutor.getPrefetchDelay();
+  }
+  public long getComputedPrefetchDelay() { return 
PrefetchExecutor.getComputedPrefetchDelay();}

Review Comment:
   `PrefetchExecutor.getComputedPrefetchDelay()` is commented as test only 
purpose, so we should remove this call here and call it directly in the test.



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/PrefetchExecutor.java:
##########
@@ -127,11 +153,44 @@ public static boolean isCompleted(Path path) {
     return true;
   }
 
-  private PrefetchExecutor() {
-  }
-
   /* Visible for testing only */
+  @RestrictedApi(explanation = "Should only be called in tests", link = "",
+    allowedOnPath = ".*/src/test/.*")
   static ScheduledExecutorService getExecutorPool() {
     return prefetchExecutorPool;
   }
+
+  /* Visible for testing only */
+  @RestrictedApi(explanation = "Should only be called in tests", link = "",
+    allowedOnPath = ".*/src/test/.*")
+  public static long getComputedPrefetchDelay() {return computedPrefetchDelay;}

Review Comment:
   Please also explain this is a non thread safe variable that would not yield 
accurate values under real use case scenarios, where we have multiple readers 
getting created and calling `PrefetchExecutor.request` concurrently.



##########
hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetch.java:
##########
@@ -298,6 +318,47 @@ public void testPrefetchDoesntSkipRefs() throws Exception {
     });
   }
 
+  @Test
+  public void testOnConfigurationChange() {
+    conf.setInt(PREFETCH_DELAY, 40000);
+    prefetchExecutorNotifier.onConfigurationChange(conf);
+    assertEquals(prefetchExecutorNotifier.getPrefetchDelay(), 40000);
+
+    // restore
+    conf.setInt(PREFETCH_DELAY, 30000);
+    prefetchExecutorNotifier.onConfigurationChange(conf);
+    assertEquals(prefetchExecutorNotifier.getPrefetchDelay(), 30000);
+  }
+
+  @Test
+  public void testPrefetchWithDelay() throws Exception {
+    conf.setInt(PREFETCH_DELAY, 60000);
+    prefetchExecutorNotifier.onConfigurationChange(conf);
+    long totalCompletedBefore = PrefetchExecutor.getPrefetchFutures().size();
+
+    HFileContext context = new 
HFileContextBuilder().withCompression(Compression.Algorithm.GZ)
+      .withBlockSize(DATA_BLOCK_SIZE).build();
+    Path storeFile = writeStoreFile("TestPrefetchWithDelay", context);
+    setComputeTiming();
+    readStoreFile(storeFile);
+    assertTrue("Elapsed Time {} | Computed Prefetch Delay {}"
+        + getElapsedTime() + 
prefetchExecutorNotifier.getComputedPrefetchDelay(),
+      getElapsedTime() >= prefetchExecutorNotifier.getComputedPrefetchDelay()

Review Comment:
   I don't think this test is correct, as you call your endTimer function only 
after prefetch is guaranteed to be completed, it may yield true even if the 
start delay wasn't respected, but the prefetch execution time took longer to 
complete. 
   
   What I would like to assert here is that right after creating a reader, even 
if we wait for 5 or 10 secs, no prefetch is running as we set the delay to 60 
secs. 



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/PrefetchExecutorNotifier.java:
##########
@@ -0,0 +1,87 @@
+/*
+ * 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
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.regionserver;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.conf.ConfigurationManager;
+import org.apache.hadoop.hbase.conf.PropagatingConfigurationObserver;
+import org.apache.hadoop.hbase.io.hfile.PrefetchExecutor;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Class to submit requests for PrefetchExecutor depending on configuration 
change
+ */
+@InterfaceAudience.Private
+public final class PrefetchExecutorNotifier implements 
PropagatingConfigurationObserver {
+  private static final Logger LOG = 
LoggerFactory.getLogger(CompactSplit.class);
+
+  /** Wait time in miliseconds before executing prefetch */
+  public static final String PREFETCH_DELAY = "hbase.hfile.prefetch.delay";
+
+
+  private final HRegionServer server;

Review Comment:
   nit: unused variable.



##########
hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetch.java:
##########
@@ -106,10 +115,19 @@ public class TestPrefetch {
   @Before
   public void setUp() throws IOException {
     conf = TEST_UTIL.getConfiguration();
+    long var = conf.getInt(PREFETCH_DELAY, 1000);
     conf.setBoolean(CacheConfig.PREFETCH_BLOCKS_ON_OPEN_KEY, true);
     fs = HFileSystem.get(conf);
     blockCache = BlockCacheFactory.createBlockCache(conf);
     cacheConf = new CacheConfig(conf, blockCache);
+    prefetchExecutorNotifier = new PrefetchExecutorNotifier(conf);
+    resetTiming();
+  }
+
+  @After
+  public void resetPrefetchDelay() throws IOException {
+    conf.setInt(PREFETCH_DELAY, 1000);
+    prefetchExecutorNotifier.onConfigurationChange(conf);

Review Comment:
   Same as above.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@hbase.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to