Repository: hadoop
Updated Branches:
  refs/heads/trunk 0473b6800 -> 5209c7503


Revert "HDDS-629. Make ApplyTransaction calls in ContainerStateMachine 
idempotent. Contributed by Shashikant Banerjee."

This reverts commit 0473b68000017cfe4f03acdcb0eedc78b509244f.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/22f37af9
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/22f37af9
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/22f37af9

Branch: refs/heads/trunk
Commit: 22f37af93583e9c63e03f0781ffb903e35544559
Parents: 0473b68
Author: Jitendra Pandey <jiten...@apache.org>
Authored: Sat Oct 13 12:14:39 2018 -0700
Committer: Jitendra Pandey <jiten...@apache.org>
Committed: Sat Oct 13 12:14:39 2018 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/ozone/OzoneConsts.java    |   1 -
 .../apache/hadoop/utils/MetadataKeyFilters.java |   3 +-
 .../common/helpers/ContainerReport.java         | 205 +++++++++++++++++++
 .../common/helpers/KeyValueContainerReport.java | 117 +++++++++++
 .../container/common/interfaces/Container.java  |   5 -
 .../container/keyvalue/KeyValueContainer.java   |   9 +-
 .../keyvalue/KeyValueContainerData.java         |  16 --
 .../keyvalue/impl/BlockManagerImpl.java         |  21 +-
 .../container/ozoneimpl/ContainerReader.java    |   7 -
 .../StorageContainerDatanodeProtocol.proto      |   1 -
 .../ozone/om/helpers/OmKeyLocationInfo.java     |   4 -
 .../hadoop/ozone/client/rpc/TestBCSID.java      | 147 -------------
 12 files changed, 328 insertions(+), 208 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/22f37af9/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java 
b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
index 8ccc648..923271c 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java
@@ -113,7 +113,6 @@ public final class OzoneConsts {
   public static final String DELETING_KEY_PREFIX = "#deleting#";
   public static final String DELETED_KEY_PREFIX = "#deleted#";
   public static final String DELETE_TRANSACTION_KEY_PREFIX = "#delTX#";
-  public static final String BLOCK_COMMIT_SEQUENCE_ID_PREFIX = "#BCSID";
 
   /**
    * OM LevelDB prefixes.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/22f37af9/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/MetadataKeyFilters.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/MetadataKeyFilters.java
 
b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/MetadataKeyFilters.java
index 04c87ae..a3430f8 100644
--- 
a/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/MetadataKeyFilters.java
+++ 
b/hadoop-hdds/common/src/main/java/org/apache/hadoop/utils/MetadataKeyFilters.java
@@ -42,8 +42,7 @@ public final class MetadataKeyFilters {
       new MetadataKeyFilters.KeyPrefixFilter()
           .addFilter(OzoneConsts.DELETING_KEY_PREFIX, true)
           .addFilter(OzoneConsts.DELETED_KEY_PREFIX, true)
-          .addFilter(OzoneConsts.DELETE_TRANSACTION_KEY_PREFIX, true)
-          .addFilter(OzoneConsts.BLOCK_COMMIT_SEQUENCE_ID_PREFIX, true);
+          .addFilter(OzoneConsts.DELETE_TRANSACTION_KEY_PREFIX, true);
 
   private MetadataKeyFilters() {
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/22f37af9/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerReport.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerReport.java
 
b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerReport.java
new file mode 100644
index 0000000..a4c1f2f
--- /dev/null
+++ 
b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerReport.java
@@ -0,0 +1,205 @@
+/*
+ * 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.container.common.helpers;
+
+import com.google.common.base.Preconditions;
+import 
org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerInfo;
+
+
+/**
+ * Container Report iterates the closed containers and sends a container report
+ * to SCM.
+ */
+public class ContainerReport {
+  private static final int UNKNOWN = -1;
+  private final String finalhash;
+  private long size;
+  private long keyCount;
+  private long bytesUsed;
+  private long readCount;
+  private long writeCount;
+  private long readBytes;
+  private long writeBytes;
+  private long containerID;
+
+  public long getContainerID() {
+    return containerID;
+  }
+
+  public void setContainerID(long containerID) {
+    this.containerID = containerID;
+  }
+
+  /**
+   * Constructs the ContainerReport.
+   *
+   * @param containerID - Container ID.
+   * @param finalhash - Final Hash.
+   */
+  public ContainerReport(long containerID, String finalhash) {
+    this.containerID = containerID;
+    this.finalhash = finalhash;
+    this.size = UNKNOWN;
+    this.keyCount = UNKNOWN;
+    this.bytesUsed = 0L;
+    this.readCount = 0L;
+    this.readBytes = 0L;
+    this.writeCount = 0L;
+    this.writeBytes = 0L;
+  }
+
+  /**
+   * Gets a containerReport from protobuf class.
+   *
+   * @param info - ContainerInfo.
+   * @return - ContainerReport.
+   */
+  public static ContainerReport getFromProtoBuf(ContainerInfo info) {
+    Preconditions.checkNotNull(info);
+    ContainerReport report = new ContainerReport(info.getContainerID(),
+        info.getFinalhash());
+    if (info.hasSize()) {
+      report.setSize(info.getSize());
+    }
+    if (info.hasKeyCount()) {
+      report.setKeyCount(info.getKeyCount());
+    }
+    if (info.hasUsed()) {
+      report.setBytesUsed(info.getUsed());
+    }
+    if (info.hasReadCount()) {
+      report.setReadCount(info.getReadCount());
+    }
+    if (info.hasReadBytes()) {
+      report.setReadBytes(info.getReadBytes());
+    }
+    if (info.hasWriteCount()) {
+      report.setWriteCount(info.getWriteCount());
+    }
+    if (info.hasWriteBytes()) {
+      report.setWriteBytes(info.getWriteBytes());
+    }
+
+    report.setContainerID(info.getContainerID());
+    return report;
+  }
+
+  /**
+   * Returns the final signature for this container.
+   *
+   * @return - hash
+   */
+  public String getFinalhash() {
+    return finalhash;
+  }
+
+  /**
+   * Returns a positive number it is a valid number, -1 if not known.
+   *
+   * @return size or -1
+   */
+  public long getSize() {
+    return size;
+  }
+
+  /**
+   * Sets the size of the container on disk.
+   *
+   * @param size - int
+   */
+  public void setSize(long size) {
+    this.size = size;
+  }
+
+  /**
+   * Gets number of keys in the container if known.
+   *
+   * @return - Number of keys or -1 for not known.
+   */
+  public long getKeyCount() {
+    return keyCount;
+  }
+
+  /**
+   * Sets the key count.
+   *
+   * @param keyCount - Key Count
+   */
+  public void setKeyCount(long keyCount) {
+    this.keyCount = keyCount;
+  }
+
+  public long getReadCount() {
+    return readCount;
+  }
+
+  public void setReadCount(long readCount) {
+    this.readCount = readCount;
+  }
+
+  public long getWriteCount() {
+    return writeCount;
+  }
+
+  public void setWriteCount(long writeCount) {
+    this.writeCount = writeCount;
+  }
+
+  public long getReadBytes() {
+    return readBytes;
+  }
+
+  public void setReadBytes(long readBytes) {
+    this.readBytes = readBytes;
+  }
+
+  public long getWriteBytes() {
+    return writeBytes;
+  }
+
+  public void setWriteBytes(long writeBytes) {
+    this.writeBytes = writeBytes;
+  }
+
+  public long getBytesUsed() {
+    return bytesUsed;
+  }
+
+  public void setBytesUsed(long bytesUsed) {
+    this.bytesUsed = bytesUsed;
+  }
+
+  /**
+   * Gets a containerInfo protobuf message from ContainerReports.
+   *
+   * @return ContainerInfo
+   */
+  public ContainerInfo getProtoBufMessage() {
+    return ContainerInfo.newBuilder()
+        .setKeyCount(this.getKeyCount())
+        .setSize(this.getSize())
+        .setUsed(this.getBytesUsed())
+        .setReadCount(this.getReadCount())
+        .setReadBytes(this.getReadBytes())
+        .setWriteCount(this.getWriteCount())
+        .setWriteBytes(this.getWriteBytes())
+        .setFinalhash(this.getFinalhash())
+        .setContainerID(this.getContainerID())
+        .build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/22f37af9/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyValueContainerReport.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyValueContainerReport.java
 
b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyValueContainerReport.java
new file mode 100644
index 0000000..b03487b
--- /dev/null
+++ 
b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyValueContainerReport.java
@@ -0,0 +1,117 @@
+/*
+ * 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.container.common.helpers;
+
+import com.google.common.base.Preconditions;
+import 
org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerInfo;
+
+import static java.lang.Math.max;
+
+/**
+ * KeyValueContainer Report iterates the closed containers and sends a
+ * container report to SCM.
+ */
+public class KeyValueContainerReport extends ContainerReport{
+  private long deleteTransactionId;
+
+  /**
+   * Constructs the KeyValueContainerReport.
+   *
+   * @param containerID - Container ID.
+   * @param finalhash - Final Hash.
+   */
+  public KeyValueContainerReport(long containerID, String finalhash) {
+    super(containerID, finalhash);
+    this.deleteTransactionId = 0;
+  }
+
+  /**
+   * Sets the deleteTransactionId if it is greater than existing.
+   * @param transactionId - deleteTransactionId
+   */
+  public void updateDeleteTransactionId(long transactionId) {
+    this.deleteTransactionId = max(transactionId, deleteTransactionId);
+  }
+
+  /**
+   * Gets the deleteTransactionId.
+   * @return - deleteTransactionId.
+   */
+  public long getDeleteTransactionId() {
+    return this.deleteTransactionId;
+  }
+
+  /**
+   * Gets a containerReport from protobuf class.
+   *
+   * @param info - ContainerInfo.
+   * @return - ContainerReport.
+   */
+  public static KeyValueContainerReport getFromProtoBuf(ContainerInfo info) {
+    Preconditions.checkNotNull(info);
+    KeyValueContainerReport report = new KeyValueContainerReport(
+        info.getContainerID(), info.getFinalhash());
+    if (info.hasSize()) {
+      report.setSize(info.getSize());
+    }
+    if (info.hasKeyCount()) {
+      report.setKeyCount(info.getKeyCount());
+    }
+    if (info.hasUsed()) {
+      report.setBytesUsed(info.getUsed());
+    }
+    if (info.hasReadCount()) {
+      report.setReadCount(info.getReadCount());
+    }
+    if (info.hasReadBytes()) {
+      report.setReadBytes(info.getReadBytes());
+    }
+    if (info.hasWriteCount()) {
+      report.setWriteCount(info.getWriteCount());
+    }
+    if (info.hasWriteBytes()) {
+      report.setWriteBytes(info.getWriteBytes());
+    }
+    if (info.hasDeleteTransactionId()) {
+      report.updateDeleteTransactionId(info.getDeleteTransactionId());
+    }
+    report.setContainerID(info.getContainerID());
+    return report;
+  }
+
+  /**
+   * Gets a containerInfo protobuf message from ContainerReports.
+   *
+   * @return ContainerInfo
+   */
+  @Override
+  public ContainerInfo getProtoBufMessage() {
+    return ContainerInfo.newBuilder()
+        .setKeyCount(this.getKeyCount())
+        .setSize(this.getSize())
+        .setUsed(this.getBytesUsed())
+        .setReadCount(this.getReadCount())
+        .setReadBytes(this.getReadBytes())
+        .setWriteCount(this.getWriteCount())
+        .setWriteBytes(this.getWriteBytes())
+        .setFinalhash(this.getFinalhash())
+        .setContainerID(this.getContainerID())
+        .setDeleteTransactionId(this.getDeleteTransactionId())
+        .build();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/22f37af9/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Container.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Container.java
 
b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Container.java
index dbef74c..9380f0c 100644
--- 
a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Container.java
+++ 
b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Container.java
@@ -132,9 +132,4 @@ public interface Container<CONTAINERDATA extends 
ContainerData> extends RwLock {
    */
   StorageContainerDatanodeProtocolProtos.ContainerInfo getContainerReport()
       throws StorageContainerException;
-
-  /**
-   * updates the blockCommitSequenceId.
-   */
-  void updateBlockCommitSequenceId(long blockCommitSequenceId);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/22f37af9/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
 
b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
index 9a5c94c..e5b344d 100644
--- 
a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
+++ 
b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainer.java
@@ -508,12 +508,6 @@ public class KeyValueContainer implements 
Container<KeyValueContainerData> {
         .getContainerID() + OzoneConsts.CONTAINER_EXTENSION);
   }
 
-  @Override
-  public void updateBlockCommitSequenceId(long blockCommitSequenceId) {
-    containerData.updateBlockCommitSequenceId(blockCommitSequenceId);
-  }
-
-
   /**
    * Returns KeyValueContainerReport for the KeyValueContainer.
    */
@@ -530,8 +524,7 @@ public class KeyValueContainer implements 
Container<KeyValueContainerData> {
         .setKeyCount(containerData.getKeyCount())
         .setUsed(containerData.getBytesUsed())
         .setState(getHddsState())
-        .setDeleteTransactionId(containerData.getDeleteTransactionId())
-        .setBlockCommitSequenceId(containerData.getBlockCommitSequenceId());
+        .setDeleteTransactionId(containerData.getDeleteTransactionId());
     return ciBuilder.build();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/22f37af9/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java
 
b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java
index 9ea84c2..7ffdbf5 100644
--- 
a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java
+++ 
b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueContainerData.java
@@ -72,8 +72,6 @@ public class KeyValueContainerData extends ContainerData {
 
   private long deleteTransactionId;
 
-  private long blockCommitSequenceId;
-
   static {
     // Initialize YAML fields
     KV_YAML_FIELDS = Lists.newArrayList();
@@ -154,20 +152,6 @@ public class KeyValueContainerData extends ContainerData {
   }
 
   /**
-   * Returns the blockCommitSequenceId.
-   */
-  public long getBlockCommitSequenceId() {
-    return blockCommitSequenceId;
-  }
-
-  /**
-   * updates the blockCommitSequenceId.
-   */
-  public void updateBlockCommitSequenceId(long id) {
-    this.blockCommitSequenceId = id;
-  }
-
-  /**
    * Get chunks path.
    * @return - Path where chunks are stored
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/22f37af9/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/BlockManagerImpl.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/BlockManagerImpl.java
 
b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/BlockManagerImpl.java
index bd19441..0dd8739 100644
--- 
a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/BlockManagerImpl.java
+++ 
b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/BlockManagerImpl.java
@@ -25,16 +25,12 @@ import org.apache.hadoop.hdds.client.BlockID;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
 import 
org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
 
-import org.apache.hadoop.hdfs.DFSUtil;
-import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.container.common.helpers.BlockData;
 import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
 import org.apache.hadoop.ozone.container.keyvalue.helpers.BlockUtils;
 import org.apache.hadoop.ozone.container.common.interfaces.Container;
 import org.apache.hadoop.ozone.container.keyvalue.interfaces.BlockManager;
 import org.apache.hadoop.ozone.container.common.utils.ContainerCache;
-import org.apache.hadoop.utils.BatchOperation;
-import org.apache.hadoop.utils.MetadataKeyFilters;
 import org.apache.hadoop.utils.MetadataStore;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -87,16 +83,8 @@ public class BlockManagerImpl implements BlockManager {
     // This is a post condition that acts as a hint to the user.
     // Should never fail.
     Preconditions.checkNotNull(db, "DB cannot be null here");
-
-    long blockCommitSequenceId = data.getBlockCommitSequenceId();
-    // update the blockData as well as BlockCommitSequenceId here
-    BatchOperation batch = new BatchOperation();
-    batch.put(Longs.toByteArray(data.getLocalID()),
-        data.getProtoBufMessage().toByteArray());
-    
batch.put(DFSUtil.string2Bytes(OzoneConsts.BLOCK_COMMIT_SEQUENCE_ID_PREFIX),
-        Longs.toByteArray(blockCommitSequenceId));
-    db.writeBatch(batch);
-    container.updateBlockCommitSequenceId(blockCommitSequenceId);
+    db.put(Longs.toByteArray(data.getLocalID()), data.getProtoBufMessage()
+        .toByteArray());
     // Increment keycount here
     container.getContainerData().incrKeyCount();
     return data.getSize();
@@ -220,9 +208,8 @@ public class BlockManagerImpl implements BlockManager {
     MetadataStore db = BlockUtils.getDB(cData, config);
     result = new ArrayList<>();
     byte[] startKeyInBytes = Longs.toByteArray(startLocalID);
-    List<Map.Entry<byte[], byte[]>> range =
-        db.getSequentialRangeKVs(startKeyInBytes, count,
-            MetadataKeyFilters.getNormalKeyFilter());
+    List<Map.Entry<byte[], byte[]>> range = db.getSequentialRangeKVs(
+        startKeyInBytes, count, null);
     for (Map.Entry<byte[], byte[]> entry : range) {
       BlockData value = BlockUtils.getBlockData(entry.getValue());
       BlockData data = new BlockData(value.getBlockID());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/22f37af9/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerReader.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerReader.java
 
b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerReader.java
index aee5775..c3a4126 100644
--- 
a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerReader.java
+++ 
b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerReader.java
@@ -191,13 +191,6 @@ public class ContainerReader implements Runnable {
           kvContainerData
               .updateDeleteTransactionId(Longs.fromByteArray(delTxnId));
         }
-        // sets the BlockCommitSequenceId.
-        byte[] bcsId = containerDB.get(
-            DFSUtil.string2Bytes(OzoneConsts.BLOCK_COMMIT_SEQUENCE_ID_PREFIX));
-        if (bcsId != null) {
-          kvContainerData
-              .updateBlockCommitSequenceId(Longs.fromByteArray(bcsId));
-        }
         containerSet.addContainer(kvContainer);
       } else {
         throw new StorageContainerException("Container File is corrupted. " +

http://git-wip-us.apache.org/repos/asf/hadoop/blob/22f37af9/hadoop-hdds/container-service/src/main/proto/StorageContainerDatanodeProtocol.proto
----------------------------------------------------------------------
diff --git 
a/hadoop-hdds/container-service/src/main/proto/StorageContainerDatanodeProtocol.proto
 
b/hadoop-hdds/container-service/src/main/proto/StorageContainerDatanodeProtocol.proto
index f8fb32d..982029c 100644
--- 
a/hadoop-hdds/container-service/src/main/proto/StorageContainerDatanodeProtocol.proto
+++ 
b/hadoop-hdds/container-service/src/main/proto/StorageContainerDatanodeProtocol.proto
@@ -215,7 +215,6 @@ message ContainerInfo {
   optional string finalhash = 9;
   optional hadoop.hdds.LifeCycleState state = 10;
   optional int64 deleteTransactionId = 11;
-  optional uint64 blockCommitSequenceId = 12;
 }
 
 /*

http://git-wip-us.apache.org/repos/asf/hadoop/blob/22f37af9/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyLocationInfo.java
----------------------------------------------------------------------
diff --git 
a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyLocationInfo.java
 
b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyLocationInfo.java
index 9d54cea..ada3567 100644
--- 
a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyLocationInfo.java
+++ 
b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmKeyLocationInfo.java
@@ -78,10 +78,6 @@ public final class OmKeyLocationInfo {
     return offset;
   }
 
-  public long getBlockCommitSequenceId() {
-    return blockCommitSequenceId;
-  }
-
   /**
    * Builder of OmKeyLocationInfo.
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/22f37af9/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestBCSID.java
----------------------------------------------------------------------
diff --git 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestBCSID.java
 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestBCSID.java
deleted file mode 100644
index ed4629c..0000000
--- 
a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestBCSID.java
+++ /dev/null
@@ -1,147 +0,0 @@
-/**
- * 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.client.rpc;
-
-import org.apache.hadoop.hdds.client.ReplicationFactor;
-import org.apache.hadoop.hdds.client.ReplicationType;
-import org.apache.hadoop.hdds.conf.OzoneConfiguration;
-import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
-import org.apache.hadoop.ozone.MiniOzoneCluster;
-import org.apache.hadoop.ozone.client.ObjectStore;
-import org.apache.hadoop.ozone.client.OzoneClient;
-import org.apache.hadoop.ozone.client.OzoneClientFactory;
-import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
-import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
-import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
-import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
-import org.apache.hadoop.test.GenericTestUtils;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-
-import static org.apache.hadoop.hdds.HddsConfigKeys.
-    HDDS_COMMAND_STATUS_REPORT_INTERVAL;
-import static org.apache.hadoop.hdds.HddsConfigKeys.
-    HDDS_CONTAINER_REPORT_INTERVAL;
-import static org.apache.hadoop.hdds.scm.ScmConfigKeys.
-    HDDS_SCM_WATCHER_TIMEOUT;
-import static org.apache.hadoop.hdds.scm.ScmConfigKeys.
-    OZONE_SCM_STALENODE_INTERVAL;
-
-/**
- * Tests the validity BCSID of a container.
- */
-public class TestBCSID {
-
-  private static MiniOzoneCluster cluster;
-  private static OzoneConfiguration conf;
-  private static OzoneClient client;
-  private static ObjectStore objectStore;
-  private static String volumeName;
-  private static String bucketName;
-
-  /**
-   * Create a MiniDFSCluster for testing.
-   *
-   * @throws IOException
-   */
-  @BeforeClass
-  public static void init() throws Exception {
-    conf = new OzoneConfiguration();
-    String path = GenericTestUtils
-        .getTempPath(TestBCSID.class.getSimpleName());
-    File baseDir = new File(path);
-    baseDir.mkdirs();
-
-    conf.setTimeDuration(HDDS_CONTAINER_REPORT_INTERVAL, 200,
-        TimeUnit.MILLISECONDS);
-    conf.setTimeDuration(HDDS_COMMAND_STATUS_REPORT_INTERVAL, 200,
-        TimeUnit.MILLISECONDS);
-    conf.setTimeDuration(HDDS_SCM_WATCHER_TIMEOUT, 1000, 
TimeUnit.MILLISECONDS);
-    conf.setTimeDuration(OZONE_SCM_STALENODE_INTERVAL, 3, TimeUnit.SECONDS);
-    conf.setQuietMode(false);
-    cluster =
-        MiniOzoneCluster.newBuilder(conf).setNumDatanodes(1).setHbInterval(200)
-            .build();
-    cluster.waitForClusterToBeReady();
-    //the easiest way to create an open container is creating a key
-    client = OzoneClientFactory.getClient(conf);
-    objectStore = client.getObjectStore();
-    volumeName = "bcsid";
-    bucketName = volumeName;
-    objectStore.createVolume(volumeName);
-    objectStore.getVolume(volumeName).createBucket(bucketName);
-  }
-
-  /**
-   * Shutdown MiniDFSCluster.
-   */
-  @AfterClass
-  public static void shutdown() {
-    if (cluster != null) {
-      cluster.shutdown();
-    }
-  }
-
-  @Test
-  public void testBCSID() throws Exception {
-    OzoneOutputStream key =
-        objectStore.getVolume(volumeName).getBucket(bucketName)
-            .createKey("ratis", 1024, ReplicationType.RATIS,
-                ReplicationFactor.ONE);
-    key.write("ratis".getBytes());
-    key.close();
-
-    // get the name of a valid container.
-    OmKeyArgs keyArgs = new OmKeyArgs.Builder().setVolumeName(volumeName).
-        setBucketName(bucketName).setType(HddsProtos.ReplicationType.RATIS)
-        .setFactor(HddsProtos.ReplicationFactor.ONE).setKeyName("ratis")
-        .build();
-    OmKeyInfo keyInfo = cluster.getOzoneManager().lookupKey(keyArgs);
-    List<OmKeyLocationInfo> keyLocationInfos =
-        keyInfo.getKeyLocationVersions().get(0).getBlocksLatestVersionOnly();
-    Assert.assertEquals(1, keyLocationInfos.size());
-    OmKeyLocationInfo omKeyLocationInfo = keyLocationInfos.get(0);
-
-    long blockCommitSequenceId =
-        cluster.getHddsDatanodes().get(0).getDatanodeStateMachine()
-            .getContainer().getContainerSet()
-            .getContainer(omKeyLocationInfo.getContainerID())
-            .getContainerReport().getBlockCommitSequenceId();
-    Assert.assertTrue(blockCommitSequenceId > 0);
-
-    // make sure the persisted block Id in OM is same as that seen in the
-    // container report to be reported to SCM.
-    Assert.assertEquals(blockCommitSequenceId,
-        omKeyLocationInfo.getBlockCommitSequenceId());
-
-    // verify that on restarting the datanode, it reloads the BCSID correctly.
-    cluster.restartHddsDatanode(0);
-    Assert.assertEquals(blockCommitSequenceId,
-        cluster.getHddsDatanodes().get(0).getDatanodeStateMachine()
-            .getContainer().getContainerSet()
-            .getContainer(omKeyLocationInfo.getContainerID())
-            .getContainerReport().getBlockCommitSequenceId());
-  }
-}
\ No newline at end of file


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

Reply via email to