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


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/PrefetchExecutor.java:
##########
@@ -31,23 +31,28 @@
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.conf.ConfigurationManager;
+import org.apache.hadoop.hbase.conf.PropagatingConfigurationObserver;
 import org.apache.hadoop.hbase.trace.TraceUtil;
 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
 import org.apache.yetus.audience.InterfaceAudience;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 @InterfaceAudience.Private
-public final class PrefetchExecutor {
+public final class PrefetchExecutor implements 
PropagatingConfigurationObserver {

Review Comment:
   Where are we registering this observer to get notified by 
ConfigurationManager?



##########
hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetch.java:
##########
@@ -298,6 +299,44 @@ public void testPrefetchDoesntSkipRefs() throws Exception {
     });
   }
 
+  @Test
+  public void testOnConfigurationChange() {
+    // change PREFETCH_DELAY_ENABLE_KEY from false to true
+    conf.setInt(PREFETCH_DELAY, 40000);
+    PrefetchExecutor.loadConfiguration(conf);
+    assertTrue(getPrefetchDelay() == 40000);
+
+    // restore
+    conf.setInt(PREFETCH_DELAY, 30000);
+    PrefetchExecutor.loadConfiguration(conf);
+    assertTrue(getPrefetchDelay() == 30000);
+
+  }
+
+  @Test
+  public void testPrefetchWithDelay() throws Exception {

Review Comment:
   Can we assert that 1) at first, no Prefetch thread is running, 2) After the 
configured delay has elapsed, prefetch thread runs?



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/PrefetchExecutor.java:
##########
@@ -134,4 +139,25 @@ private PrefetchExecutor() {
   static ScheduledExecutorService getExecutorPool() {
     return prefetchExecutorPool;
   }
+
+  public static int getPrefetchDelay() {
+    return prefetchDelayMillis;
+  }
+
+  @Override
+  public void onConfigurationChange(Configuration conf) {

Review Comment:
   nit: please add info logging that config has changed.



-- 
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