mynameborat commented on a change in pull request #912: SEP-19 : Refactoring 
sideInputs from SamzaContainer to ContainerStorageManager
URL: https://github.com/apache/samza/pull/912#discussion_r258171369
 
 

 ##########
 File path: 
samza-core/src/main/scala/org/apache/samza/storage/ContainerStorageManager.java
 ##########
 @@ -406,14 +725,24 @@ public void stopStores() {
   }
 
   public void shutdown() {
-    this.taskRestoreManagers.forEach((taskInstance, taskRestoreManager) -> {
-        if (taskRestoreManager != null) {
-          LOG.debug("Shutting down task storage manager for taskName: {} ", 
taskInstance);
-          taskRestoreManager.stop();
-        } else {
-          LOG.debug("Skipping task storage manager shutdown for taskName: {}", 
taskInstance);
-        }
-      });
+    // stop all nonsideinputstores including persistent and non-persistent 
stores
+    this.containerModel.getTasks().forEach((taskName, taskModel) ->
+        getNonSideInputStores(taskName).forEach((storeName, store) -> 
store.stop())
+    );
+
+    // cancel all future sideInput flushes, and shutdown the executor
+    if (sideInputsFlushFuture != null) {
+      sideInputsFlushFuture.cancel(false);
+    }
+    sideInputsFlushExecutor.shutdown();
+    this.shutDownSideInputRead = true;
+
+    // stop all sideinput consumers and stores
+    if (sideInputSystemConsumers != null) {
+      this.sideInputSystemConsumers.stop();
+    }
+    
this.sideInputStorageManagers.values().stream().collect(Collectors.toSet()).
+        forEach(sideInputStorageManager -> sideInputStorageManager.stop());
 
 Review comment:
   You'd need to wait for `sideInputsFlushExcecutor` shutdown to complete 
before invoking a stop on the sideInputStorageManager to prevent any concurrent 
modification of the offset file / flushing a closed store.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

Reply via email to