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 2021/10/04 23:26:48 UTC

[GitHub] [ozone] umamaheswararao commented on a change in pull request #2702: HDDS-5755 EC: Refactor ECBlockOutputStreamEntry to accommodate all block group related ECBlockOuputStreams.

umamaheswararao commented on a change in pull request #2702:
URL: https://github.com/apache/ozone/pull/2702#discussion_r721766624



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockOutputStreamEntryPool.java
##########
@@ -283,26 +338,21 @@ void commitKey(long offset) throws IOException {
     }
   }
 
-  public BlockOutputStreamEntry getCurrentStreamEntry() {
+  BlockOutputStreamEntry getCurrentStreamEntry() {
     if (streamEntries.isEmpty() || streamEntries.size() <= currentStreamIndex) {
       return null;
     } else {
       return streamEntries.get(currentStreamIndex);
     }
   }
 
-  public int getCurrIdx(){
-    return currentStreamIndex;
-  }
-
-  public void setCurrIdx(int currIdx) {
-    this.currentStreamIndex = currIdx;
-  }
-
-  public void updateToNextStream(int rotation) {
-    currentStreamIndex = (currentStreamIndex + 1) % rotation;
-  }
-
+  /**
+   * Allocates a new block with OM, currently ECKeyOutputStream is an exclusive

Review comment:
       KeyOutputStream also uses this method. I would to suggest to avoid this doc here. If you wanted to improve the docs, let's file a master JIRA and improve. Otherwise we will have more diff compared to master and need to deal merge conflicts. So, we are following that if anything qualifies for master, we are just filing master JIRAs and committing there.

##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestECKeyOutputStream.java
##########
@@ -199,64 +191,6 @@ public void testCreateRatisKeyAndWithECBucketDefaults() throws Exception {
     }
   }
 
-

Review comment:
       Why we removed this test?

##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockOutputStreamEntry.java
##########
@@ -36,7 +36,14 @@
 import com.google.common.annotations.VisibleForTesting;
 
 /**

Review comment:
       On a high level, lot of changes in this file can go into master. So, I would suggest to do minimal changes. All EC specific to go into ECClasses. EC specific javadoc explanation can go into ECBlockoutputStreaEntry.

##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockOutputStreamEntry.java
##########
@@ -17,53 +17,259 @@
  */
 package org.apache.hadoop.ozone.client.io;
 
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.ImmutableList;
 import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.scm.OzoneClientConfig;
 import org.apache.hadoop.hdds.scm.XceiverClientFactory;
 import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.storage.BlockOutputStream;
 import org.apache.hadoop.hdds.scm.storage.BufferPool;
 import org.apache.hadoop.hdds.scm.storage.ECBlockOutputStream;
 import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier;
 import org.apache.hadoop.security.token.Token;
+import org.apache.ratis.util.Preconditions;
 
 import java.io.IOException;
+import java.io.OutputStream;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
 
 /**
  * Helper for {@link ECBlockOutputStream}.
  */
 public class ECBlockOutputStreamEntry extends BlockOutputStreamEntry{
-  private final boolean isParityStreamEntry;
-  private ECBlockOutputStream out;
+  private ECBlockOutputStream[] blockOutputStreams;
+  private final ECReplicationConfig replicationConfig;
+  private Map<DatanodeDetails, Integer> replicaIndicies = new HashMap<>();
+
+  private int currentStreamIdx = 0;
   @SuppressWarnings({"parameternumber", "squid:S00107"})
   ECBlockOutputStreamEntry(BlockID blockID, String key,
       XceiverClientFactory xceiverClientManager, Pipeline pipeline, long length,
       BufferPool bufferPool, Token<OzoneBlockTokenIdentifier> token,
-      OzoneClientConfig config, boolean isParityStream) {
+      OzoneClientConfig config) {
     super(blockID, key, xceiverClientManager, pipeline, length, bufferPool,
         token, config);
-    this.isParityStreamEntry = isParityStream;
+    Preconditions.assertInstanceOf(
+        pipeline.getReplicationConfig(), ECReplicationConfig.class);
+    this.replicationConfig =
+        (ECReplicationConfig) pipeline.getReplicationConfig();
+  }
+
+  @Override
+  void createOutputStream() throws IOException {
+    Pipeline ecPipeline = getPipeline();
+    List<DatanodeDetails> nodes = getPipeline().getNodes();
+    blockOutputStreams =
+        new ECBlockOutputStream[nodes.size()];
+    for (int i = 0; i< getPipeline().getNodes().size(); i++) {
+      blockOutputStreams[i] = new ECBlockOutputStream(
+          getBlockID(),
+          getXceiverClientManager(),
+          createSingleECBlockPipeline(ecPipeline, nodes.get(i), i+1),
+          getBufferPool(),
+          getConf(),
+          getToken());
+    }
+  }
+
+  @Override
+  public OutputStream getOutputStream() {
+    if (!isInitialized()) {
+      return null;
+    }
+    return blockOutputStreams[currentStreamIdx];
+  }
+
+  @Override
+  boolean isInitialized() {
+    return blockOutputStreams != null;
+  }
+
+  public int getCurrentStreamIdx() {
+    return currentStreamIdx;
+  }
+
+  public void useNextBlockStream() {
+    currentStreamIdx++;
+  }
+
+  public void forceToFirstParityBlock(){
+    currentStreamIdx = replicationConfig.getData();
+  }
+
+  @Override
+  void incCurrentPosition() {
+    if (isWritingParity()) {
+      return;
+    }
+    super.incCurrentPosition();
+  }
+
+  @Override
+  void incCurrentPosition(long len) {
+    if (isWritingParity()){
+      return;
+    }
+    super.incCurrentPosition(len);
+  }
+
+  @Override
+  public void flush() throws IOException {
+    if (!isInitialized()) {
+      return;
+    }
+    for(int i=0; i<=currentStreamIdx && i<blockOutputStreams.length; i++) {
+      blockOutputStreams[i].flush();
+    }
+  }
+
+  @Override
+  boolean isClosed() {
+    if (!isInitialized()) {
+      return false;
+    }
+    return blockStreams().allMatch(BlockOutputStream::isClosed);
   }
 
   @Override
-  ECBlockOutputStream createOutputStream() throws IOException {
-    this.out = new ECBlockOutputStream(getBlockID(), getXceiverClientManager(),
-        getPipeline(), getBufferPool(), getConf(), getToken());
-    return this.out;
+  public void close() throws IOException {
+    if (!isInitialized()) {
+      return;
+    }
+    for (ECBlockOutputStream stream : blockOutputStreams) {
+      stream.close();
+    }
+    updateBlockID(underlyingBlockID());
+  }
+
+  @Override
+  long getTotalAckDataLength() {
+    if (!isInitialized()) {
+      return 0;
+    }
+    // blockID is the same for EC blocks inside one block group managed by
+    // this entry.
+    updateBlockID(underlyingBlockID());
+    //TODO: A future implementation might require something like this, but
+    // currently as ECBlockOutputStream is inheriting from BlockOutputStream
+    // this method returns 0 all the time from the unrelying streams.
+    // After we have a confirmed ack mechanism, like there is in
+    // RatisBlockOutputStream, we should revisit this part, and decide if we
+    // want to filter out parity here for example.
+//    return blockStreams()
+//        .mapToLong(BlockOutputStream::getTotalAckDataLength)
+//        .sum();
+    return 0;
+  }
+
+  /**
+   * Returns the amount of bytes that were attempted to be sent through towards
+   * the DataNodes, and the write call succeeded without an exception.
+   * In EC entries the parity writes does not count into this, as the written
+   * data length represents the attempts of the classes using the entry, and
+   * not the attempts of the entry itself.
+   * @return 0 if the stream is not initialized, the amount of data bytes that
+   *    were attempted to be written to the entry.
+   */
+  //TODO: this might become problematic, and should be tested during the
+  //      implementation of retries and error handling, as if there is a retry,
+  //      then some data might have to be written twice.
+  //      This current implementation is an assumption here.
+  //      We might need to account the parity bytes written here, or elsewhere.
+  @Override
+  long getWrittenDataLength() {
+    if (!isInitialized()) {
+      return 0;
+    }
+    return blockStreams()

Review comment:
       Where are we skipping parity blocks length in blogGroup Len?

##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyOutputStream.java
##########
@@ -27,7 +27,6 @@
 import java.util.stream.Collectors;
 
 import org.apache.hadoop.fs.FSExceptionMessages;

Review comment:
       Why do we need to change about FileEncryption changes here? Same as above: can they qualify into master?

##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockOutputStreamEntry.java
##########
@@ -17,53 +17,259 @@
  */
 package org.apache.hadoop.ozone.client.io;
 
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.ImmutableList;
 import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.scm.OzoneClientConfig;
 import org.apache.hadoop.hdds.scm.XceiverClientFactory;
 import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.storage.BlockOutputStream;
 import org.apache.hadoop.hdds.scm.storage.BufferPool;
 import org.apache.hadoop.hdds.scm.storage.ECBlockOutputStream;
 import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier;
 import org.apache.hadoop.security.token.Token;
+import org.apache.ratis.util.Preconditions;
 
 import java.io.IOException;
+import java.io.OutputStream;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
 
 /**
  * Helper for {@link ECBlockOutputStream}.
  */
 public class ECBlockOutputStreamEntry extends BlockOutputStreamEntry{
-  private final boolean isParityStreamEntry;
-  private ECBlockOutputStream out;
+  private ECBlockOutputStream[] blockOutputStreams;
+  private final ECReplicationConfig replicationConfig;
+  private Map<DatanodeDetails, Integer> replicaIndicies = new HashMap<>();
+
+  private int currentStreamIdx = 0;
   @SuppressWarnings({"parameternumber", "squid:S00107"})
   ECBlockOutputStreamEntry(BlockID blockID, String key,
       XceiverClientFactory xceiverClientManager, Pipeline pipeline, long length,
       BufferPool bufferPool, Token<OzoneBlockTokenIdentifier> token,
-      OzoneClientConfig config, boolean isParityStream) {
+      OzoneClientConfig config) {
     super(blockID, key, xceiverClientManager, pipeline, length, bufferPool,
         token, config);
-    this.isParityStreamEntry = isParityStream;
+    Preconditions.assertInstanceOf(
+        pipeline.getReplicationConfig(), ECReplicationConfig.class);
+    this.replicationConfig =
+        (ECReplicationConfig) pipeline.getReplicationConfig();
+  }
+
+  @Override
+  void createOutputStream() throws IOException {
+    Pipeline ecPipeline = getPipeline();

Review comment:
       I think we don;t need to pass fill pipeline. WE can just create single node pipeline for that stream.

##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
##########
@@ -1071,7 +1071,7 @@ public OzoneOutputStream createMultipartKey(String volumeName,
     keyOutputStream.addPreallocateBlocks(
         openKey.getKeyInfo().getLatestVersionLocations(),
         openKey.getOpenVersion());

Review comment:
       We may consider this changes to master?

##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockOutputStreamEntry.java
##########
@@ -17,53 +17,259 @@
  */
 package org.apache.hadoop.ozone.client.io;
 
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.ImmutableList;
 import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.scm.OzoneClientConfig;
 import org.apache.hadoop.hdds.scm.XceiverClientFactory;
 import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.storage.BlockOutputStream;
 import org.apache.hadoop.hdds.scm.storage.BufferPool;
 import org.apache.hadoop.hdds.scm.storage.ECBlockOutputStream;
 import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier;
 import org.apache.hadoop.security.token.Token;
+import org.apache.ratis.util.Preconditions;
 
 import java.io.IOException;
+import java.io.OutputStream;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
 
 /**
  * Helper for {@link ECBlockOutputStream}.
  */
 public class ECBlockOutputStreamEntry extends BlockOutputStreamEntry{
-  private final boolean isParityStreamEntry;
-  private ECBlockOutputStream out;
+  private ECBlockOutputStream[] blockOutputStreams;
+  private final ECReplicationConfig replicationConfig;
+  private Map<DatanodeDetails, Integer> replicaIndicies = new HashMap<>();
+
+  private int currentStreamIdx = 0;
   @SuppressWarnings({"parameternumber", "squid:S00107"})
   ECBlockOutputStreamEntry(BlockID blockID, String key,
       XceiverClientFactory xceiverClientManager, Pipeline pipeline, long length,
       BufferPool bufferPool, Token<OzoneBlockTokenIdentifier> token,
-      OzoneClientConfig config, boolean isParityStream) {
+      OzoneClientConfig config) {
     super(blockID, key, xceiverClientManager, pipeline, length, bufferPool,
         token, config);
-    this.isParityStreamEntry = isParityStream;
+    Preconditions.assertInstanceOf(
+        pipeline.getReplicationConfig(), ECReplicationConfig.class);
+    this.replicationConfig =
+        (ECReplicationConfig) pipeline.getReplicationConfig();
+  }
+
+  @Override
+  void createOutputStream() throws IOException {
+    Pipeline ecPipeline = getPipeline();
+    List<DatanodeDetails> nodes = getPipeline().getNodes();
+    blockOutputStreams =
+        new ECBlockOutputStream[nodes.size()];
+    for (int i = 0; i< getPipeline().getNodes().size(); i++) {
+      blockOutputStreams[i] = new ECBlockOutputStream(
+          getBlockID(),
+          getXceiverClientManager(),
+          createSingleECBlockPipeline(ecPipeline, nodes.get(i), i+1),
+          getBufferPool(),
+          getConf(),
+          getToken());
+    }
+  }
+
+  @Override
+  public OutputStream getOutputStream() {
+    if (!isInitialized()) {
+      return null;
+    }
+    return blockOutputStreams[currentStreamIdx];
+  }
+
+  @Override
+  boolean isInitialized() {
+    return blockOutputStreams != null;
+  }
+
+  public int getCurrentStreamIdx() {
+    return currentStreamIdx;
+  }
+
+  public void useNextBlockStream() {
+    currentStreamIdx++;
+  }
+
+  public void forceToFirstParityBlock(){
+    currentStreamIdx = replicationConfig.getData();
+  }
+
+  @Override
+  void incCurrentPosition() {
+    if (isWritingParity()) {
+      return;
+    }
+    super.incCurrentPosition();
+  }
+
+  @Override
+  void incCurrentPosition(long len) {
+    if (isWritingParity()){
+      return;
+    }
+    super.incCurrentPosition(len);
+  }
+
+  @Override
+  public void flush() throws IOException {
+    if (!isInitialized()) {
+      return;
+    }
+    for(int i=0; i<=currentStreamIdx && i<blockOutputStreams.length; i++) {
+      blockOutputStreams[i].flush();
+    }
+  }
+
+  @Override
+  boolean isClosed() {
+    if (!isInitialized()) {
+      return false;
+    }
+    return blockStreams().allMatch(BlockOutputStream::isClosed);
   }
 
   @Override
-  ECBlockOutputStream createOutputStream() throws IOException {
-    this.out = new ECBlockOutputStream(getBlockID(), getXceiverClientManager(),
-        getPipeline(), getBufferPool(), getConf(), getToken());
-    return this.out;
+  public void close() throws IOException {
+    if (!isInitialized()) {
+      return;
+    }
+    for (ECBlockOutputStream stream : blockOutputStreams) {
+      stream.close();
+    }
+    updateBlockID(underlyingBlockID());
+  }
+
+  @Override
+  long getTotalAckDataLength() {
+    if (!isInitialized()) {
+      return 0;
+    }
+    // blockID is the same for EC blocks inside one block group managed by
+    // this entry.
+    updateBlockID(underlyingBlockID());
+    //TODO: A future implementation might require something like this, but
+    // currently as ECBlockOutputStream is inheriting from BlockOutputStream
+    // this method returns 0 all the time from the unrelying streams.
+    // After we have a confirmed ack mechanism, like there is in
+    // RatisBlockOutputStream, we should revisit this part, and decide if we
+    // want to filter out parity here for example.
+//    return blockStreams()
+//        .mapToLong(BlockOutputStream::getTotalAckDataLength)
+//        .sum();

Review comment:
       Why are we returning 0?

##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/BlockOutputStreamEntryPool.java
##########
@@ -156,10 +180,22 @@ void addKeyLocationInfo(OmKeyLocationInfo subKeyInfo) {
             .setConfig(config)
             .setLength(subKeyInfo.getLength())
             .setBufferPool(bufferPool)
-            .setToken(subKeyInfo.getToken());
-    streamEntries.add(builder.build());
+            .setToken(subKeyInfo.getToken())
+            .build();
+
+  }
+
+  private void addKeyLocationInfo(OmKeyLocationInfo subKeyInfo) {
+    Preconditions.checkNotNull(subKeyInfo.getPipeline());
+    streamEntries.add(createStreamEntry(subKeyInfo));
   }
 
+  /**

Review comment:
       Looks like lot of these java doc improvements should go to master. 

##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/ECBlockOutputStreamEntry.java
##########
@@ -17,53 +17,259 @@
  */
 package org.apache.hadoop.ozone.client.io;
 
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.ImmutableList;
 import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.scm.OzoneClientConfig;
 import org.apache.hadoop.hdds.scm.XceiverClientFactory;
 import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.storage.BlockOutputStream;
 import org.apache.hadoop.hdds.scm.storage.BufferPool;
 import org.apache.hadoop.hdds.scm.storage.ECBlockOutputStream;
 import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier;
 import org.apache.hadoop.security.token.Token;
+import org.apache.ratis.util.Preconditions;
 
 import java.io.IOException;
+import java.io.OutputStream;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
 
 /**
  * Helper for {@link ECBlockOutputStream}.
  */
 public class ECBlockOutputStreamEntry extends BlockOutputStreamEntry{
-  private final boolean isParityStreamEntry;
-  private ECBlockOutputStream out;
+  private ECBlockOutputStream[] blockOutputStreams;
+  private final ECReplicationConfig replicationConfig;
+  private Map<DatanodeDetails, Integer> replicaIndicies = new HashMap<>();
+
+  private int currentStreamIdx = 0;
   @SuppressWarnings({"parameternumber", "squid:S00107"})
   ECBlockOutputStreamEntry(BlockID blockID, String key,
       XceiverClientFactory xceiverClientManager, Pipeline pipeline, long length,
       BufferPool bufferPool, Token<OzoneBlockTokenIdentifier> token,
-      OzoneClientConfig config, boolean isParityStream) {
+      OzoneClientConfig config) {
     super(blockID, key, xceiverClientManager, pipeline, length, bufferPool,
         token, config);
-    this.isParityStreamEntry = isParityStream;
+    Preconditions.assertInstanceOf(
+        pipeline.getReplicationConfig(), ECReplicationConfig.class);
+    this.replicationConfig =
+        (ECReplicationConfig) pipeline.getReplicationConfig();
+  }
+
+  @Override
+  void createOutputStream() throws IOException {
+    Pipeline ecPipeline = getPipeline();
+    List<DatanodeDetails> nodes = getPipeline().getNodes();
+    blockOutputStreams =
+        new ECBlockOutputStream[nodes.size()];
+    for (int i = 0; i< getPipeline().getNodes().size(); i++) {
+      blockOutputStreams[i] = new ECBlockOutputStream(
+          getBlockID(),
+          getXceiverClientManager(),
+          createSingleECBlockPipeline(ecPipeline, nodes.get(i), i+1),
+          getBufferPool(),
+          getConf(),
+          getToken());
+    }
+  }
+
+  @Override
+  public OutputStream getOutputStream() {
+    if (!isInitialized()) {
+      return null;
+    }
+    return blockOutputStreams[currentStreamIdx];
+  }
+
+  @Override
+  boolean isInitialized() {
+    return blockOutputStreams != null;
+  }
+
+  public int getCurrentStreamIdx() {
+    return currentStreamIdx;
+  }
+
+  public void useNextBlockStream() {
+    currentStreamIdx++;
+  }
+
+  public void forceToFirstParityBlock(){
+    currentStreamIdx = replicationConfig.getData();
+  }
+
+  @Override
+  void incCurrentPosition() {
+    if (isWritingParity()) {
+      return;
+    }
+    super.incCurrentPosition();
+  }
+
+  @Override
+  void incCurrentPosition(long len) {
+    if (isWritingParity()){
+      return;
+    }
+    super.incCurrentPosition(len);
+  }
+
+  @Override
+  public void flush() throws IOException {
+    if (!isInitialized()) {
+      return;
+    }
+    for(int i=0; i<=currentStreamIdx && i<blockOutputStreams.length; i++) {
+      blockOutputStreams[i].flush();
+    }
+  }
+
+  @Override
+  boolean isClosed() {
+    if (!isInitialized()) {
+      return false;
+    }

Review comment:
       What if some streams not initialized?
   In the case when file hase 1MB, we will not intialize 2nd and 3rd streams as we don;t write anything there.




-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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