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


##########
hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetch.java:
##########
@@ -336,6 +337,62 @@ public void testPrefetchDoesntSkipRefs() throws Exception {
     });
   }
 
+  @Test
+  public void testOnConfigurationChange() {
+    PrefetchExecutorNotifier prefetchExecutorNotifier = new 
PrefetchExecutorNotifier(conf);
+    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);
+
+    conf.setInt(PREFETCH_DELAY, 1000);
+    prefetchExecutorNotifier.onConfigurationChange(conf);
+  }
+
+  @Test
+  public void testPrefetchWithDelay() throws Exception {
+    // Configure custom delay
+    PrefetchExecutorNotifier prefetchExecutorNotifier = new 
PrefetchExecutorNotifier(conf);
+    conf.setInt(PREFETCH_DELAY, 25000);
+    prefetchExecutorNotifier.onConfigurationChange(conf);
+
+    HFileContext context = new 
HFileContextBuilder().withCompression(Compression.Algorithm.GZ)
+      .withBlockSize(DATA_BLOCK_SIZE).build();
+    Path storeFile = writeStoreFile("TestPrefetchWithDelay", context);
+
+    HFile.Reader reader = HFile.createReader(fs, storeFile, cacheConf, true, 
conf);
+    long startTime = System.currentTimeMillis();
+
+    // Wait for 20 seconds, no thread should start prefetch
+    Thread.sleep(20000);
+    assertFalse("Prefetch threads should not be running at this point", 
reader.prefetchStarted());
+    while (!reader.prefetchStarted()) {
+      assertTrue("Prefetch delay has not been expired yet",
+        getElapsedTime(startTime) < PrefetchExecutor.getPrefetchDelay());
+    }
+
+    // Prefech threads started working but not completed yet
+    assertFalse(reader.prefetchComplete());

Review Comment:
   Why are we testing this? IMO, it's irrelevant for the feature being 
implemented here.



##########
hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestPrefetch.java:
##########
@@ -336,6 +337,62 @@ public void testPrefetchDoesntSkipRefs() throws Exception {
     });
   }
 
+  @Test
+  public void testOnConfigurationChange() {
+    PrefetchExecutorNotifier prefetchExecutorNotifier = new 
PrefetchExecutorNotifier(conf);
+    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);
+
+    conf.setInt(PREFETCH_DELAY, 1000);
+    prefetchExecutorNotifier.onConfigurationChange(conf);
+  }
+
+  @Test
+  public void testPrefetchWithDelay() throws Exception {
+    // Configure custom delay
+    PrefetchExecutorNotifier prefetchExecutorNotifier = new 
PrefetchExecutorNotifier(conf);
+    conf.setInt(PREFETCH_DELAY, 25000);
+    prefetchExecutorNotifier.onConfigurationChange(conf);
+
+    HFileContext context = new 
HFileContextBuilder().withCompression(Compression.Algorithm.GZ)
+      .withBlockSize(DATA_BLOCK_SIZE).build();
+    Path storeFile = writeStoreFile("TestPrefetchWithDelay", context);
+
+    HFile.Reader reader = HFile.createReader(fs, storeFile, cacheConf, true, 
conf);
+    long startTime = System.currentTimeMillis();
+
+    // Wait for 20 seconds, no thread should start prefetch
+    Thread.sleep(20000);
+    assertFalse("Prefetch threads should not be running at this point", 
reader.prefetchStarted());
+    while (!reader.prefetchStarted()) {
+      assertTrue("Prefetch delay has not been expired yet",
+        getElapsedTime(startTime) < PrefetchExecutor.getPrefetchDelay());
+    }
+
+    // Prefech threads started working but not completed yet
+    assertFalse(reader.prefetchComplete());
+
+    // In prefetch executor, we further compute passed in delay using 
variation and a random
+    // multiplier to get 'effective delay'. Hence, in the test, for delay of 
25000 milli-secs
+    // check that prefetch is started after 20000 milli-sec and prefetch 
started after that.
+    // However, prefetch should not start after configured delay.
+    if (reader.prefetchStarted()) {
+      LOG.info("elapsed time {}, Delay {}", getElapsedTime(startTime),
+        PrefetchExecutor.getPrefetchDelay());
+      assertTrue("Prefetch should start post configured delay",
+        getElapsedTime(startTime) <= PrefetchExecutor.getPrefetchDelay());
+    }

Review Comment:
   This is really confusing. Why prefetch is supposed to start before the 
passed delay? 



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to