[GitHub] [hadoop-ozone] timmylicheng commented on a change in pull request #1049: HDDS-3662 Decouple finalizeAndDestroyPipeline.

2020-07-09 Thread GitBox


timmylicheng commented on a change in pull request #1049:
URL: https://github.com/apache/hadoop-ozone/pull/1049#discussion_r452602085



##
File path: 
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManagerV2Impl.java
##
@@ -410,18 +399,29 @@ public void scrubPipeline(ReplicationType type, 
ReplicationFactor factor)
 ScmConfigKeys.OZONE_SCM_PIPELINE_ALLOCATED_TIMEOUT,
 ScmConfigKeys.OZONE_SCM_PIPELINE_ALLOCATED_TIMEOUT_DEFAULT,
 TimeUnit.MILLISECONDS);
-List needToSrubPipelines = stateManager.getPipelines(type, 
factor,
-Pipeline.PipelineState.ALLOCATED).stream()
-.filter(p -> currentTime.toEpochMilli() - p.getCreationTimestamp()
-.toEpochMilli() >= pipelineScrubTimeoutInMills)
-.collect(Collectors.toList());
-for (Pipeline p : needToSrubPipelines) {
-  LOG.info("Scrubbing pipeline: id: " + p.getId().toString() +
-  " since it stays at ALLOCATED stage for " +
-  Duration.between(currentTime, p.getCreationTimestamp()).toMinutes() +
-  " mins.");
-  finalizeAndDestroyPipeline(p, false);
+
+List candidates = stateManager.getPipelines(type, factor);
+
+for (Pipeline p : candidates) {
+  // scrub pipelines who stay ALLOCATED for too long.
+  if (p.getPipelineState() == Pipeline.PipelineState.ALLOCATED &&
+  (currentTime.toEpochMilli() - p.getCreationTimestamp()
+  .toEpochMilli() >= pipelineScrubTimeoutInMills)) {
+LOG.info("Scrubbing pipeline: id: " + p.getId().toString() +
+" since it stays at ALLOCATED stage for " +
+Duration.between(currentTime, p.getCreationTimestamp())
+.toMinutes() + " mins.");
+closePipeline(p, false);
+  }
+  // scrub pipelines who stay CLOSED for too long.
+  if (p.getPipelineState() == Pipeline.PipelineState.CLOSED) {

Review comment:
   I updated with the scheduler to schedule removing pipeline after closing 
contaienrs.





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



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



[GitHub] [hadoop-ozone] timmylicheng commented on a change in pull request #1049: HDDS-3662 Decouple finalizeAndDestroyPipeline.

2020-07-08 Thread GitBox


timmylicheng commented on a change in pull request #1049:
URL: https://github.com/apache/hadoop-ozone/pull/1049#discussion_r451482429



##
File path: 
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManagerV2Impl.java
##
@@ -410,18 +399,29 @@ public void scrubPipeline(ReplicationType type, 
ReplicationFactor factor)
 ScmConfigKeys.OZONE_SCM_PIPELINE_ALLOCATED_TIMEOUT,
 ScmConfigKeys.OZONE_SCM_PIPELINE_ALLOCATED_TIMEOUT_DEFAULT,
 TimeUnit.MILLISECONDS);
-List needToSrubPipelines = stateManager.getPipelines(type, 
factor,
-Pipeline.PipelineState.ALLOCATED).stream()
-.filter(p -> currentTime.toEpochMilli() - p.getCreationTimestamp()
-.toEpochMilli() >= pipelineScrubTimeoutInMills)
-.collect(Collectors.toList());
-for (Pipeline p : needToSrubPipelines) {
-  LOG.info("Scrubbing pipeline: id: " + p.getId().toString() +
-  " since it stays at ALLOCATED stage for " +
-  Duration.between(currentTime, p.getCreationTimestamp()).toMinutes() +
-  " mins.");
-  finalizeAndDestroyPipeline(p, false);
+
+List candidates = stateManager.getPipelines(type, factor);
+
+for (Pipeline p : candidates) {
+  // scrub pipelines who stay ALLOCATED for too long.
+  if (p.getPipelineState() == Pipeline.PipelineState.ALLOCATED &&
+  (currentTime.toEpochMilli() - p.getCreationTimestamp()
+  .toEpochMilli() >= pipelineScrubTimeoutInMills)) {
+LOG.info("Scrubbing pipeline: id: " + p.getId().toString() +
+" since it stays at ALLOCATED stage for " +
+Duration.between(currentTime, p.getCreationTimestamp())
+.toMinutes() + " mins.");
+closePipeline(p, false);
+  }
+  // scrub pipelines who stay CLOSED for too long.
+  if (p.getPipelineState() == Pipeline.PipelineState.CLOSED) {

Review comment:
   I feel like once pipeline is at CLOSED state, it can be removed right 
away since it has no chance getting back to OPEN. So I remove the 
OZONE_SCM_PIPELINE_DESTROY_TIMEOUT check here.





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



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



[GitHub] [hadoop-ozone] timmylicheng commented on a change in pull request #1049: HDDS-3662 Decouple finalizeAndDestroyPipeline.

2020-07-08 Thread GitBox


timmylicheng commented on a change in pull request #1049:
URL: https://github.com/apache/hadoop-ozone/pull/1049#discussion_r451333792



##
File path: 
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManagerV2Impl.java
##
@@ -310,94 +321,72 @@ public void openPipeline(PipelineID pipelineId) throws 
IOException {
   }
 
   /**
-   * Finalizes pipeline in the SCM. Removes pipeline and makes rpc call to
-   * destroy pipeline on the datanodes immediately or after timeout based on 
the
-   * value of onTimeout parameter.
-   *
-   * @param pipeline- Pipeline to be destroyed
-   * @param onTimeout   - if true pipeline is removed and destroyed on
-   *datanodes after timeout
-   * @throws IOException
-   */
-  @Override
-  public void finalizeAndDestroyPipeline(Pipeline pipeline, boolean onTimeout)
-  throws IOException {
-LOG.info("Destroying pipeline:{}", pipeline);
-finalizePipeline(pipeline.getId());
-if (onTimeout) {
-  long pipelineDestroyTimeoutInMillis =
-  
conf.getTimeDuration(ScmConfigKeys.OZONE_SCM_PIPELINE_DESTROY_TIMEOUT,
-  ScmConfigKeys.OZONE_SCM_PIPELINE_DESTROY_TIMEOUT_DEFAULT,
-  TimeUnit.MILLISECONDS);
-  scheduler.schedule(() -> destroyPipeline(pipeline),
-  pipelineDestroyTimeoutInMillis, TimeUnit.MILLISECONDS, LOG,
-  String.format("Destroy pipeline failed for pipeline:%s", pipeline));
-} else {
-  destroyPipeline(pipeline);
-}
-  }
-
-  /**
-   * Moves the pipeline to CLOSED state and sends close container command for
-   * all the containers in the pipeline.
+   * Removes the pipeline from the db and pipeline state map.
*
-   * @param pipelineId - ID of the pipeline to be moved to CLOSED state.
+   * @param pipeline - pipeline to be removed
* @throws IOException
*/
-  private void finalizePipeline(PipelineID pipelineId) throws IOException {
+  protected void removePipeline(Pipeline pipeline) throws IOException {
+pipelineFactory.close(pipeline.getType(), pipeline);
+PipelineID pipelineID = pipeline.getId();
 lock.writeLock().lock();
 try {
-  Pipeline pipeline = stateManager.getPipeline(pipelineId);
-  if (!pipeline.isClosed()) {
-stateManager.updatePipelineState(
-pipelineId.getProtobuf(), 
HddsProtos.PipelineState.PIPELINE_CLOSED);
-LOG.info("Pipeline {} moved to CLOSED state", pipeline);
-  }
-
-  // TODO fire events to datanodes for closing pipelines
-//  Set containerIDs = stateManager.getContainers(pipelineId);
-//  for (ContainerID containerID : containerIDs) {
-//eventPublisher.fireEvent(SCMEvents.CLOSE_CONTAINER, containerID);
-//  }
-  metrics.removePipelineMetrics(pipelineId);
+  stateManager.removePipeline(pipelineID.getProtobuf());
+  metrics.incNumPipelineDestroyed();
+} catch (IOException ex) {
+  metrics.incNumPipelineDestroyFailed();
+  throw ex;
 } finally {
   lock.writeLock().unlock();
 }
   }
 
   /**
-   * Removes pipeline from SCM. Sends ratis command to destroy pipeline on all
-   * the datanodes for ratis pipelines.
-   *
-   * @param pipeline- Pipeline to be destroyed
+   * Fire events to close all containers related to the input pipeline.
+   * @param pipelineId - ID of the pipeline.
* @throws IOException
*/
-  protected void destroyPipeline(Pipeline pipeline) throws IOException {
-pipelineFactory.close(pipeline.getType(), pipeline);
-// remove the pipeline from the pipeline manager
-removePipeline(pipeline.getId());
-triggerPipelineCreation();
+  protected void closeContainersForPipeline(final PipelineID pipelineId)
+  throws IOException {
+Set containerIDs = stateManager.getContainers(pipelineId);
+for (ContainerID containerID : containerIDs) {
+  eventPublisher.fireEvent(SCMEvents.CLOSE_CONTAINER, containerID);
+}
   }
 
   /**
-   * Removes the pipeline from the db and pipeline state map.
-   *
-   * @param pipelineId - ID of the pipeline to be removed
+   * put pipeline in CLOSED state.
+   * @param pipeline - ID of the pipeline.
+   * @param onTimeout - whether to remove pipeline after some time.
* @throws IOException
*/
-  protected void removePipeline(PipelineID pipelineId) throws IOException {
+  @Override
+  public void closePipeline(Pipeline pipeline, boolean onTimeout)
+  throws IOException {
+PipelineID pipelineID = pipeline.getId();
 lock.writeLock().lock();
 try {
-  stateManager.removePipeline(pipelineId.getProtobuf());
-  metrics.incNumPipelineDestroyed();
-} catch (IOException ex) {
-  metrics.incNumPipelineDestroyFailed();
-  throw ex;
+  if (!pipeline.isClosed()) {
+stateManager.updatePipelineState(pipelineID.getProtobuf(),
+HddsProtos.PipelineState.PIPELINE_CLOSED);
+LOG.info("Pipeline {} moved to CLOSED state", pipeline)

[GitHub] [hadoop-ozone] timmylicheng commented on a change in pull request #1049: HDDS-3662 Decouple finalizeAndDestroyPipeline.

2020-07-06 Thread GitBox


timmylicheng commented on a change in pull request #1049:
URL: https://github.com/apache/hadoop-ozone/pull/1049#discussion_r450025900



##
File path: 
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManagerV2Impl.java
##
@@ -130,7 +134,7 @@ public static PipelineManagerV2Impl newPipelineManager(
   }
 
   @Override
-  public Pipeline createPipeline(ReplicationType type,
+  public synchronized Pipeline createPipeline(ReplicationType type,

Review comment:
   Deleted.





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



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



[GitHub] [hadoop-ozone] timmylicheng commented on a change in pull request #1049: HDDS-3662 Decouple finalizeAndDestroyPipeline.

2020-07-06 Thread GitBox


timmylicheng commented on a change in pull request #1049:
URL: https://github.com/apache/hadoop-ozone/pull/1049#discussion_r450025786



##
File path: 
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManagerV2Impl.java
##
@@ -310,94 +321,72 @@ public void openPipeline(PipelineID pipelineId) throws 
IOException {
   }
 
   /**
-   * Finalizes pipeline in the SCM. Removes pipeline and makes rpc call to
-   * destroy pipeline on the datanodes immediately or after timeout based on 
the
-   * value of onTimeout parameter.
-   *
-   * @param pipeline- Pipeline to be destroyed
-   * @param onTimeout   - if true pipeline is removed and destroyed on
-   *datanodes after timeout
-   * @throws IOException
-   */
-  @Override
-  public void finalizeAndDestroyPipeline(Pipeline pipeline, boolean onTimeout)
-  throws IOException {
-LOG.info("Destroying pipeline:{}", pipeline);
-finalizePipeline(pipeline.getId());
-if (onTimeout) {
-  long pipelineDestroyTimeoutInMillis =
-  
conf.getTimeDuration(ScmConfigKeys.OZONE_SCM_PIPELINE_DESTROY_TIMEOUT,
-  ScmConfigKeys.OZONE_SCM_PIPELINE_DESTROY_TIMEOUT_DEFAULT,
-  TimeUnit.MILLISECONDS);
-  scheduler.schedule(() -> destroyPipeline(pipeline),
-  pipelineDestroyTimeoutInMillis, TimeUnit.MILLISECONDS, LOG,
-  String.format("Destroy pipeline failed for pipeline:%s", pipeline));
-} else {
-  destroyPipeline(pipeline);
-}
-  }
-
-  /**
-   * Moves the pipeline to CLOSED state and sends close container command for
-   * all the containers in the pipeline.
+   * Removes the pipeline from the db and pipeline state map.
*
-   * @param pipelineId - ID of the pipeline to be moved to CLOSED state.
+   * @param pipeline - pipeline to be removed
* @throws IOException
*/
-  private void finalizePipeline(PipelineID pipelineId) throws IOException {
+  protected void removePipeline(Pipeline pipeline) throws IOException {
+pipelineFactory.close(pipeline.getType(), pipeline);
+PipelineID pipelineID = pipeline.getId();
 lock.writeLock().lock();
 try {
-  Pipeline pipeline = stateManager.getPipeline(pipelineId);
-  if (!pipeline.isClosed()) {
-stateManager.updatePipelineState(
-pipelineId.getProtobuf(), 
HddsProtos.PipelineState.PIPELINE_CLOSED);
-LOG.info("Pipeline {} moved to CLOSED state", pipeline);
-  }
-
-  // TODO fire events to datanodes for closing pipelines
-//  Set containerIDs = stateManager.getContainers(pipelineId);
-//  for (ContainerID containerID : containerIDs) {
-//eventPublisher.fireEvent(SCMEvents.CLOSE_CONTAINER, containerID);
-//  }
-  metrics.removePipelineMetrics(pipelineId);
+  closeContainersForPipeline(pipelineID);

Review comment:
   Updated in PipelineManagerV2. That was a mistake. Thanks 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:
us...@infra.apache.org



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



[GitHub] [hadoop-ozone] timmylicheng commented on a change in pull request #1049: HDDS-3662 Decouple finalizeAndDestroyPipeline.

2020-06-11 Thread GitBox


timmylicheng commented on a change in pull request #1049:
URL: https://github.com/apache/hadoop-ozone/pull/1049#discussion_r439238634



##
File path: 
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManagerV2Impl.java
##
@@ -420,10 +392,50 @@ public void scrubPipeline(ReplicationType type, 
ReplicationFactor factor)
   " since it stays at ALLOCATED stage for " +
   Duration.between(currentTime, p.getCreationTimestamp()).toMinutes() +
   " mins.");
-  finalizeAndDestroyPipeline(p, false);
+  closePipeline(p, false);
+  closeContainersForPipeline(p.getId());
 }
   }
 
+  private void scrubClosedPipeline(
+  ReplicationType type, ReplicationFactor factor, Instant currentTime)
+  throws IOException {
+long pipelineDestroyTimeoutInMillis =
+conf.getTimeDuration(ScmConfigKeys.OZONE_SCM_PIPELINE_DESTROY_TIMEOUT,
+ScmConfigKeys.OZONE_SCM_PIPELINE_DESTROY_TIMEOUT_DEFAULT,
+TimeUnit.MILLISECONDS);
+List closedPipelines = stateManager.getPipelines(type, factor,
+Pipeline.PipelineState.CLOSED).stream()

Review comment:
   I updated with for loop

##
File path: 
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManagerV2Impl.java
##
@@ -310,102 +315,69 @@ public void openPipeline(PipelineID pipelineId) throws 
IOException {
   }
 
   /**
-   * Finalizes pipeline in the SCM. Removes pipeline and makes rpc call to
-   * destroy pipeline on the datanodes immediately or after timeout based on 
the
-   * value of onTimeout parameter.
-   *
-   * @param pipeline- Pipeline to be destroyed
-   * @param onTimeout   - if true pipeline is removed and destroyed on
-   *datanodes after timeout
-   * @throws IOException
-   */
-  @Override
-  public void finalizeAndDestroyPipeline(Pipeline pipeline, boolean onTimeout)
-  throws IOException {
-LOG.info("Destroying pipeline:{}", pipeline);
-finalizePipeline(pipeline.getId());
-if (onTimeout) {
-  long pipelineDestroyTimeoutInMillis =
-  
conf.getTimeDuration(ScmConfigKeys.OZONE_SCM_PIPELINE_DESTROY_TIMEOUT,
-  ScmConfigKeys.OZONE_SCM_PIPELINE_DESTROY_TIMEOUT_DEFAULT,
-  TimeUnit.MILLISECONDS);
-  scheduler.schedule(() -> destroyPipeline(pipeline),
-  pipelineDestroyTimeoutInMillis, TimeUnit.MILLISECONDS, LOG,
-  String.format("Destroy pipeline failed for pipeline:%s", pipeline));
-} else {
-  destroyPipeline(pipeline);
-}
-  }
-
-  /**
-   * Moves the pipeline to CLOSED state and sends close container command for
-   * all the containers in the pipeline.
+   * Removes the pipeline from the db and pipeline state map.
*
-   * @param pipelineId - ID of the pipeline to be moved to CLOSED state.
+   * @param pipeline - pipeline to be removed
* @throws IOException
*/
-  private void finalizePipeline(PipelineID pipelineId) throws IOException {
+  protected void removePipeline(Pipeline pipeline) throws IOException {
+pipelineFactory.close(pipeline.getType(), pipeline);
+PipelineID pipelineID = pipeline.getId();
+closeContainersForPipeline(pipelineID);
 lock.writeLock().lock();
 try {
-  Pipeline pipeline = stateManager.getPipeline(pipelineId);
-  if (!pipeline.isClosed()) {
-stateManager.updatePipelineState(pipelineId.getProtobuf(),
-HddsProtos.PipelineState.PIPELINE_CLOSED);
-LOG.info("Pipeline {} moved to CLOSED state", pipeline);
-  }
-
-  // TODO fire events to datanodes for closing pipelines
-//  Set containerIDs = stateManager.getContainers(pipelineId);
-//  for (ContainerID containerID : containerIDs) {
-//eventPublisher.fireEvent(SCMEvents.CLOSE_CONTAINER, containerID);
-//  }
-  metrics.removePipelineMetrics(pipelineId);
+  stateManager.removePipeline(pipelineID.getProtobuf());
+  metrics.incNumPipelineDestroyed();
+} catch (IOException ex) {
+  metrics.incNumPipelineDestroyFailed();
+  throw ex;
 } finally {
   lock.writeLock().unlock();
 }
   }
 
   /**
-   * Removes pipeline from SCM. Sends ratis command to destroy pipeline on all
-   * the datanodes for ratis pipelines.
-   *
-   * @param pipeline- Pipeline to be destroyed
+   * Fire events to close all containers related to the input pipeline.
+   * @param pipelineId - ID of the pipeline.
* @throws IOException
*/
-  protected void destroyPipeline(Pipeline pipeline) throws IOException {
-pipelineFactory.close(pipeline.getType(), pipeline);
-// remove the pipeline from the pipeline manager
-removePipeline(pipeline.getId());
-triggerPipelineCreation();
+  protected void closeContainersForPipeline(final PipelineID pipelineId)
+  throws IOException {
+Set containerIDs = stateManager.getContainers(pipelineId);
+for (ContainerID containerID : contain

[GitHub] [hadoop-ozone] timmylicheng commented on a change in pull request #1049: HDDS-3662 Decouple finalizeAndDestroyPipeline.

2020-06-11 Thread GitBox


timmylicheng commented on a change in pull request #1049:
URL: https://github.com/apache/hadoop-ozone/pull/1049#discussion_r439189923



##
File path: 
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManagerV2Impl.java
##
@@ -310,102 +315,69 @@ public void openPipeline(PipelineID pipelineId) throws 
IOException {
   }
 
   /**
-   * Finalizes pipeline in the SCM. Removes pipeline and makes rpc call to
-   * destroy pipeline on the datanodes immediately or after timeout based on 
the
-   * value of onTimeout parameter.
-   *
-   * @param pipeline- Pipeline to be destroyed
-   * @param onTimeout   - if true pipeline is removed and destroyed on
-   *datanodes after timeout
-   * @throws IOException
-   */
-  @Override
-  public void finalizeAndDestroyPipeline(Pipeline pipeline, boolean onTimeout)
-  throws IOException {
-LOG.info("Destroying pipeline:{}", pipeline);
-finalizePipeline(pipeline.getId());
-if (onTimeout) {
-  long pipelineDestroyTimeoutInMillis =
-  
conf.getTimeDuration(ScmConfigKeys.OZONE_SCM_PIPELINE_DESTROY_TIMEOUT,
-  ScmConfigKeys.OZONE_SCM_PIPELINE_DESTROY_TIMEOUT_DEFAULT,
-  TimeUnit.MILLISECONDS);
-  scheduler.schedule(() -> destroyPipeline(pipeline),
-  pipelineDestroyTimeoutInMillis, TimeUnit.MILLISECONDS, LOG,
-  String.format("Destroy pipeline failed for pipeline:%s", pipeline));
-} else {
-  destroyPipeline(pipeline);
-}
-  }
-
-  /**
-   * Moves the pipeline to CLOSED state and sends close container command for
-   * all the containers in the pipeline.
+   * Removes the pipeline from the db and pipeline state map.
*
-   * @param pipelineId - ID of the pipeline to be moved to CLOSED state.
+   * @param pipeline - pipeline to be removed
* @throws IOException
*/
-  private void finalizePipeline(PipelineID pipelineId) throws IOException {
+  protected void removePipeline(Pipeline pipeline) throws IOException {
+pipelineFactory.close(pipeline.getType(), pipeline);
+PipelineID pipelineID = pipeline.getId();
+closeContainersForPipeline(pipelineID);
 lock.writeLock().lock();
 try {
-  Pipeline pipeline = stateManager.getPipeline(pipelineId);
-  if (!pipeline.isClosed()) {
-stateManager.updatePipelineState(pipelineId.getProtobuf(),
-HddsProtos.PipelineState.PIPELINE_CLOSED);
-LOG.info("Pipeline {} moved to CLOSED state", pipeline);
-  }
-
-  // TODO fire events to datanodes for closing pipelines
-//  Set containerIDs = stateManager.getContainers(pipelineId);
-//  for (ContainerID containerID : containerIDs) {
-//eventPublisher.fireEvent(SCMEvents.CLOSE_CONTAINER, containerID);
-//  }
-  metrics.removePipelineMetrics(pipelineId);
+  stateManager.removePipeline(pipelineID.getProtobuf());
+  metrics.incNumPipelineDestroyed();
+} catch (IOException ex) {
+  metrics.incNumPipelineDestroyFailed();
+  throw ex;
 } finally {
   lock.writeLock().unlock();
 }
   }
 
   /**
-   * Removes pipeline from SCM. Sends ratis command to destroy pipeline on all
-   * the datanodes for ratis pipelines.
-   *
-   * @param pipeline- Pipeline to be destroyed
+   * Fire events to close all containers related to the input pipeline.
+   * @param pipelineId - ID of the pipeline.
* @throws IOException
*/
-  protected void destroyPipeline(Pipeline pipeline) throws IOException {
-pipelineFactory.close(pipeline.getType(), pipeline);
-// remove the pipeline from the pipeline manager
-removePipeline(pipeline.getId());
-triggerPipelineCreation();
+  protected void closeContainersForPipeline(final PipelineID pipelineId)
+  throws IOException {
+Set containerIDs = stateManager.getContainers(pipelineId);
+for (ContainerID containerID : containerIDs) {
+  eventPublisher.fireEvent(SCMEvents.CLOSE_CONTAINER, containerID);
+}
   }
 
   /**
-   * Removes the pipeline from the db and pipeline state map.
-   *
-   * @param pipelineId - ID of the pipeline to be removed
+   * put pipeline in CLOSED state.
+   * @param pipeline - ID of the pipeline.
+   * @param onTimeout - whether to remove pipeline after some time.
* @throws IOException
*/
-  protected void removePipeline(PipelineID pipelineId) throws IOException {
+  @Override
+  public void closePipeline(Pipeline pipeline, boolean onTimeout)

Review comment:
   If onTimeout is false, closePipeline will remove pipeline on spot. 
RemovePipeline is going to close containers and remove pipeline from db as well





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


[GitHub] [hadoop-ozone] timmylicheng commented on a change in pull request #1049: HDDS-3662 Decouple finalizeAndDestroyPipeline.

2020-06-11 Thread GitBox


timmylicheng commented on a change in pull request #1049:
URL: https://github.com/apache/hadoop-ozone/pull/1049#discussion_r439189989



##
File path: 
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineManagerV2Impl.java
##
@@ -420,10 +392,50 @@ public void scrubPipeline(ReplicationType type, 
ReplicationFactor factor)
   " since it stays at ALLOCATED stage for " +
   Duration.between(currentTime, p.getCreationTimestamp()).toMinutes() +
   " mins.");
-  finalizeAndDestroyPipeline(p, false);
+  closePipeline(p, false);
+  closeContainersForPipeline(p.getId());

Review comment:
   Ok good catch





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



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