XComp commented on code in PR #20919:
URL: https://github.com/apache/flink/pull/20919#discussion_r1030074324


##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/lookup/fullcache/inputformat/InputFormatCacheLoader.java:
##########
@@ -39,12 +39,14 @@
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
 import java.util.stream.Collectors;
 
 /** {@link CacheLoader} that used {@link InputFormat} for loading data into 
the cache. */
 public class InputFormatCacheLoader extends CacheLoader {
     private static final long serialVersionUID = 1L;
     private static final Logger LOG = 
LoggerFactory.getLogger(InputFormatCacheLoader.class);
+    private static final long TIMEOUT_AFTER_INTERRUPT = 10; // 10 sec

Review Comment:
   ```suggestion
       private static final long TIMEOUT_AFTER_INTERRUPT_IN_SEC = 10; // 10 sec
   ```
   It's good practice to add the unit to the variable instead of using 
comments. That way, the reader is immediately aware of the unit used for the 
variable and doesn't have to scroll up to check the comment next to the 
declaration.



##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/lookup/fullcache/inputformat/InputFormatCacheLoader.java:
##########
@@ -107,7 +109,11 @@ protected void reloadCache() throws Exception {
         } catch (InterruptedException ignored) { // we use interrupt to close 
reload thread
         } finally {
             if (cacheLoadTaskService != null) {
+                // if main cache reload thread encountered an exception,
+                // it interrupts underlying InputSplitCacheLoadTasks threads
                 cacheLoadTaskService.shutdownNow();
+                // timeout 10 sec should definitely be enough to wait for 
finish after interrupt

Review Comment:
   ```suggestion
   ```
   nit: I'm not sure whether it's a good to add the 10s information here in the 
comment. This will make it necessary for anyone who plans to change the value 
to edit two different lines of code. I'm also inclined to say that the comment 
isn't adding any value to the code. Therefore, removing it would solve the 
duplicate information issue.



##########
flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/lookup/fullcache/inputformat/InputFormatCacheLoader.java:
##########
@@ -107,7 +109,11 @@ protected void reloadCache() throws Exception {
         } catch (InterruptedException ignored) { // we use interrupt to close 
reload thread
         } finally {
             if (cacheLoadTaskService != null) {
+                // if main cache reload thread encountered an exception,
+                // it interrupts underlying InputSplitCacheLoadTasks threads
                 cacheLoadTaskService.shutdownNow();

Review Comment:
   Out of curiosity: Was it a deliberate decision to rely on the commonForkPool 
(see 
[ReloadTriggerContext:52](https://github.com/apache/flink/blob/10a6f41fa12284f15af55c359c0c0954800f02de/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/table/lookup/fullcache/ReloadTriggerContext.java#L52)
 if `numSplits == 1` and create a dedicated thread pool if we have more than 1 
split? I'm just wondering whether aligning both would be an option by relying 
on `CompletableFuture.runAsync` here as well and, that way, avoiding to wait 
for the shutdown.
   
   I couldn't find any reasoning to do that other than readability/less code.



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