[ 
https://issues.apache.org/jira/browse/HADOOP-17835?focusedWorklogId=633941&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-633941
 ]

ASF GitHub Bot logged work on HADOOP-17835:
-------------------------------------------

                Author: ASF GitHub Bot
            Created on: 05/Aug/21 02:53
            Start Date: 05/Aug/21 02:53
    Worklog Time Spent: 10m 
      Work Description: aajisaka commented on a change in pull request #3266:
URL: https://github.com/apache/hadoop/pull/3266#discussion_r683090545



##########
File path: 
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/ZKDelegationTokenSecretManager.java
##########
@@ -363,105 +357,122 @@ public void startThreads() throws IOException {
       throw new RuntimeException("Could not create ZK paths");
     }
     try {
-      keyCache = new PathChildrenCache(zkClient, ZK_DTSM_MASTER_KEY_ROOT, 
true);
+      keyCache = CuratorCache.bridgeBuilder(zkClient, ZK_DTSM_MASTER_KEY_ROOT)
+          .build();
       if (keyCache != null) {
-        keyCache.start(StartMode.BUILD_INITIAL_CACHE);
-        keyCache.getListenable().addListener(new PathChildrenCacheListener() {
-          @Override
-          public void childEvent(CuratorFramework client,
-              PathChildrenCacheEvent event)
-              throws Exception {
-            switch (event.getType()) {
-            case CHILD_ADDED:
-              processKeyAddOrUpdate(event.getData().getData());
-              break;
-            case CHILD_UPDATED:
-              processKeyAddOrUpdate(event.getData().getData());
-              break;
-            case CHILD_REMOVED:
-              processKeyRemoved(event.getData().getPath());
-              break;
-            default:
-              break;
-            }
-          }
-        }, listenerThreadPool);
+        CuratorCacheListener keyCacheListener = CuratorCacheListener.builder()
+            .forCreates(childData -> {
+              try {
+                processKeyAddOrUpdate(childData.getData());
+              } catch (IOException e) {
+                LOG.error("Error while processing Curator keyCacheListener "
+                    + "NODE_CREATED event");
+                throw new UncheckedIOException(e);
+              }
+            })
+            .forChanges((oldNode, node) -> {
+              try {
+                processKeyAddOrUpdate(node.getData());
+              } catch (IOException e) {
+                LOG.error("Error while processing Curator keyCacheListener "
+                    + "NODE_CHANGED event");
+                throw new UncheckedIOException(e);
+              }
+            })
+            .forDeletes(childData -> processKeyRemoved(childData.getPath()))
+            .build();
+        keyCache.listenable().addListener(keyCacheListener);
+        keyCache.start();
         loadFromZKCache(false);
       }
     } catch (Exception e) {
-      throw new IOException("Could not start PathChildrenCache for keys", e);
+      throw new IOException("Could not start Curator keyCacheListener for 
keys",
+          e);
     }
     if (isTokenWatcherEnabled) {
       LOG.info("TokenCache is enabled");
       try {
-        tokenCache = new PathChildrenCache(zkClient, ZK_DTSM_TOKENS_ROOT, 
true);
+        tokenCache = CuratorCache.bridgeBuilder(zkClient, ZK_DTSM_TOKENS_ROOT)
+            .build();
         if (tokenCache != null) {
-          tokenCache.start(StartMode.BUILD_INITIAL_CACHE);
-          tokenCache.getListenable().addListener(new 
PathChildrenCacheListener() {
-
-            @Override
-            public void childEvent(CuratorFramework client,
-                                   PathChildrenCacheEvent event) throws 
Exception {
-              switch (event.getType()) {
-                case CHILD_ADDED:
-                  processTokenAddOrUpdate(event.getData().getData());
-                  break;
-                case CHILD_UPDATED:
-                  processTokenAddOrUpdate(event.getData().getData());
-                  break;
-                case CHILD_REMOVED:
-                  processTokenRemoved(event.getData());
-                  break;
-                default:
-                  break;
-              }
-            }
-          }, listenerThreadPool);
+          CuratorCacheListener tokenCacheListener = 
CuratorCacheListener.builder()
+              .forCreates(childData -> {
+                try {
+                  processTokenAddOrUpdate(childData.getData());
+                } catch (IOException e) {
+                  LOG.error("Error while processing Curator tokenCacheListener 
"
+                      + "NODE_CREATED event");
+                  throw new UncheckedIOException(e);
+                }
+              })
+              .forChanges((oldNode, node) -> {
+                try {
+                  processTokenAddOrUpdate(node.getData());
+                } catch (IOException e) {
+                  LOG.error("Error while processing Curator tokenCacheListener 
"
+                      + "NODE_CHANGED event");
+                  throw new UncheckedIOException(e);
+                }
+              })
+              .forDeletes(childData -> {
+                try {
+                  processTokenRemoved(childData);
+                } catch (IOException e) {
+                  LOG.error("Error while processing Curator tokenCacheListener 
"
+                      + "NODE_DELETED event");
+                  throw new UncheckedIOException(e);
+                }
+              })
+              .build();
+          tokenCache.listenable().addListener(tokenCacheListener);
+          tokenCache.start();
           loadFromZKCache(true);
         }
       } catch (Exception e) {
-        throw new IOException("Could not start PathChildrenCache for tokens", 
e);
+        throw new IOException(
+            "Could not start Curator tokenCacheListener for tokens", e);
       }
     }
     super.startThreads();
   }
 
   /**
-   * Load the PathChildrenCache into the in-memory map. Possible caches to be
+   * Load the CuratorCache into the in-memory map. Possible caches to be
    * loaded are keyCache and tokenCache.
    *
    * @param isTokenCache true if loading tokenCache, false if loading keyCache.
    */
   private void loadFromZKCache(final boolean isTokenCache) {
     final String cacheName = isTokenCache ? "token" : "key";
     LOG.info("Starting to load {} cache.", cacheName);
-    final List<ChildData> children;
+    final Stream<ChildData> children;
     if (isTokenCache) {
-      children = tokenCache.getCurrentData();
+      children = tokenCache.stream();
     } else {
-      children = keyCache.getCurrentData();
+      children = keyCache.stream();
     }
 
-    int count = 0;
-    for (ChildData child : children) {
+    final AtomicInteger count = new AtomicInteger(0);
+    children.forEach(childData -> {

Review comment:
       I read the Curator's source code, and found that `children` is not 
parallel. Should we make it parallel by `children.parallel().forEach()` or not?




-- 
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: common-issues-unsubscr...@hadoop.apache.org

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


Issue Time Tracking
-------------------

    Worklog Id:     (was: 633941)
    Time Spent: 0.5h  (was: 20m)

> Use CuratorCache implementation instead of PathChildrenCache / TreeCache
> ------------------------------------------------------------------------
>
>                 Key: HADOOP-17835
>                 URL: https://issues.apache.org/jira/browse/HADOOP-17835
>             Project: Hadoop Common
>          Issue Type: Task
>            Reporter: Viraj Jasani
>            Assignee: Viraj Jasani
>            Priority: Major
>              Labels: pull-request-available
>          Time Spent: 0.5h
>  Remaining Estimate: 0h
>
> As we have moved to Curator 5.2.0 for Hadoop 3.4.0, we should start using new 
> CuratorCache service implementation in place of deprecated PathChildrenCache 
> and TreeCache usecases.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org

Reply via email to