[ https://issues.apache.org/jira/browse/HDDS-1406?focusedWorklogId=233149&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-233149 ]
ASF GitHub Bot logged work on HDDS-1406: ---------------------------------------- Author: ASF GitHub Bot Created on: 25/Apr/19 22:52 Start Date: 25/Apr/19 22:52 Worklog Time Spent: 10m Work Description: arp7 commented on pull request #714: HDDS-1406. Avoid usage of commonPool in RatisPipelineUtils. URL: https://github.com/apache/hadoop/pull/714#discussion_r278759939 ########## File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineUtils.java ########## @@ -146,19 +165,37 @@ private static void callRatisRpc(List<DatanodeDetails> datanodes, SecurityConfig(ozoneConf)); final TimeDuration requestTimeout = RatisHelper.getClientRequestTimeout(ozoneConf); - datanodes.parallelStream().forEach(d -> { - final RaftPeer p = RatisHelper.toRaftPeer(d); - try (RaftClient client = RatisHelper - .newRaftClient(SupportedRpcType.valueOfIgnoreCase(rpcType), p, - retryPolicy, maxOutstandingRequests, tlsConfig, requestTimeout)) { - rpc.accept(client, p); - } catch (IOException ioe) { - String errMsg = - "Failed invoke Ratis rpc " + rpc + " for " + d.getUuid(); - LOG.error(errMsg, ioe); - exceptions.add(new IOException(errMsg, ioe)); - } - }); + try { + POOL.submit(() -> { + datanodes.parallelStream().forEach(d -> { + final RaftPeer p = RatisHelper.toRaftPeer(d); + try (RaftClient client = RatisHelper + .newRaftClient(SupportedRpcType.valueOfIgnoreCase(rpcType), p, + retryPolicy, maxOutstandingRequests, tlsConfig, + requestTimeout)) { + rpc.accept(client, p); + } catch (IOException ioe) { + String errMsg = + "Failed invoke Ratis rpc " + rpc + " for " + d.getUuid(); + LOG.error(errMsg, ioe); + exceptions.add(new IOException(errMsg, ioe)); + } + }); + }).get(); + } catch (ExecutionException ex) { Review comment: You can simplify the code a bit by catching multiple exceptions in one clause. e.g. catch(ExecutionException|RejectedExecutionException e) ---------------------------------------------------------------- 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: 233149) Time Spent: 3h 40m (was: 3.5h) > 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: 3h 40m > 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