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

ASF GitHub Bot logged work on HDDS-1406:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 16/May/19 09:27
            Start Date: 16/May/19 09:27
    Worklog Time Spent: 10m 
      Work Description: lokeshj1703 commented on pull request #714: HDDS-1406. 
Avoid usage of commonPool in RatisPipelineUtils.
URL: https://github.com/apache/hadoop/pull/714#discussion_r284614769
 
 

 ##########
 File path: 
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineProvider.java
 ##########
 @@ -133,7 +173,86 @@ public Pipeline create(ReplicationFactor factor,
         .build();
   }
 
+
+  @Override
+  public void shutdown() {
+    forkJoinPool.shutdownNow();
+  }
+
   protected void initializePipeline(Pipeline pipeline) throws IOException {
-    RatisPipelineUtils.createPipeline(pipeline, conf);
+    createPipeline(pipeline);
+  }
+
+  /**
+   * Sends ratis command to create pipeline on all the datanodes.
+   *
+   * @param pipeline  - Pipeline to be created
+   * @throws IOException if creation fails
+   */
+  public void createPipeline(Pipeline pipeline)
+      throws IOException {
+    final RaftGroup group = RatisHelper.newRaftGroup(pipeline);
+    LOG.debug("creating pipeline:{} with {}", pipeline.getId(), group);
+    callRatisRpc(pipeline.getNodes(),
+        (raftClient, peer) -> {
+          RaftClientReply reply = raftClient.groupAdd(group, peer.getId());
+          if (reply == null || !reply.isSuccess()) {
+            String msg = "Pipeline initialization failed for pipeline:"
+                + pipeline.getId() + " node:" + peer.getId();
+            LOG.error(msg);
+            throw new IOException(msg);
+          }
+        });
+  }
+
+  private void callRatisRpc(List<DatanodeDetails> datanodes,
+      CheckedBiConsumer< RaftClient, RaftPeer, IOException> rpc)
+      throws IOException {
+    if (datanodes.isEmpty()) {
+      return;
+    }
+
+    final String rpcType = conf
+        .get(ScmConfigKeys.DFS_CONTAINER_RATIS_RPC_TYPE_KEY,
+            ScmConfigKeys.DFS_CONTAINER_RATIS_RPC_TYPE_DEFAULT);
+    final RetryPolicy retryPolicy = RatisHelper.createRetryPolicy(conf);
+    final List< IOException > exceptions =
+        Collections.synchronizedList(new ArrayList<>());
+    final int maxOutstandingRequests =
+        HddsClientUtils.getMaxOutstandingRequests(conf);
+    final GrpcTlsConfig tlsConfig = RatisHelper.createTlsClientConfig(new
+        SecurityConfig(conf));
+    final TimeDuration requestTimeout =
+        RatisHelper.getClientRequestTimeout(conf);
+    try {
+      forkJoinPool.submit(() -> {
 
 Review comment:
   Can we please verify that none of the threads are waiting for the parallel 
stream call to finish?
 
----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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

    Worklog Id:     (was: 243219)
    Time Spent: 8h 20m  (was: 8h 10m)

> Avoid usage of commonPool in RatisPipelineUtils
> -----------------------------------------------
>
>                 Key: HDDS-1406
>                 URL: https://issues.apache.org/jira/browse/HDDS-1406
>             Project: Hadoop Distributed Data Store
>          Issue Type: Bug
>            Reporter: Bharat Viswanadham
>            Assignee: Bharat Viswanadham
>            Priority: Major
>              Labels: pull-request-available
>          Time Spent: 8h 20m
>  Remaining Estimate: 0h
>
> We use parallelStream in during createPipline, this internally uses 
> commonPool. Use Our own ForkJoinPool with parallelisim set with number of 
> processors.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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

Reply via email to