Repository: hadoop
Updated Branches:
  refs/heads/ozone-0.3 abfff4ccc -> b51bc6b93


HDDS-603. Add BlockCommitSequenceId field per Container and expose it in 
Container Reports. Contributed by Shashikant Banerjee.


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

Branch: refs/heads/ozone-0.3
Commit: 0008357940d4bdb4c9162e15a0967bd287999a97
Parents: abfff4c
Author: Jitendra Pandey <jiten...@apache.org>
Authored: Sat Oct 13 12:15:42 2018 -0700
Committer: Jitendra Pandey <jiten...@apache.org>
Committed: Mon Oct 15 13:30: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, 208 insertions(+), 328 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/00083579/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 923271c..8ccc648 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,6 +113,7 @@ 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/00083579/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 a3430f8..04c87ae 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,7 +42,8 @@ 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.DELETE_TRANSACTION_KEY_PREFIX, true)
+          .addFilter(OzoneConsts.BLOCK_COMMIT_SEQUENCE_ID_PREFIX, true);
 
   private MetadataKeyFilters() {
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/00083579/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
deleted file mode 100644
index a4c1f2f..0000000
--- 
a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerReport.java
+++ /dev/null
@@ -1,205 +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
- *
- *      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/00083579/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
deleted file mode 100644
index b03487b..0000000
--- 
a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/KeyValueContainerReport.java
+++ /dev/null
@@ -1,117 +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
- *
- *      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/00083579/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 9380f0c..dbef74c 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,4 +132,9 @@ 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/00083579/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 e5b344d..9a5c94c 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,6 +508,12 @@ public class KeyValueContainer implements 
Container<KeyValueContainerData> {
         .getContainerID() + OzoneConsts.CONTAINER_EXTENSION);
   }
 
+  @Override
+  public void updateBlockCommitSequenceId(long blockCommitSequenceId) {
+    containerData.updateBlockCommitSequenceId(blockCommitSequenceId);
+  }
+
+
   /**
    * Returns KeyValueContainerReport for the KeyValueContainer.
    */
@@ -524,7 +530,8 @@ public class KeyValueContainer implements 
Container<KeyValueContainerData> {
         .setKeyCount(containerData.getKeyCount())
         .setUsed(containerData.getBytesUsed())
         .setState(getHddsState())
-        .setDeleteTransactionId(containerData.getDeleteTransactionId());
+        .setDeleteTransactionId(containerData.getDeleteTransactionId())
+        .setBlockCommitSequenceId(containerData.getBlockCommitSequenceId());
     return ciBuilder.build();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/00083579/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 7ffdbf5..9ea84c2 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,6 +72,8 @@ public class KeyValueContainerData extends ContainerData {
 
   private long deleteTransactionId;
 
+  private long blockCommitSequenceId;
+
   static {
     // Initialize YAML fields
     KV_YAML_FIELDS = Lists.newArrayList();
@@ -152,6 +154,20 @@ 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/00083579/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 0dd8739..bd19441 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,12 +25,16 @@ 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;
@@ -83,8 +87,16 @@ 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");
-    db.put(Longs.toByteArray(data.getLocalID()), data.getProtoBufMessage()
-        .toByteArray());
+
+    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);
     // Increment keycount here
     container.getContainerData().incrKeyCount();
     return data.getSize();
@@ -208,8 +220,9 @@ 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, null);
+    List<Map.Entry<byte[], byte[]>> range =
+        db.getSequentialRangeKVs(startKeyInBytes, count,
+            MetadataKeyFilters.getNormalKeyFilter());
     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/00083579/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 c3a4126..aee5775 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,6 +191,13 @@ 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/00083579/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 982029c..f8fb32d 100644
--- 
a/hadoop-hdds/container-service/src/main/proto/StorageContainerDatanodeProtocol.proto
+++ 
b/hadoop-hdds/container-service/src/main/proto/StorageContainerDatanodeProtocol.proto
@@ -215,6 +215,7 @@ 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/00083579/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 ada3567..9d54cea 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,6 +78,10 @@ public final class OmKeyLocationInfo {
     return offset;
   }
 
+  public long getBlockCommitSequenceId() {
+    return blockCommitSequenceId;
+  }
+
   /**
    * Builder of OmKeyLocationInfo.
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/00083579/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
new file mode 100644
index 0000000..ed4629c
--- /dev/null
+++ 
b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestBCSID.java
@@ -0,0 +1,147 @@
+/**
+ * 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