[ https://issues.apache.org/jira/browse/HDDS-2034?focusedWorklogId=317781&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-317781 ]
ASF GitHub Bot logged work on HDDS-2034: ---------------------------------------- Author: ASF GitHub Bot Created on: 24/Sep/19 19:36 Start Date: 24/Sep/19 19:36 Worklog Time Spent: 10m Work Description: anuengineer commented on pull request #1469: HDDS-2034. Async RATIS pipeline creation and destroy through heartbea… URL: https://github.com/apache/hadoop/pull/1469#discussion_r327797006 ########## File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineProvider.java ########## @@ -155,12 +147,25 @@ public Pipeline create(ReplicationFactor factor) throws IOException { Pipeline pipeline = Pipeline.newBuilder() .setId(PipelineID.randomId()) - .setState(PipelineState.OPEN) + .setState(PipelineState.ALLOCATED) .setType(ReplicationType.RATIS) .setFactor(factor) .setNodes(dns) .build(); - initializePipeline(pipeline); + + // Send command to datanode to create pipeline + final CreatePipelineCommand createCommand = + new CreatePipelineCommand(pipeline.getId(), pipeline.getType(), + factor, dns); + + dns.stream().forEach(node -> { Review comment: Just a thought, probably need @nandakumar131 to weigh in too. 1. Since we are using the Ratis client in the DN's we might be able to get away with posting this command only to any one of the data nodes. 2. If we are posting this command to all DNs, we must be prepared to handle the fact that 2 out of 3 creates in the DN will fail. 3. You current code is correct, since the createPipeline inside the DN -- handles the fact that createpipeline can fail. I am just flagging that information here, so that next code reviewer can also reflect upon this. No work or change needed. ---------------------------------------------------------------- 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: 317781) Time Spent: 5.5h (was: 5h 20m) > Async RATIS pipeline creation and destroy through heartbeat commands > -------------------------------------------------------------------- > > Key: HDDS-2034 > URL: https://issues.apache.org/jira/browse/HDDS-2034 > Project: Hadoop Distributed Data Store > Issue Type: Sub-task > Reporter: Sammi Chen > Assignee: Sammi Chen > Priority: Major > Labels: pull-request-available > Time Spent: 5.5h > Remaining Estimate: 0h > > Currently, pipeline creation and destroy are synchronous operations. SCM > directly connect to each datanode of the pipeline through gRPC channel to > create the pipeline to destroy the pipeline. > This task is to remove the gRPC channel, send pipeline creation and destroy > action through heartbeat command to each datanode. -- This message was sent by Atlassian Jira (v8.3.4#803005) --------------------------------------------------------------------- To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org