You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ozone.apache.org by GitBox <gi...@apache.org> on 2020/01/21 05:14:14 UTC

[GitHub] [hadoop-ozone] avijayanhwx opened a new pull request #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.

avijayanhwx opened a new pull request #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.
URL: https://github.com/apache/hadoop-ozone/pull/466
 
 
   ## What changes were proposed in this pull request?
   
   - Implement pipeline bootstrap from SCM in Recon.
   - Handle CREATE pipeline use case in Recon (Recon sees a new pipeline ID in Datanode heartbeat)
   - Stub out pipeline provider in Recon such that it can NEVER create a Ratis pipeline.
   - Added SCM RPC endpoint to get a single Pipeline info.
   
   ## What is the link to the Apache JIRA
   https://issues.apache.org/jira/browse/HDDS-2869
   
   ## How was this patch tested?
   Manually tested on a 3 node docker cluster.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] avijayanhwx commented on a change in pull request #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.

Posted by GitBox <gi...@apache.org>.
avijayanhwx commented on a change in pull request #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.
URL: https://github.com/apache/hadoop-ozone/pull/466#discussion_r372093199
 
 

 ##########
 File path: hadoop-ozone/dist/src/main/compose/ozone/docker-compose.yaml
 ##########
 @@ -67,6 +67,5 @@ services:
     ports:
       - 9888:9888
     environment:
-      WAITFOR: om:9874
 
 Review comment:
   I just stumbled upon this while testing. We no longer need this. Recon can start up anytime and can learn OM and SCM state.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] arp7 commented on a change in pull request #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.

Posted by GitBox <gi...@apache.org>.
arp7 commented on a change in pull request #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.
URL: https://github.com/apache/hadoop-ozone/pull/466#discussion_r372092272
 
 

 ##########
 File path: hadoop-ozone/dist/src/main/compose/ozone/docker-compose.yaml
 ##########
 @@ -67,6 +67,5 @@ services:
     ports:
       - 9888:9888
     environment:
-      WAITFOR: om:9874
 
 Review comment:
   Why is this removed?

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] avijayanhwx commented on a change in pull request #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.

Posted by GitBox <gi...@apache.org>.
avijayanhwx commented on a change in pull request #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.
URL: https://github.com/apache/hadoop-ozone/pull/466#discussion_r371553651
 
 

 ##########
 File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineFactory.java
 ##########
 @@ -35,7 +35,7 @@
 /**
  * Creates pipeline based on replication type.
  */
-public final class PipelineFactory {
+public class PipelineFactory {
 
 Review comment:
   This is extended in org.apache.hadoop.ozone.recon.scm.ReconPipelineFactory. 

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] swagle commented on a change in pull request #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.

Posted by GitBox <gi...@apache.org>.
swagle commented on a change in pull request #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.
URL: https://github.com/apache/hadoop-ozone/pull/466#discussion_r371546781
 
 

 ##########
 File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/SCMPipelineManager.java
 ##########
 @@ -200,6 +210,16 @@ public Pipeline getPipeline(PipelineID pipelineID)
     }
   }
 
+  @Override
+  public boolean containsPipeline(PipelineID pipelineID) {
 
 Review comment:
   Instead of contains, we could have made the get return an Optiona<Pipeline>

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] swagle commented on a change in pull request #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.

Posted by GitBox <gi...@apache.org>.
swagle commented on a change in pull request #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.
URL: https://github.com/apache/hadoop-ozone/pull/466#discussion_r371548604
 
 

 ##########
 File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconPipelineFactory.java
 ##########
 @@ -0,0 +1,72 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.scm;
+
+import java.util.List;
+
+import org.apache.commons.collections.map.DefaultedMap;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineFactory;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineProvider;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Class to stub out SCM's pipeline providers. This makes sure Recon can
+ * never be on the pipeline CREATE or CLOSE path.
+ */
+public class ReconPipelineFactory extends PipelineFactory {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ReconPipelineFactory.class);
+
+  ReconPipelineFactory() {
+    ReconMockPipelineProvider reconMockPipelineProvider =
+        new ReconMockPipelineProvider();
+    setProviders(new DefaultedMap(reconMockPipelineProvider));
+  }
+
+  static class ReconMockPipelineProvider implements PipelineProvider {
 
 Review comment:
   I would avoid calling this a Mock since unit tests mocks have a different meaning as they return stubs data structures. 

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] arp7 commented on a change in pull request #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.

Posted by GitBox <gi...@apache.org>.
arp7 commented on a change in pull request #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.
URL: https://github.com/apache/hadoop-ozone/pull/466#discussion_r372091718
 
 

 ##########
 File path: hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/SCMAction.java
 ##########
 @@ -32,6 +32,7 @@
   GET_CONTAINER_WITH_PIPELINE,
   LIST_CONTAINER,
   LIST_PIPELINE,
+  GET_PIPELINE,
 
 Review comment:
   Usually we add new fields at the end of enums.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] avijayanhwx commented on a change in pull request #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.

Posted by GitBox <gi...@apache.org>.
avijayanhwx commented on a change in pull request #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.
URL: https://github.com/apache/hadoop-ozone/pull/466#discussion_r371872738
 
 

 ##########
 File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineReportHandler.java
 ##########
 @@ -120,4 +116,24 @@ private void processPipelineReport(PipelineReport report, DatanodeDetails dn,
       }
     }
   }
+
+
+  protected void reportDatanode(Pipeline pipeline, DatanodeDetails dn)
 
 Review comment:
   Done

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] swagle commented on a change in pull request #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.

Posted by GitBox <gi...@apache.org>.
swagle commented on a change in pull request #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.
URL: https://github.com/apache/hadoop-ozone/pull/466#discussion_r371547638
 
 

 ##########
 File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/SCMPipelineManager.java
 ##########
 @@ -523,4 +543,21 @@ protected File getPipelineDBPath(Configuration configuration) {
     File metaDir = ServerUtils.getScmDbDir(configuration);
     return new File(metaDir, SCM_PIPELINE_DB);
   }
+
+  protected ReadWriteLock getLock() {
 
 Review comment:
   Instead of a get on a lock object, it would make sense to make the reference accessible.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] swagle commented on a change in pull request #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.

Posted by GitBox <gi...@apache.org>.
swagle commented on a change in pull request #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.
URL: https://github.com/apache/hadoop-ozone/pull/466#discussion_r371979925
 
 

 ##########
 File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/SCMPipelineManager.java
 ##########
 @@ -523,4 +543,21 @@ protected File getPipelineDBPath(Configuration configuration) {
     File metaDir = ServerUtils.getScmDbDir(configuration);
     return new File(metaDir, SCM_PIPELINE_DB);
   }
+
+  protected ReadWriteLock getLock() {
 
 Review comment:
   I see, ok let's ignore this comment.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] swagle commented on a change in pull request #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.

Posted by GitBox <gi...@apache.org>.
swagle commented on a change in pull request #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.
URL: https://github.com/apache/hadoop-ozone/pull/466#discussion_r371547977
 
 

 ##########
 File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java
 ##########
 @@ -408,6 +408,15 @@ public Pipeline createReplicationPipeline(HddsProtos.ReplicationType type,
     return scm.getPipelineManager().getPipelines();
   }
 
+  @Override
+  public Pipeline getPipeline(HddsProtos.PipelineID pipelineID)
+      throws IOException {
+    AUDIT.logReadSuccess(
 
 Review comment:
   This might not be appropriate for Audit since it is not client access, we should log it in debug mode IMO.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] swagle commented on a change in pull request #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.

Posted by GitBox <gi...@apache.org>.
swagle commented on a change in pull request #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.
URL: https://github.com/apache/hadoop-ozone/pull/466#discussion_r371548604
 
 

 ##########
 File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconPipelineFactory.java
 ##########
 @@ -0,0 +1,72 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.scm;
+
+import java.util.List;
+
+import org.apache.commons.collections.map.DefaultedMap;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineFactory;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineProvider;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Class to stub out SCM's pipeline providers. This makes sure Recon can
+ * never be on the pipeline CREATE or CLOSE path.
+ */
+public class ReconPipelineFactory extends PipelineFactory {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ReconPipelineFactory.class);
+
+  ReconPipelineFactory() {
+    ReconMockPipelineProvider reconMockPipelineProvider =
+        new ReconMockPipelineProvider();
+    setProviders(new DefaultedMap(reconMockPipelineProvider));
+  }
+
+  static class ReconMockPipelineProvider implements PipelineProvider {
 
 Review comment:
   I would avoid calling this a Mock since unit tests mocks have a different meaning as they return stubbed data structures. 

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] adoroszlai merged pull request #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.

Posted by GitBox <gi...@apache.org>.
adoroszlai merged pull request #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.
URL: https://github.com/apache/hadoop-ozone/pull/466
 
 
   

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] swagle commented on a change in pull request #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.

Posted by GitBox <gi...@apache.org>.
swagle commented on a change in pull request #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.
URL: https://github.com/apache/hadoop-ozone/pull/466#discussion_r371545481
 
 

 ##########
 File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineReportHandler.java
 ##########
 @@ -120,4 +116,24 @@ private void processPipelineReport(PipelineReport report, DatanodeDetails dn,
       }
     }
   }
+
+
+  protected void reportDatanode(Pipeline pipeline, DatanodeDetails dn)
 
 Review comment:
   For cleaner semantics, this can be renamed to setReportedDatanode.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] adoroszlai commented on issue #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.

Posted by GitBox <gi...@apache.org>.
adoroszlai commented on issue #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.
URL: https://github.com/apache/hadoop-ozone/pull/466#issuecomment-579865727
 
 
   Thanks @avijayanhwx for the contribution, and @arp7 and @swagle for the review.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] adoroszlai commented on issue #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.

Posted by GitBox <gi...@apache.org>.
adoroszlai commented on issue #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.
URL: https://github.com/apache/hadoop-ozone/pull/466#issuecomment-579865095
 
 
   > I have not wired up the DN-> Recon path by default
   
   Thanks, that explains it.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] avijayanhwx commented on issue #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.

Posted by GitBox <gi...@apache.org>.
avijayanhwx commented on issue #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.
URL: https://github.com/apache/hadoop-ozone/pull/466#issuecomment-579863624
 
 
   > Thanks @avijayanhwx for implementing this. I tried it locally on a docker-compose cluster and found that Recon does not get new pipeline information.
   > 
   > 1. Since now the recon container does not wait with startup, it received 0 pipelines initially:
   >    ```
   >    recon_1     | 2020-01-29 08:41:05,263 [main] INFO scm.ReconStorageContainerManagerFacade: Obtained 0 pipelines from SCM.
   >    recon_1     | 2020-01-29 08:41:05,264 [main] INFO scm.ReconPipelineManager: Recon has 0 pipelines in house.
   >    ```
   >    
   >    
   >    That would be OK, but when the pipelines were created a bit later, they did not show up in Recon.  (I also checked with `WAITFOR` restored, 3+1 initial pipelines were received in that case, but I feel it's a bit timing-dependent.)
   > 2. Closed the initial 3-node pipeline via `scmcli`, which eventually triggered creation of new one in SCM, but Recon never noticed.
   > 
   > Can you please check?
   
   @adoroszlai I have not wired up the DN-> Recon path by default since that may cause some unintended test failures before finishing the whole "Recon as a Passive SCM" feature. In my local, I test with the following configs added. 
   
   > OZONE-SITE.XML_ozone.recon.datanode.address=recon:9891
   > OZONE-SITE.XML_ozone.recon.address=recon:9891
   > OZONE-SITE.XML_ozone.recon.datanode.bind.host=recon 
   
   That will lead to pipelines being picked up by Recon.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] swagle commented on a change in pull request #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.

Posted by GitBox <gi...@apache.org>.
swagle commented on a change in pull request #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.
URL: https://github.com/apache/hadoop-ozone/pull/466#discussion_r371550154
 
 

 ##########
 File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/StorageContainerServiceProviderImpl.java
 ##########
 @@ -0,0 +1,94 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.spi.impl;
+
+import static org.apache.hadoop.hdds.scm.client.ContainerOperationClient.newContainerRpcClient;
+
+import java.io.IOException;
+import java.util.List;
+
+import javax.inject.Inject;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocol;
+import org.apache.hadoop.ozone.recon.spi.StorageContainerServiceProvider;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Implementation for StorageContainerServiceProvider that talks with actual
+ * cluster SCM.
+ */
+public class StorageContainerServiceProviderImpl
+    implements StorageContainerServiceProvider {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(StorageContainerServiceProviderImpl.class);
+
+  private OzoneConfiguration configuration;
+  private StorageContainerLocationProtocol scmClient;
+  private boolean isInitialized = false;
 
 Review comment:
   This should be declared volatile.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] swagle commented on a change in pull request #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.

Posted by GitBox <gi...@apache.org>.
swagle commented on a change in pull request #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.
URL: https://github.com/apache/hadoop-ozone/pull/466#discussion_r371979655
 
 

 ##########
 File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/SCMPipelineManager.java
 ##########
 @@ -200,6 +210,16 @@ public Pipeline getPipeline(PipelineID pipelineID)
     }
   }
 
+  @Override
+  public boolean containsPipeline(PipelineID pipelineID) {
 
 Review comment:
   It's a minor nit, I am ok with it asis. More of a preference thing, returning Optional already sets the contract and reduces code dup.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] swagle commented on a change in pull request #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.

Posted by GitBox <gi...@apache.org>.
swagle commented on a change in pull request #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.
URL: https://github.com/apache/hadoop-ozone/pull/466#discussion_r371546781
 
 

 ##########
 File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/SCMPipelineManager.java
 ##########
 @@ -200,6 +210,16 @@ public Pipeline getPipeline(PipelineID pipelineID)
     }
   }
 
+  @Override
+  public boolean containsPipeline(PipelineID pipelineID) {
 
 Review comment:
   Instead of contains, we could have made the get return an Optional<Pipeline>

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] avijayanhwx commented on a change in pull request #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.

Posted by GitBox <gi...@apache.org>.
avijayanhwx commented on a change in pull request #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.
URL: https://github.com/apache/hadoop-ozone/pull/466#discussion_r371874740
 
 

 ##########
 File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/SCMPipelineManager.java
 ##########
 @@ -200,6 +210,16 @@ public Pipeline getPipeline(PipelineID pipelineID)
     }
   }
 
+  @Override
+  public boolean containsPipeline(PipelineID pipelineID) {
 
 Review comment:
   Since the API is "contains", I wanted to keep it true/false. Can I address this in the next patch?

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] swagle commented on a change in pull request #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.

Posted by GitBox <gi...@apache.org>.
swagle commented on a change in pull request #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.
URL: https://github.com/apache/hadoop-ozone/pull/466#discussion_r371550351
 
 

 ##########
 File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/StorageContainerServiceProviderImpl.java
 ##########
 @@ -0,0 +1,94 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.spi.impl;
+
+import static org.apache.hadoop.hdds.scm.client.ContainerOperationClient.newContainerRpcClient;
+
+import java.io.IOException;
+import java.util.List;
+
+import javax.inject.Inject;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocol;
+import org.apache.hadoop.ozone.recon.spi.StorageContainerServiceProvider;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Implementation for StorageContainerServiceProvider that talks with actual
+ * cluster SCM.
+ */
+public class StorageContainerServiceProviderImpl
+    implements StorageContainerServiceProvider {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(StorageContainerServiceProviderImpl.class);
+
+  private OzoneConfiguration configuration;
+  private StorageContainerLocationProtocol scmClient;
+  private boolean isInitialized = false;
+
+  @Inject
+  public StorageContainerServiceProviderImpl(OzoneConfiguration configuration,
+      StorageContainerLocationProtocol scmClient) {
+    this.configuration = configuration;
+    this.scmClient = scmClient;
+    if (this.scmClient != null) {
+      isInitialized = true;
+    }
+  }
+
+  private void initialize() {
+    try {
+      this.scmClient = newContainerRpcClient(configuration);
+    } catch (IOException ioEx) {
+      LOG.error("Exception encountered while creating SCM client.", ioEx);
+    }
+    isInitialized = true;
+  }
+
+  @Override
+  public List<Pipeline> getPipelines() throws IOException {
+    if (!isInitialized) {
 
 Review comment:
   Same here, and next check becomes redundant. 

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] swagle commented on a change in pull request #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.

Posted by GitBox <gi...@apache.org>.
swagle commented on a change in pull request #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.
URL: https://github.com/apache/hadoop-ozone/pull/466#discussion_r371544668
 
 

 ##########
 File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineFactory.java
 ##########
 @@ -35,7 +35,7 @@
 /**
  * Creates pipeline based on replication type.
  */
-public final class PipelineFactory {
+public class PipelineFactory {
 
 Review comment:
   The access can be made package-private?

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] avijayanhwx commented on a change in pull request #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.

Posted by GitBox <gi...@apache.org>.
avijayanhwx commented on a change in pull request #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.
URL: https://github.com/apache/hadoop-ozone/pull/466#discussion_r372093444
 
 

 ##########
 File path: hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/SCMAction.java
 ##########
 @@ -32,6 +32,7 @@
   GET_CONTAINER_WITH_PIPELINE,
   LIST_CONTAINER,
   LIST_PIPELINE,
+  GET_PIPELINE,
 
 Review comment:
   Sure, I can move it to the end.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] avijayanhwx commented on a change in pull request #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.

Posted by GitBox <gi...@apache.org>.
avijayanhwx commented on a change in pull request #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.
URL: https://github.com/apache/hadoop-ozone/pull/466#discussion_r371862609
 
 

 ##########
 File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/SCMPipelineManager.java
 ##########
 @@ -523,4 +543,21 @@ protected File getPipelineDBPath(Configuration configuration) {
     File metaDir = ServerUtils.getScmDbDir(configuration);
     return new File(metaDir, SCM_PIPELINE_DB);
   }
+
+  protected ReadWriteLock getLock() {
 
 Review comment:
   That will lead to findbug violation. Do you see any issues using a getter?

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] avijayanhwx commented on a change in pull request #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.

Posted by GitBox <gi...@apache.org>.
avijayanhwx commented on a change in pull request #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.
URL: https://github.com/apache/hadoop-ozone/pull/466#discussion_r371873124
 
 

 ##########
 File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/StorageContainerServiceProviderImpl.java
 ##########
 @@ -0,0 +1,94 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.spi.impl;
+
+import static org.apache.hadoop.hdds.scm.client.ContainerOperationClient.newContainerRpcClient;
+
+import java.io.IOException;
+import java.util.List;
+
+import javax.inject.Inject;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocol;
+import org.apache.hadoop.ozone.recon.spi.StorageContainerServiceProvider;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Implementation for StorageContainerServiceProvider that talks with actual
+ * cluster SCM.
+ */
+public class StorageContainerServiceProviderImpl
+    implements StorageContainerServiceProvider {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(StorageContainerServiceProviderImpl.class);
+
+  private OzoneConfiguration configuration;
+  private StorageContainerLocationProtocol scmClient;
+  private boolean isInitialized = false;
 
 Review comment:
   Done

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] swagle commented on a change in pull request #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.

Posted by GitBox <gi...@apache.org>.
swagle commented on a change in pull request #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.
URL: https://github.com/apache/hadoop-ozone/pull/466#discussion_r371548315
 
 

 ##########
 File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconPipelineFactory.java
 ##########
 @@ -0,0 +1,72 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.scm;
+
+import java.util.List;
+
+import org.apache.commons.collections.map.DefaultedMap;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineFactory;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineProvider;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Class to stub out SCM's pipeline providers. This makes sure Recon can
+ * never be on the pipeline CREATE or CLOSE path.
+ */
+public class ReconPipelineFactory extends PipelineFactory {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ReconPipelineFactory.class);
+
+  ReconPipelineFactory() {
+    ReconMockPipelineProvider reconMockPipelineProvider =
+        new ReconMockPipelineProvider();
+    setProviders(new DefaultedMap(reconMockPipelineProvider));
+  }
+
+  static class ReconMockPipelineProvider implements PipelineProvider {
+
+    @Override
+    public Pipeline create(HddsProtos.ReplicationFactor factor){
+      LOG.warn("Trying to create pipeline in Recon, which is prohibited!");
+      return null;
 
 Review comment:
   Should we consider throwing a RuntimeException instead?

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] avijayanhwx commented on a change in pull request #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.

Posted by GitBox <gi...@apache.org>.
avijayanhwx commented on a change in pull request #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.
URL: https://github.com/apache/hadoop-ozone/pull/466#discussion_r371875291
 
 

 ##########
 File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java
 ##########
 @@ -408,6 +408,15 @@ public Pipeline createReplicationPipeline(HddsProtos.ReplicationType type,
     return scm.getPipelineManager().getPipelines();
   }
 
+  @Override
+  public Pipeline getPipeline(HddsProtos.PipelineID pipelineID)
+      throws IOException {
+    AUDIT.logReadSuccess(
 
 Review comment:
   Done

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] avijayanhwx commented on issue #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.

Posted by GitBox <gi...@apache.org>.
avijayanhwx commented on issue #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.
URL: https://github.com/apache/hadoop-ozone/pull/466#issuecomment-579919000
 
 
   > Thanks @adoroszlai for actually trying it out :-) 👍
   
   +1. Thanks @adoroszlai 

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] swagle commented on a change in pull request #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.

Posted by GitBox <gi...@apache.org>.
swagle commented on a change in pull request #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.
URL: https://github.com/apache/hadoop-ozone/pull/466#discussion_r371546781
 
 

 ##########
 File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/SCMPipelineManager.java
 ##########
 @@ -200,6 +210,16 @@ public Pipeline getPipeline(PipelineID pipelineID)
     }
   }
 
+  @Override
+  public boolean containsPipeline(PipelineID pipelineID) {
 
 Review comment:
   Instead of contains, we could have made the get return an Optional

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] avijayanhwx commented on a change in pull request #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.

Posted by GitBox <gi...@apache.org>.
avijayanhwx commented on a change in pull request #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.
URL: https://github.com/apache/hadoop-ozone/pull/466#discussion_r371862127
 
 

 ##########
 File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconPipelineFactory.java
 ##########
 @@ -0,0 +1,72 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.scm;
+
+import java.util.List;
+
+import org.apache.commons.collections.map.DefaultedMap;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineFactory;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineProvider;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Class to stub out SCM's pipeline providers. This makes sure Recon can
+ * never be on the pipeline CREATE or CLOSE path.
+ */
+public class ReconPipelineFactory extends PipelineFactory {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ReconPipelineFactory.class);
+
+  ReconPipelineFactory() {
+    ReconMockPipelineProvider reconMockPipelineProvider =
+        new ReconMockPipelineProvider();
+    setProviders(new DefaultedMap(reconMockPipelineProvider));
+  }
+
+  static class ReconMockPipelineProvider implements PipelineProvider {
+
+    @Override
+    public Pipeline create(HddsProtos.ReplicationFactor factor){
+      LOG.warn("Trying to create pipeline in Recon, which is prohibited!");
+      return null;
 
 Review comment:
   We don't need to do this since this class protects us against any unintended actions from Recon. 

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] swagle commented on issue #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.

Posted by GitBox <gi...@apache.org>.
swagle commented on issue #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.
URL: https://github.com/apache/hadoop-ozone/pull/466#issuecomment-579908070
 
 
   Thanks @adoroszlai for actually trying it out :-) 👍 

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] swagle commented on a change in pull request #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.

Posted by GitBox <gi...@apache.org>.
swagle commented on a change in pull request #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.
URL: https://github.com/apache/hadoop-ozone/pull/466#discussion_r371546781
 
 

 ##########
 File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/SCMPipelineManager.java
 ##########
 @@ -200,6 +210,16 @@ public Pipeline getPipeline(PipelineID pipelineID)
     }
   }
 
+  @Override
+  public boolean containsPipeline(PipelineID pipelineID) {
 
 Review comment:
   Instead of contains, we could have made the get return an "Optional<Pipeline>"

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] swagle commented on a change in pull request #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.

Posted by GitBox <gi...@apache.org>.
swagle commented on a change in pull request #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.
URL: https://github.com/apache/hadoop-ozone/pull/466#discussion_r371550058
 
 

 ##########
 File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/StorageContainerServiceProviderImpl.java
 ##########
 @@ -0,0 +1,94 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.recon.spi.impl;
+
+import static org.apache.hadoop.hdds.scm.client.ContainerOperationClient.newContainerRpcClient;
+
+import java.io.IOException;
+import java.util.List;
+
+import javax.inject.Inject;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocol;
+import org.apache.hadoop.ozone.recon.spi.StorageContainerServiceProvider;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Implementation for StorageContainerServiceProvider that talks with actual
+ * cluster SCM.
+ */
+public class StorageContainerServiceProviderImpl
+    implements StorageContainerServiceProvider {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(StorageContainerServiceProviderImpl.class);
+
+  private OzoneConfiguration configuration;
+  private StorageContainerLocationProtocol scmClient;
+  private boolean isInitialized = false;
+
+  @Inject
+  public StorageContainerServiceProviderImpl(OzoneConfiguration configuration,
+      StorageContainerLocationProtocol scmClient) {
+    this.configuration = configuration;
+    this.scmClient = scmClient;
+    if (this.scmClient != null) {
 
 Review comment:
   How about calling initialize from the constructor and make that idempotent using the flag.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] arp7 commented on a change in pull request #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.

Posted by GitBox <gi...@apache.org>.
arp7 commented on a change in pull request #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.
URL: https://github.com/apache/hadoop-ozone/pull/466#discussion_r372092348
 
 

 ##########
 File path: hadoop-ozone/dist/src/main/compose/ozonesecure/docker-compose.yaml
 ##########
 @@ -79,7 +79,6 @@ services:
     env_file:
       - ./docker-config
     environment:
-      WAITFOR: om:9874
 
 Review comment:
   Same.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org


[GitHub] [hadoop-ozone] arp7 commented on a change in pull request #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.

Posted by GitBox <gi...@apache.org>.
arp7 commented on a change in pull request #466: HDDS-2869. Handle pipeline bootstrap from SCM and create pipeline use case in Recon.
URL: https://github.com/apache/hadoop-ozone/pull/466#discussion_r372096299
 
 

 ##########
 File path: hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/SCMAction.java
 ##########
 @@ -32,6 +32,7 @@
   GET_CONTAINER_WITH_PIPELINE,
   LIST_CONTAINER,
   LIST_PIPELINE,
+  GET_PIPELINE,
 
 Review comment:
   I think it is okay for now as we don't have to maintain compatibility.

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-issues-help@hadoop.apache.org