You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ozone.apache.org by GitBox <gi...@apache.org> on 2020/10/28 13:43:40 UTC

[GitHub] [ozone] rakeshadr opened a new pull request #1528: HDDS-4357: Rename : make rename an atomic ops by updating key path entry in dir/file table

rakeshadr opened a new pull request #1528:
URL: https://github.com/apache/ozone/pull/1528


   
   ## What changes were proposed in this pull request?
   
   https://issues.apache.org/jira/browse/HDDS-4357
   
   ## What is the link to the Apache JIRA
   
   This task is to handle rename key path request and make it an atomic operation by updating the DirTable or FileTable.
   
   ## How was this patch tested?
   
   Added TestOzoneFileSystem UT. 
   
   Note: Few refactoring has to be done once HDDS-4332 getFileStatus/ListStatus API is committed to the branch.
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [ozone] rakeshadr commented on a change in pull request #1528: HDDS-4357: Rename : make rename an atomic ops by updating key path entry in dir/file table

Posted by GitBox <gi...@apache.org>.
rakeshadr commented on a change in pull request #1528:
URL: https://github.com/apache/ozone/pull/1528#discussion_r518642991



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMFileRequest.java
##########
@@ -583,4 +589,180 @@ public static OmKeyInfo getOmKeyInfoFromFileTable(boolean openFileTable,
     return dbOmKeyInfo;
   }
 
+  /**
+   * Gets OmKeyInfo if exists for the given key name in the DB.
+   *
+   * @param omMetadataMgr metadata manager
+   * @param volumeName    volume name
+   * @param bucketName    bucket name
+   * @param keyName       key name
+   * @param scmBlockSize  scm block size
+   * @return OzoneFileStatus
+   * @throws IOException DB failure
+   */
+  @Nullable
+  public static OzoneFileStatus getOMKeyInfoIfExists(
+      OMMetadataManager omMetadataMgr, String volumeName, String bucketName,
+      String keyName, long scmBlockSize) throws IOException {
+
+    Path keyPath = Paths.get(keyName);
+    Iterator<Path> elements = keyPath.iterator();
+    String bucketKey = omMetadataMgr.getBucketKey(volumeName, bucketName);
+    OmBucketInfo omBucketInfo =
+            omMetadataMgr.getBucketTable().get(bucketKey);
+
+    long lastKnownParentId = omBucketInfo.getObjectID();
+    OmDirectoryInfo omDirInfo = null;
+    while (elements.hasNext()) {
+      String fileName = elements.next().toString();
+
+      // For example, /vol1/buck1/a/b/c/d/e/file1.txt
+      // 1. Do lookup path component on directoryTable starting from bucket
+      // 'buck1' to the leaf node component, which is 'file1.txt'.
+      // 2. If there is no dir exists for the leaf node component 'file1.txt'
+      // then do look it on fileTable.
+      String dbNodeName = omMetadataMgr.getOzonePathKey(
+              lastKnownParentId, fileName);
+      omDirInfo = omMetadataMgr.getDirectoryTable().get(dbNodeName);
+
+      if (omDirInfo != null) {
+        lastKnownParentId = omDirInfo.getObjectID();
+      } else if (!elements.hasNext()) {
+        // reached last path component. Check file exists for the given path.
+        OmKeyInfo omKeyInfo = OMFileRequest.getOmKeyInfoFromFileTable(false,
+                omMetadataMgr, dbNodeName, keyName);
+        if (omKeyInfo != null) {
+          return new OzoneFileStatus(omKeyInfo, scmBlockSize, false);
+        }
+      } else {
+        // Missing intermediate directory and just return null;
+        // key not found in DB
+        return null;
+      }
+    }
+
+    if (omDirInfo != null) {
+      OmKeyInfo omKeyInfo = getOmKeyInfo(volumeName, bucketName, omDirInfo,
+              keyName);
+      return new OzoneFileStatus(omKeyInfo, scmBlockSize, true);
+    }
+
+    // key not found in DB
+    return null;
+  }
+
+  /**
+   * Prepare OmKeyInfo from OmDirectoryInfo.
+   *
+   * @param volumeName volume name
+   * @param bucketName bucket name
+   * @param dirInfo    directory info
+   * @param keyName    user given key name
+   * @return OmKeyInfo object
+   */
+  @NotNull
+  public static OmKeyInfo getOmKeyInfo(String volumeName, String bucketName,
+      OmDirectoryInfo dirInfo, String keyName) {
+
+    OmKeyInfo.Builder builder = new OmKeyInfo.Builder();
+    builder.setParentObjectID(dirInfo.getParentObjectID());
+    builder.setKeyName(keyName);
+    builder.setAcls(dirInfo.getAcls());
+    builder.addAllMetadata(dirInfo.getMetadata());
+    builder.setVolumeName(volumeName);
+    builder.setBucketName(bucketName);
+    builder.setCreationTime(dirInfo.getCreationTime());
+    builder.setModificationTime(dirInfo.getModificationTime());
+    builder.setObjectID(dirInfo.getObjectID());
+    builder.setUpdateID(dirInfo.getUpdateID());
+    builder.setFileName(dirInfo.getName());
+    builder.setReplicationType(HddsProtos.ReplicationType.RATIS);
+    builder.setReplicationFactor(HddsProtos.ReplicationFactor.ONE);
+    builder.setOmKeyLocationInfos(Collections.singletonList(
+            new OmKeyLocationInfoGroup(0, new ArrayList<>())));
+    return builder.build();
+  }
+
+  /**
+   * Build DirectoryInfo from OmKeyInfo.
+   *
+   * @param keyInfo omKeyInfo
+   * @return omDirectoryInfo object
+   */
+  public static OmDirectoryInfo getDirectoryInfo(OmKeyInfo keyInfo){
+    OmDirectoryInfo.Builder builder = new OmDirectoryInfo.Builder();
+    builder.setParentObjectID(keyInfo.getParentObjectID());
+    builder.setAcls(keyInfo.getAcls());
+    builder.addAllMetadata(keyInfo.getMetadata());
+    builder.setCreationTime(keyInfo.getCreationTime());
+    builder.setModificationTime(keyInfo.getModificationTime());
+    builder.setObjectID(keyInfo.getObjectID());
+    builder.setUpdateID(keyInfo.getUpdateID());
+    builder.setName(OzoneFSUtils.getFileName(keyInfo.getKeyName()));
+    return builder.build();
+  }
+
+  /**
+   * Verify that the given toKey directory is a sub directory of fromKey
+   * directory.
+   * <p>
+   * For example, special case of renaming a directory to its own
+   * sub-directory is not allowed.
+   *
+   * @param fromKeyName source path
+   * @param toKeyName destination path
+   */
+  public static void verifyToDirIsASubDirOfFromDirectory(String fromKeyName,
+      String toKeyName, boolean isDir){
+    if (!isDir) {
+      return;
+    }
+    Path dstParent = Paths.get(toKeyName).getParent();
+    while (dstParent != null && !Paths.get(fromKeyName).equals(dstParent)) {
+      dstParent = dstParent.getParent();
+    }
+    Preconditions.checkArgument(dstParent == null,
+            "Cannot rename a directory to its own subdirectory");
+    return;
+  }
+
+  /**
+   * Verify parent exists for the destination path and return destination
+   * path parent Id.
+   * <p>
+   * Check whether dst parent dir exists or not. If the parent exists, then the
+   * source can be renamed to dst path.
+   *
+   * @param volumeName  volume name
+   * @param bucketName  bucket name
+   * @param toKeyName   destination path
+   * @param fromKeyName source path
+   * @param metaMgr     metadata manager
+   * @throws IOException if the destination parent dir doesn't exists.
+   */
+  public static long getToKeyNameParentId(String volumeName,
+      String bucketName, String toKeyName, String fromKeyName,
+      OMMetadataManager metaMgr) throws IOException {
+
+    int totalDirsCount = OzoneFSUtils.getFileCount(toKeyName);
+    // skip parent is root '/'
+    if (totalDirsCount <= 1) {
+      String bucketKey = metaMgr.getBucketKey(volumeName, bucketName);
+      OmBucketInfo omBucketInfo =
+              metaMgr.getBucketTable().get(bucketKey);
+      return omBucketInfo.getObjectID();
+    }
+
+    String toKeyParentDir = OzoneFSUtils.getParentDir(toKeyName);
+
+    OzoneFileStatus toKeyParentDirStatus = getOMKeyInfoIfExists(metaMgr,
+            volumeName, bucketName, toKeyParentDir, 0);
+    // check if the immediate parent exists
+    if (toKeyParentDirStatus == null) {

Review comment:
       Sure, I will do it in new commit




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [ozone] rakeshadr commented on a change in pull request #1528: HDDS-4357: Rename : make rename an atomic ops by updating key path entry in dir/file table

Posted by GitBox <gi...@apache.org>.
rakeshadr commented on a change in pull request #1528:
URL: https://github.com/apache/ozone/pull/1528#discussion_r518642276



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyRenameRequestV1.java
##########
@@ -0,0 +1,291 @@
+/**
+ * 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.om.request.key;
+
+import com.google.common.base.Optional;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.cache.CacheKey;
+import org.apache.hadoop.hdds.utils.db.cache.CacheValue;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.audit.AuditLogger;
+import org.apache.hadoop.ozone.audit.OMAction;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OMMetrics;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OzoneFSUtils;
+import org.apache.hadoop.ozone.om.helpers.OzoneFileStatus;
+import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.file.OMFileRequest;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.key.OMKeyRenameResponse;
+import org.apache.hadoop.ozone.om.response.key.OMKeyRenameResponseV1;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.*;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Map;
+
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.KEY_NOT_FOUND;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK;
+
+/**
+ * Handles rename key request layout version V1.
+ */
+public class OMKeyRenameRequestV1 extends OMKeyRenameRequest {
+
+  private static final Logger LOG =
+          LoggerFactory.getLogger(OMKeyRenameRequestV1.class);
+
+  public OMKeyRenameRequestV1(OMRequest omRequest) {
+    super(omRequest);
+  }
+
+  @Override
+  @SuppressWarnings("methodlength")
+  public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
+      long trxnLogIndex, OzoneManagerDoubleBufferHelper omDoubleBufferHelper) {
+
+    RenameKeyRequest renameKeyRequest = getOmRequest().getRenameKeyRequest();
+    KeyArgs keyArgs = renameKeyRequest.getKeyArgs();
+    Map<String, String> auditMap = buildAuditMap(keyArgs, renameKeyRequest);
+
+    String volumeName = keyArgs.getVolumeName();
+    String bucketName = keyArgs.getBucketName();
+    String fromKeyName = keyArgs.getKeyName();
+    String toKeyName = renameKeyRequest.getToKeyName();
+
+    OMMetrics omMetrics = ozoneManager.getMetrics();
+    omMetrics.incNumKeyRenames();
+
+    AuditLogger auditLogger = ozoneManager.getAuditLogger();
+
+    OMResponse.Builder omResponse = OmResponseUtil.getOMResponseBuilder(
+            getOmRequest());
+
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+    boolean acquiredLock = false;
+    OMClientResponse omClientResponse = null;
+    IOException exception = null;
+    OmKeyInfo fromKeyValue;
+    String fromKey = null;
+    Result result;
+    try {
+      if (toKeyName.length() == 0 || fromKeyName.length() == 0) {
+        throw new OMException("Key name is empty",
+                OMException.ResultCodes.INVALID_KEY_NAME);
+      }
+
+      keyArgs = resolveBucketLink(ozoneManager, keyArgs, auditMap);
+      volumeName = keyArgs.getVolumeName();
+      bucketName = keyArgs.getBucketName();
+
+      // check Acls to see if user has access to perform delete operation on
+      // old key and create operation on new key
+      checkKeyAcls(ozoneManager, volumeName, bucketName, fromKeyName,
+              IAccessAuthorizer.ACLType.DELETE, OzoneObj.ResourceType.KEY);
+      checkKeyAcls(ozoneManager, volumeName, bucketName, toKeyName,
+              IAccessAuthorizer.ACLType.CREATE, OzoneObj.ResourceType.KEY);
+
+      acquiredLock = omMetadataManager.getLock().acquireWriteLock(BUCKET_LOCK,
+              volumeName, bucketName);
+
+      // Validate bucket and volume exists or not.
+      validateBucketAndVolume(omMetadataManager, volumeName, bucketName);
+
+      // Check if toKey exists
+      OzoneFileStatus fromKeyFileStatus =
+              OMFileRequest.getOMKeyInfoIfExists(omMetadataManager, volumeName,
+                      bucketName, fromKeyName, 0);
+      // fromKeyName should exist
+      if (fromKeyFileStatus == null) {
+        // TODO: Add support for renaming open key
+        throw new OMException("Key not found " + fromKey, KEY_NOT_FOUND);
+      }
+
+      // source exists
+      fromKeyValue = fromKeyFileStatus.getKeyInfo();
+      boolean isRenameDirectory = fromKeyFileStatus.isDirectory();
+
+      OzoneFileStatus toKeyFileStatus =
+              OMFileRequest.getOMKeyInfoIfExists(omMetadataManager,
+                      volumeName, bucketName, toKeyName, 0);
+      OmKeyInfo toKeyValue;
+
+      // Destination exists cases:
+      if(toKeyFileStatus != null) {
+
+        toKeyValue = toKeyFileStatus.getKeyInfo();
+
+        if (fromKeyValue.getKeyName().equals(toKeyValue.getKeyName())) {
+          // case-1) src == destin then check source and destin of same type
+          // If dst is a file then return true. Otherwise fail the operation.
+          if (!toKeyFileStatus.isDirectory()) {
+            result = Result.SUCCESS;
+          } else {
+            throw new OMException("Key already exists " + toKeyName,
+                    OMException.ResultCodes.KEY_ALREADY_EXISTS);
+          }
+        } else if (toKeyFileStatus.isDirectory()) {
+          // case-2) If dst is a directory, rename source as sub-path of it.
+          // For example: rename /source to /dst will lead to /dst/source
+          String fromFileName = OzoneFSUtils.getFileName(fromKeyName);

Review comment:
       Good catch. I will do this check.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [ozone] rakeshadr commented on a change in pull request #1528: HDDS-4357: Rename : make rename an atomic ops by updating key path entry in dir/file table

Posted by GitBox <gi...@apache.org>.
rakeshadr commented on a change in pull request #1528:
URL: https://github.com/apache/ozone/pull/1528#discussion_r518851589



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyRenameRequestV1.java
##########
@@ -0,0 +1,291 @@
+/**
+ * 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.om.request.key;
+
+import com.google.common.base.Optional;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.cache.CacheKey;
+import org.apache.hadoop.hdds.utils.db.cache.CacheValue;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.audit.AuditLogger;
+import org.apache.hadoop.ozone.audit.OMAction;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OMMetrics;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OzoneFSUtils;
+import org.apache.hadoop.ozone.om.helpers.OzoneFileStatus;
+import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.file.OMFileRequest;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.key.OMKeyRenameResponse;
+import org.apache.hadoop.ozone.om.response.key.OMKeyRenameResponseV1;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.*;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Map;
+
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.KEY_NOT_FOUND;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK;
+
+/**
+ * Handles rename key request layout version V1.
+ */
+public class OMKeyRenameRequestV1 extends OMKeyRenameRequest {
+
+  private static final Logger LOG =
+          LoggerFactory.getLogger(OMKeyRenameRequestV1.class);
+
+  public OMKeyRenameRequestV1(OMRequest omRequest) {
+    super(omRequest);
+  }
+
+  @Override
+  @SuppressWarnings("methodlength")
+  public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
+      long trxnLogIndex, OzoneManagerDoubleBufferHelper omDoubleBufferHelper) {
+
+    RenameKeyRequest renameKeyRequest = getOmRequest().getRenameKeyRequest();
+    KeyArgs keyArgs = renameKeyRequest.getKeyArgs();
+    Map<String, String> auditMap = buildAuditMap(keyArgs, renameKeyRequest);
+
+    String volumeName = keyArgs.getVolumeName();
+    String bucketName = keyArgs.getBucketName();
+    String fromKeyName = keyArgs.getKeyName();
+    String toKeyName = renameKeyRequest.getToKeyName();
+
+    OMMetrics omMetrics = ozoneManager.getMetrics();
+    omMetrics.incNumKeyRenames();
+
+    AuditLogger auditLogger = ozoneManager.getAuditLogger();
+
+    OMResponse.Builder omResponse = OmResponseUtil.getOMResponseBuilder(
+            getOmRequest());
+
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+    boolean acquiredLock = false;
+    OMClientResponse omClientResponse = null;
+    IOException exception = null;
+    OmKeyInfo fromKeyValue;
+    String fromKey = null;
+    Result result;
+    try {
+      if (toKeyName.length() == 0 || fromKeyName.length() == 0) {
+        throw new OMException("Key name is empty",
+                OMException.ResultCodes.INVALID_KEY_NAME);
+      }
+
+      keyArgs = resolveBucketLink(ozoneManager, keyArgs, auditMap);
+      volumeName = keyArgs.getVolumeName();
+      bucketName = keyArgs.getBucketName();
+
+      // check Acls to see if user has access to perform delete operation on
+      // old key and create operation on new key
+      checkKeyAcls(ozoneManager, volumeName, bucketName, fromKeyName,
+              IAccessAuthorizer.ACLType.DELETE, OzoneObj.ResourceType.KEY);
+      checkKeyAcls(ozoneManager, volumeName, bucketName, toKeyName,
+              IAccessAuthorizer.ACLType.CREATE, OzoneObj.ResourceType.KEY);
+
+      acquiredLock = omMetadataManager.getLock().acquireWriteLock(BUCKET_LOCK,
+              volumeName, bucketName);
+
+      // Validate bucket and volume exists or not.
+      validateBucketAndVolume(omMetadataManager, volumeName, bucketName);
+
+      // Check if toKey exists
+      OzoneFileStatus fromKeyFileStatus =
+              OMFileRequest.getOMKeyInfoIfExists(omMetadataManager, volumeName,
+                      bucketName, fromKeyName, 0);
+      // fromKeyName should exist
+      if (fromKeyFileStatus == null) {
+        // TODO: Add support for renaming open key
+        throw new OMException("Key not found " + fromKey, KEY_NOT_FOUND);
+      }
+
+      // source exists
+      fromKeyValue = fromKeyFileStatus.getKeyInfo();
+      boolean isRenameDirectory = fromKeyFileStatus.isDirectory();
+
+      OzoneFileStatus toKeyFileStatus =
+              OMFileRequest.getOMKeyInfoIfExists(omMetadataManager,
+                      volumeName, bucketName, toKeyName, 0);
+      OmKeyInfo toKeyValue;
+
+      // Destination exists cases:
+      if(toKeyFileStatus != null) {
+
+        toKeyValue = toKeyFileStatus.getKeyInfo();
+
+        if (fromKeyValue.getKeyName().equals(toKeyValue.getKeyName())) {
+          // case-1) src == destin then check source and destin of same type
+          // If dst is a file then return true. Otherwise fail the operation.
+          if (!toKeyFileStatus.isDirectory()) {

Review comment:
       Thanks for the confirmation. As I said, I have created new PR #1557. It is more or less same logic and done following things:
   1) Addressed your comments
   2) Resolved merge conflicts
   
   Could you please review the changes when you get a chance. Thanks!




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [ozone] linyiqun commented on pull request #1528: HDDS-4357: Rename : make rename an atomic ops by updating key path entry in dir/file table

Posted by GitBox <gi...@apache.org>.
linyiqun commented on pull request #1528:
URL: https://github.com/apache/ozone/pull/1528#issuecomment-722987584


   >... Please go through my reply.
   
   I didn't see your reply, where is your reply for my review comments?
   
   @rakeshadr , current latest commit include many previous commits? Would you update with the latest commit as HDDS-4332 already merged.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [ozone] linyiqun edited a comment on pull request #1528: HDDS-4357: Rename : make rename an atomic ops by updating key path entry in dir/file table

Posted by GitBox <gi...@apache.org>.
linyiqun edited a comment on pull request #1528:
URL: https://github.com/apache/ozone/pull/1528#issuecomment-722987584


   >... Please go through my reply.
   
   I didn't see your reply, do you commit the comments?
   
   @rakeshadr , current latest commit include many previous commits? Would you update with the latest commit as HDDS-4332 already merged.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [ozone] rakeshadr commented on a change in pull request #1528: HDDS-4357: Rename : make rename an atomic ops by updating key path entry in dir/file table

Posted by GitBox <gi...@apache.org>.
rakeshadr commented on a change in pull request #1528:
URL: https://github.com/apache/ozone/pull/1528#discussion_r518642786



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyRenameRequestV1.java
##########
@@ -0,0 +1,291 @@
+/**
+ * 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.om.request.key;
+
+import com.google.common.base.Optional;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.cache.CacheKey;
+import org.apache.hadoop.hdds.utils.db.cache.CacheValue;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.audit.AuditLogger;
+import org.apache.hadoop.ozone.audit.OMAction;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OMMetrics;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OzoneFSUtils;
+import org.apache.hadoop.ozone.om.helpers.OzoneFileStatus;
+import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.file.OMFileRequest;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.key.OMKeyRenameResponse;
+import org.apache.hadoop.ozone.om.response.key.OMKeyRenameResponseV1;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.*;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Map;
+
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.KEY_NOT_FOUND;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK;
+
+/**
+ * Handles rename key request layout version V1.
+ */
+public class OMKeyRenameRequestV1 extends OMKeyRenameRequest {
+
+  private static final Logger LOG =
+          LoggerFactory.getLogger(OMKeyRenameRequestV1.class);
+
+  public OMKeyRenameRequestV1(OMRequest omRequest) {
+    super(omRequest);
+  }
+
+  @Override
+  @SuppressWarnings("methodlength")
+  public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
+      long trxnLogIndex, OzoneManagerDoubleBufferHelper omDoubleBufferHelper) {
+
+    RenameKeyRequest renameKeyRequest = getOmRequest().getRenameKeyRequest();
+    KeyArgs keyArgs = renameKeyRequest.getKeyArgs();
+    Map<String, String> auditMap = buildAuditMap(keyArgs, renameKeyRequest);
+
+    String volumeName = keyArgs.getVolumeName();
+    String bucketName = keyArgs.getBucketName();
+    String fromKeyName = keyArgs.getKeyName();
+    String toKeyName = renameKeyRequest.getToKeyName();
+
+    OMMetrics omMetrics = ozoneManager.getMetrics();
+    omMetrics.incNumKeyRenames();
+
+    AuditLogger auditLogger = ozoneManager.getAuditLogger();
+
+    OMResponse.Builder omResponse = OmResponseUtil.getOMResponseBuilder(
+            getOmRequest());
+
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+    boolean acquiredLock = false;
+    OMClientResponse omClientResponse = null;
+    IOException exception = null;
+    OmKeyInfo fromKeyValue;
+    String fromKey = null;
+    Result result;
+    try {
+      if (toKeyName.length() == 0 || fromKeyName.length() == 0) {
+        throw new OMException("Key name is empty",
+                OMException.ResultCodes.INVALID_KEY_NAME);
+      }
+
+      keyArgs = resolveBucketLink(ozoneManager, keyArgs, auditMap);
+      volumeName = keyArgs.getVolumeName();
+      bucketName = keyArgs.getBucketName();
+
+      // check Acls to see if user has access to perform delete operation on
+      // old key and create operation on new key
+      checkKeyAcls(ozoneManager, volumeName, bucketName, fromKeyName,
+              IAccessAuthorizer.ACLType.DELETE, OzoneObj.ResourceType.KEY);
+      checkKeyAcls(ozoneManager, volumeName, bucketName, toKeyName,
+              IAccessAuthorizer.ACLType.CREATE, OzoneObj.ResourceType.KEY);
+
+      acquiredLock = omMetadataManager.getLock().acquireWriteLock(BUCKET_LOCK,
+              volumeName, bucketName);
+
+      // Validate bucket and volume exists or not.
+      validateBucketAndVolume(omMetadataManager, volumeName, bucketName);
+
+      // Check if toKey exists
+      OzoneFileStatus fromKeyFileStatus =
+              OMFileRequest.getOMKeyInfoIfExists(omMetadataManager, volumeName,
+                      bucketName, fromKeyName, 0);
+      // fromKeyName should exist
+      if (fromKeyFileStatus == null) {
+        // TODO: Add support for renaming open key
+        throw new OMException("Key not found " + fromKey, KEY_NOT_FOUND);
+      }
+
+      // source exists
+      fromKeyValue = fromKeyFileStatus.getKeyInfo();
+      boolean isRenameDirectory = fromKeyFileStatus.isDirectory();
+
+      OzoneFileStatus toKeyFileStatus =
+              OMFileRequest.getOMKeyInfoIfExists(omMetadataManager,
+                      volumeName, bucketName, toKeyName, 0);
+      OmKeyInfo toKeyValue;
+
+      // Destination exists cases:
+      if(toKeyFileStatus != null) {
+
+        toKeyValue = toKeyFileStatus.getKeyInfo();
+
+        if (fromKeyValue.getKeyName().equals(toKeyValue.getKeyName())) {
+          // case-1) src == destin then check source and destin of same type
+          // If dst is a file then return true. Otherwise fail the operation.
+          if (!toKeyFileStatus.isDirectory()) {

Review comment:
       I've referred following code in [BasicOzoneFileSystem.java#L369](https://github.com/apache/ozone/blob/master/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicOzoneFileSystem.java#L369) and IIUC, it is returning false for a directory and true for a file.
   
      Appreciate if you could point me to the code you are referring to understand the case well. Thanks!




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [ozone] linyiqun commented on a change in pull request #1528: HDDS-4357: Rename : make rename an atomic ops by updating key path entry in dir/file table

Posted by GitBox <gi...@apache.org>.
linyiqun commented on a change in pull request #1528:
URL: https://github.com/apache/ozone/pull/1528#discussion_r519097524



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyRenameRequestV1.java
##########
@@ -0,0 +1,291 @@
+/**
+ * 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.om.request.key;
+
+import com.google.common.base.Optional;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.cache.CacheKey;
+import org.apache.hadoop.hdds.utils.db.cache.CacheValue;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.audit.AuditLogger;
+import org.apache.hadoop.ozone.audit.OMAction;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OMMetrics;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OzoneFSUtils;
+import org.apache.hadoop.ozone.om.helpers.OzoneFileStatus;
+import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.file.OMFileRequest;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.key.OMKeyRenameResponse;
+import org.apache.hadoop.ozone.om.response.key.OMKeyRenameResponseV1;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.*;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Map;
+
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.KEY_NOT_FOUND;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK;
+
+/**
+ * Handles rename key request layout version V1.
+ */
+public class OMKeyRenameRequestV1 extends OMKeyRenameRequest {
+
+  private static final Logger LOG =
+          LoggerFactory.getLogger(OMKeyRenameRequestV1.class);
+
+  public OMKeyRenameRequestV1(OMRequest omRequest) {
+    super(omRequest);
+  }
+
+  @Override
+  @SuppressWarnings("methodlength")
+  public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
+      long trxnLogIndex, OzoneManagerDoubleBufferHelper omDoubleBufferHelper) {
+
+    RenameKeyRequest renameKeyRequest = getOmRequest().getRenameKeyRequest();
+    KeyArgs keyArgs = renameKeyRequest.getKeyArgs();
+    Map<String, String> auditMap = buildAuditMap(keyArgs, renameKeyRequest);
+
+    String volumeName = keyArgs.getVolumeName();
+    String bucketName = keyArgs.getBucketName();
+    String fromKeyName = keyArgs.getKeyName();
+    String toKeyName = renameKeyRequest.getToKeyName();
+
+    OMMetrics omMetrics = ozoneManager.getMetrics();
+    omMetrics.incNumKeyRenames();
+
+    AuditLogger auditLogger = ozoneManager.getAuditLogger();
+
+    OMResponse.Builder omResponse = OmResponseUtil.getOMResponseBuilder(
+            getOmRequest());
+
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+    boolean acquiredLock = false;
+    OMClientResponse omClientResponse = null;
+    IOException exception = null;
+    OmKeyInfo fromKeyValue;
+    String fromKey = null;
+    Result result;
+    try {
+      if (toKeyName.length() == 0 || fromKeyName.length() == 0) {
+        throw new OMException("Key name is empty",
+                OMException.ResultCodes.INVALID_KEY_NAME);
+      }
+
+      keyArgs = resolveBucketLink(ozoneManager, keyArgs, auditMap);
+      volumeName = keyArgs.getVolumeName();
+      bucketName = keyArgs.getBucketName();
+
+      // check Acls to see if user has access to perform delete operation on
+      // old key and create operation on new key
+      checkKeyAcls(ozoneManager, volumeName, bucketName, fromKeyName,
+              IAccessAuthorizer.ACLType.DELETE, OzoneObj.ResourceType.KEY);
+      checkKeyAcls(ozoneManager, volumeName, bucketName, toKeyName,
+              IAccessAuthorizer.ACLType.CREATE, OzoneObj.ResourceType.KEY);
+
+      acquiredLock = omMetadataManager.getLock().acquireWriteLock(BUCKET_LOCK,
+              volumeName, bucketName);
+
+      // Validate bucket and volume exists or not.
+      validateBucketAndVolume(omMetadataManager, volumeName, bucketName);
+
+      // Check if toKey exists
+      OzoneFileStatus fromKeyFileStatus =
+              OMFileRequest.getOMKeyInfoIfExists(omMetadataManager, volumeName,
+                      bucketName, fromKeyName, 0);
+      // fromKeyName should exist
+      if (fromKeyFileStatus == null) {
+        // TODO: Add support for renaming open key
+        throw new OMException("Key not found " + fromKey, KEY_NOT_FOUND);
+      }
+
+      // source exists
+      fromKeyValue = fromKeyFileStatus.getKeyInfo();
+      boolean isRenameDirectory = fromKeyFileStatus.isDirectory();
+
+      OzoneFileStatus toKeyFileStatus =
+              OMFileRequest.getOMKeyInfoIfExists(omMetadataManager,
+                      volumeName, bucketName, toKeyName, 0);
+      OmKeyInfo toKeyValue;
+
+      // Destination exists cases:
+      if(toKeyFileStatus != null) {
+
+        toKeyValue = toKeyFileStatus.getKeyInfo();
+
+        if (fromKeyValue.getKeyName().equals(toKeyValue.getKeyName())) {
+          // case-1) src == destin then check source and destin of same type
+          // If dst is a file then return true. Otherwise fail the operation.
+          if (!toKeyFileStatus.isDirectory()) {

Review comment:
       Sure, have taken the review, : ).




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [ozone] linyiqun commented on a change in pull request #1528: HDDS-4357: Rename : make rename an atomic ops by updating key path entry in dir/file table

Posted by GitBox <gi...@apache.org>.
linyiqun commented on a change in pull request #1528:
URL: https://github.com/apache/ozone/pull/1528#discussion_r514214305



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyRenameRequestV1.java
##########
@@ -0,0 +1,291 @@
+/**
+ * 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.om.request.key;
+
+import com.google.common.base.Optional;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.cache.CacheKey;
+import org.apache.hadoop.hdds.utils.db.cache.CacheValue;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.audit.AuditLogger;
+import org.apache.hadoop.ozone.audit.OMAction;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OMMetrics;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OzoneFSUtils;
+import org.apache.hadoop.ozone.om.helpers.OzoneFileStatus;
+import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.file.OMFileRequest;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.key.OMKeyRenameResponse;
+import org.apache.hadoop.ozone.om.response.key.OMKeyRenameResponseV1;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.*;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Map;
+
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.KEY_NOT_FOUND;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK;
+
+/**
+ * Handles rename key request layout version V1.
+ */
+public class OMKeyRenameRequestV1 extends OMKeyRenameRequest {
+
+  private static final Logger LOG =
+          LoggerFactory.getLogger(OMKeyRenameRequestV1.class);
+
+  public OMKeyRenameRequestV1(OMRequest omRequest) {
+    super(omRequest);
+  }
+
+  @Override
+  @SuppressWarnings("methodlength")
+  public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
+      long trxnLogIndex, OzoneManagerDoubleBufferHelper omDoubleBufferHelper) {
+
+    RenameKeyRequest renameKeyRequest = getOmRequest().getRenameKeyRequest();
+    KeyArgs keyArgs = renameKeyRequest.getKeyArgs();
+    Map<String, String> auditMap = buildAuditMap(keyArgs, renameKeyRequest);
+
+    String volumeName = keyArgs.getVolumeName();
+    String bucketName = keyArgs.getBucketName();
+    String fromKeyName = keyArgs.getKeyName();
+    String toKeyName = renameKeyRequest.getToKeyName();
+
+    OMMetrics omMetrics = ozoneManager.getMetrics();
+    omMetrics.incNumKeyRenames();
+
+    AuditLogger auditLogger = ozoneManager.getAuditLogger();
+
+    OMResponse.Builder omResponse = OmResponseUtil.getOMResponseBuilder(
+            getOmRequest());
+
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+    boolean acquiredLock = false;
+    OMClientResponse omClientResponse = null;
+    IOException exception = null;
+    OmKeyInfo fromKeyValue;
+    String fromKey = null;
+    Result result;
+    try {
+      if (toKeyName.length() == 0 || fromKeyName.length() == 0) {
+        throw new OMException("Key name is empty",
+                OMException.ResultCodes.INVALID_KEY_NAME);
+      }
+
+      keyArgs = resolveBucketLink(ozoneManager, keyArgs, auditMap);
+      volumeName = keyArgs.getVolumeName();
+      bucketName = keyArgs.getBucketName();
+
+      // check Acls to see if user has access to perform delete operation on
+      // old key and create operation on new key
+      checkKeyAcls(ozoneManager, volumeName, bucketName, fromKeyName,
+              IAccessAuthorizer.ACLType.DELETE, OzoneObj.ResourceType.KEY);
+      checkKeyAcls(ozoneManager, volumeName, bucketName, toKeyName,
+              IAccessAuthorizer.ACLType.CREATE, OzoneObj.ResourceType.KEY);
+
+      acquiredLock = omMetadataManager.getLock().acquireWriteLock(BUCKET_LOCK,
+              volumeName, bucketName);
+
+      // Validate bucket and volume exists or not.
+      validateBucketAndVolume(omMetadataManager, volumeName, bucketName);
+
+      // Check if toKey exists
+      OzoneFileStatus fromKeyFileStatus =
+              OMFileRequest.getOMKeyInfoIfExists(omMetadataManager, volumeName,
+                      bucketName, fromKeyName, 0);
+      // fromKeyName should exist
+      if (fromKeyFileStatus == null) {
+        // TODO: Add support for renaming open key
+        throw new OMException("Key not found " + fromKey, KEY_NOT_FOUND);
+      }
+
+      // source exists
+      fromKeyValue = fromKeyFileStatus.getKeyInfo();
+      boolean isRenameDirectory = fromKeyFileStatus.isDirectory();
+
+      OzoneFileStatus toKeyFileStatus =
+              OMFileRequest.getOMKeyInfoIfExists(omMetadataManager,
+                      volumeName, bucketName, toKeyName, 0);
+      OmKeyInfo toKeyValue;
+
+      // Destination exists cases:
+      if(toKeyFileStatus != null) {
+
+        toKeyValue = toKeyFileStatus.getKeyInfo();
+
+        if (fromKeyValue.getKeyName().equals(toKeyValue.getKeyName())) {
+          // case-1) src == destin then check source and destin of same type
+          // If dst is a file then return true. Otherwise fail the operation.
+          if (!toKeyFileStatus.isDirectory()) {

Review comment:
       should we directly throw error here no matter toKeyFileStatus is a directory or file. I see in KeyManagerImpl#renameKey, it just returns.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [ozone] linyiqun commented on a change in pull request #1528: HDDS-4357: Rename : make rename an atomic ops by updating key path entry in dir/file table

Posted by GitBox <gi...@apache.org>.
linyiqun commented on a change in pull request #1528:
URL: https://github.com/apache/ozone/pull/1528#discussion_r514214305



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyRenameRequestV1.java
##########
@@ -0,0 +1,291 @@
+/**
+ * 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.om.request.key;
+
+import com.google.common.base.Optional;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.cache.CacheKey;
+import org.apache.hadoop.hdds.utils.db.cache.CacheValue;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.audit.AuditLogger;
+import org.apache.hadoop.ozone.audit.OMAction;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OMMetrics;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OzoneFSUtils;
+import org.apache.hadoop.ozone.om.helpers.OzoneFileStatus;
+import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.file.OMFileRequest;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.key.OMKeyRenameResponse;
+import org.apache.hadoop.ozone.om.response.key.OMKeyRenameResponseV1;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.*;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Map;
+
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.KEY_NOT_FOUND;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK;
+
+/**
+ * Handles rename key request layout version V1.
+ */
+public class OMKeyRenameRequestV1 extends OMKeyRenameRequest {
+
+  private static final Logger LOG =
+          LoggerFactory.getLogger(OMKeyRenameRequestV1.class);
+
+  public OMKeyRenameRequestV1(OMRequest omRequest) {
+    super(omRequest);
+  }
+
+  @Override
+  @SuppressWarnings("methodlength")
+  public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
+      long trxnLogIndex, OzoneManagerDoubleBufferHelper omDoubleBufferHelper) {
+
+    RenameKeyRequest renameKeyRequest = getOmRequest().getRenameKeyRequest();
+    KeyArgs keyArgs = renameKeyRequest.getKeyArgs();
+    Map<String, String> auditMap = buildAuditMap(keyArgs, renameKeyRequest);
+
+    String volumeName = keyArgs.getVolumeName();
+    String bucketName = keyArgs.getBucketName();
+    String fromKeyName = keyArgs.getKeyName();
+    String toKeyName = renameKeyRequest.getToKeyName();
+
+    OMMetrics omMetrics = ozoneManager.getMetrics();
+    omMetrics.incNumKeyRenames();
+
+    AuditLogger auditLogger = ozoneManager.getAuditLogger();
+
+    OMResponse.Builder omResponse = OmResponseUtil.getOMResponseBuilder(
+            getOmRequest());
+
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+    boolean acquiredLock = false;
+    OMClientResponse omClientResponse = null;
+    IOException exception = null;
+    OmKeyInfo fromKeyValue;
+    String fromKey = null;
+    Result result;
+    try {
+      if (toKeyName.length() == 0 || fromKeyName.length() == 0) {
+        throw new OMException("Key name is empty",
+                OMException.ResultCodes.INVALID_KEY_NAME);
+      }
+
+      keyArgs = resolveBucketLink(ozoneManager, keyArgs, auditMap);
+      volumeName = keyArgs.getVolumeName();
+      bucketName = keyArgs.getBucketName();
+
+      // check Acls to see if user has access to perform delete operation on
+      // old key and create operation on new key
+      checkKeyAcls(ozoneManager, volumeName, bucketName, fromKeyName,
+              IAccessAuthorizer.ACLType.DELETE, OzoneObj.ResourceType.KEY);
+      checkKeyAcls(ozoneManager, volumeName, bucketName, toKeyName,
+              IAccessAuthorizer.ACLType.CREATE, OzoneObj.ResourceType.KEY);
+
+      acquiredLock = omMetadataManager.getLock().acquireWriteLock(BUCKET_LOCK,
+              volumeName, bucketName);
+
+      // Validate bucket and volume exists or not.
+      validateBucketAndVolume(omMetadataManager, volumeName, bucketName);
+
+      // Check if toKey exists
+      OzoneFileStatus fromKeyFileStatus =
+              OMFileRequest.getOMKeyInfoIfExists(omMetadataManager, volumeName,
+                      bucketName, fromKeyName, 0);
+      // fromKeyName should exist
+      if (fromKeyFileStatus == null) {
+        // TODO: Add support for renaming open key
+        throw new OMException("Key not found " + fromKey, KEY_NOT_FOUND);
+      }
+
+      // source exists
+      fromKeyValue = fromKeyFileStatus.getKeyInfo();
+      boolean isRenameDirectory = fromKeyFileStatus.isDirectory();
+
+      OzoneFileStatus toKeyFileStatus =
+              OMFileRequest.getOMKeyInfoIfExists(omMetadataManager,
+                      volumeName, bucketName, toKeyName, 0);
+      OmKeyInfo toKeyValue;
+
+      // Destination exists cases:
+      if(toKeyFileStatus != null) {
+
+        toKeyValue = toKeyFileStatus.getKeyInfo();
+
+        if (fromKeyValue.getKeyName().equals(toKeyValue.getKeyName())) {
+          // case-1) src == destin then check source and destin of same type
+          // If dst is a file then return true. Otherwise fail the operation.
+          if (!toKeyFileStatus.isDirectory()) {

Review comment:
       should we directory thrown error here no matter toKeyFileStatus is a directory or file. I see in KeyManagerImpl#renameKey, it just returns.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [ozone] rakeshadr closed pull request #1528: HDDS-4357: Rename : make rename an atomic ops by updating key path entry in dir/file table

Posted by GitBox <gi...@apache.org>.
rakeshadr closed pull request #1528:
URL: https://github.com/apache/ozone/pull/1528


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [ozone] linyiqun commented on a change in pull request #1528: HDDS-4357: Rename : make rename an atomic ops by updating key path entry in dir/file table

Posted by GitBox <gi...@apache.org>.
linyiqun commented on a change in pull request #1528:
URL: https://github.com/apache/ozone/pull/1528#discussion_r514372611



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMFileRequest.java
##########
@@ -583,4 +589,180 @@ public static OmKeyInfo getOmKeyInfoFromFileTable(boolean openFileTable,
     return dbOmKeyInfo;
   }
 
+  /**
+   * Gets OmKeyInfo if exists for the given key name in the DB.
+   *
+   * @param omMetadataMgr metadata manager
+   * @param volumeName    volume name
+   * @param bucketName    bucket name
+   * @param keyName       key name
+   * @param scmBlockSize  scm block size
+   * @return OzoneFileStatus
+   * @throws IOException DB failure
+   */
+  @Nullable
+  public static OzoneFileStatus getOMKeyInfoIfExists(
+      OMMetadataManager omMetadataMgr, String volumeName, String bucketName,
+      String keyName, long scmBlockSize) throws IOException {
+
+    Path keyPath = Paths.get(keyName);
+    Iterator<Path> elements = keyPath.iterator();
+    String bucketKey = omMetadataMgr.getBucketKey(volumeName, bucketName);
+    OmBucketInfo omBucketInfo =
+            omMetadataMgr.getBucketTable().get(bucketKey);
+
+    long lastKnownParentId = omBucketInfo.getObjectID();
+    OmDirectoryInfo omDirInfo = null;
+    while (elements.hasNext()) {
+      String fileName = elements.next().toString();
+
+      // For example, /vol1/buck1/a/b/c/d/e/file1.txt
+      // 1. Do lookup path component on directoryTable starting from bucket
+      // 'buck1' to the leaf node component, which is 'file1.txt'.
+      // 2. If there is no dir exists for the leaf node component 'file1.txt'
+      // then do look it on fileTable.
+      String dbNodeName = omMetadataMgr.getOzonePathKey(
+              lastKnownParentId, fileName);
+      omDirInfo = omMetadataMgr.getDirectoryTable().get(dbNodeName);
+
+      if (omDirInfo != null) {
+        lastKnownParentId = omDirInfo.getObjectID();
+      } else if (!elements.hasNext()) {
+        // reached last path component. Check file exists for the given path.
+        OmKeyInfo omKeyInfo = OMFileRequest.getOmKeyInfoFromFileTable(false,
+                omMetadataMgr, dbNodeName, keyName);
+        if (omKeyInfo != null) {
+          return new OzoneFileStatus(omKeyInfo, scmBlockSize, false);
+        }
+      } else {
+        // Missing intermediate directory and just return null;
+        // key not found in DB
+        return null;
+      }
+    }
+
+    if (omDirInfo != null) {
+      OmKeyInfo omKeyInfo = getOmKeyInfo(volumeName, bucketName, omDirInfo,
+              keyName);
+      return new OzoneFileStatus(omKeyInfo, scmBlockSize, true);
+    }
+
+    // key not found in DB
+    return null;
+  }
+
+  /**
+   * Prepare OmKeyInfo from OmDirectoryInfo.
+   *
+   * @param volumeName volume name
+   * @param bucketName bucket name
+   * @param dirInfo    directory info
+   * @param keyName    user given key name
+   * @return OmKeyInfo object
+   */
+  @NotNull
+  public static OmKeyInfo getOmKeyInfo(String volumeName, String bucketName,
+      OmDirectoryInfo dirInfo, String keyName) {
+
+    OmKeyInfo.Builder builder = new OmKeyInfo.Builder();
+    builder.setParentObjectID(dirInfo.getParentObjectID());
+    builder.setKeyName(keyName);
+    builder.setAcls(dirInfo.getAcls());
+    builder.addAllMetadata(dirInfo.getMetadata());
+    builder.setVolumeName(volumeName);
+    builder.setBucketName(bucketName);
+    builder.setCreationTime(dirInfo.getCreationTime());
+    builder.setModificationTime(dirInfo.getModificationTime());
+    builder.setObjectID(dirInfo.getObjectID());
+    builder.setUpdateID(dirInfo.getUpdateID());
+    builder.setFileName(dirInfo.getName());
+    builder.setReplicationType(HddsProtos.ReplicationType.RATIS);
+    builder.setReplicationFactor(HddsProtos.ReplicationFactor.ONE);
+    builder.setOmKeyLocationInfos(Collections.singletonList(
+            new OmKeyLocationInfoGroup(0, new ArrayList<>())));
+    return builder.build();
+  }
+
+  /**
+   * Build DirectoryInfo from OmKeyInfo.
+   *
+   * @param keyInfo omKeyInfo
+   * @return omDirectoryInfo object
+   */
+  public static OmDirectoryInfo getDirectoryInfo(OmKeyInfo keyInfo){
+    OmDirectoryInfo.Builder builder = new OmDirectoryInfo.Builder();
+    builder.setParentObjectID(keyInfo.getParentObjectID());
+    builder.setAcls(keyInfo.getAcls());
+    builder.addAllMetadata(keyInfo.getMetadata());
+    builder.setCreationTime(keyInfo.getCreationTime());
+    builder.setModificationTime(keyInfo.getModificationTime());
+    builder.setObjectID(keyInfo.getObjectID());
+    builder.setUpdateID(keyInfo.getUpdateID());
+    builder.setName(OzoneFSUtils.getFileName(keyInfo.getKeyName()));
+    return builder.build();
+  }
+
+  /**
+   * Verify that the given toKey directory is a sub directory of fromKey
+   * directory.
+   * <p>
+   * For example, special case of renaming a directory to its own
+   * sub-directory is not allowed.
+   *
+   * @param fromKeyName source path
+   * @param toKeyName destination path
+   */
+  public static void verifyToDirIsASubDirOfFromDirectory(String fromKeyName,
+      String toKeyName, boolean isDir){
+    if (!isDir) {
+      return;
+    }
+    Path dstParent = Paths.get(toKeyName).getParent();
+    while (dstParent != null && !Paths.get(fromKeyName).equals(dstParent)) {
+      dstParent = dstParent.getParent();
+    }
+    Preconditions.checkArgument(dstParent == null,
+            "Cannot rename a directory to its own subdirectory");
+    return;
+  }
+
+  /**
+   * Verify parent exists for the destination path and return destination
+   * path parent Id.
+   * <p>
+   * Check whether dst parent dir exists or not. If the parent exists, then the
+   * source can be renamed to dst path.
+   *
+   * @param volumeName  volume name
+   * @param bucketName  bucket name
+   * @param toKeyName   destination path
+   * @param fromKeyName source path
+   * @param metaMgr     metadata manager
+   * @throws IOException if the destination parent dir doesn't exists.
+   */
+  public static long getToKeyNameParentId(String volumeName,
+      String bucketName, String toKeyName, String fromKeyName,
+      OMMetadataManager metaMgr) throws IOException {
+
+    int totalDirsCount = OzoneFSUtils.getFileCount(toKeyName);
+    // skip parent is root '/'
+    if (totalDirsCount <= 1) {
+      String bucketKey = metaMgr.getBucketKey(volumeName, bucketName);
+      OmBucketInfo omBucketInfo =
+              metaMgr.getBucketTable().get(bucketKey);
+      return omBucketInfo.getObjectID();
+    }
+
+    String toKeyParentDir = OzoneFSUtils.getParentDir(toKeyName);
+
+    OzoneFileStatus toKeyParentDirStatus = getOMKeyInfoIfExists(metaMgr,
+            volumeName, bucketName, toKeyParentDir, 0);
+    // check if the immediate parent exists
+    if (toKeyParentDirStatus == null) {

Review comment:
       Can we add a safe check toKeyParentDirStatus.isDirectory()to make sure toKeyParentDirStatus is a directory 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [ozone] rakeshadr commented on a change in pull request #1528: HDDS-4357: Rename : make rename an atomic ops by updating key path entry in dir/file table

Posted by GitBox <gi...@apache.org>.
rakeshadr commented on a change in pull request #1528:
URL: https://github.com/apache/ozone/pull/1528#discussion_r519243714



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyRenameRequestV1.java
##########
@@ -0,0 +1,291 @@
+/**
+ * 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.om.request.key;
+
+import com.google.common.base.Optional;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.cache.CacheKey;
+import org.apache.hadoop.hdds.utils.db.cache.CacheValue;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.audit.AuditLogger;
+import org.apache.hadoop.ozone.audit.OMAction;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OMMetrics;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OzoneFSUtils;
+import org.apache.hadoop.ozone.om.helpers.OzoneFileStatus;
+import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.file.OMFileRequest;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.key.OMKeyRenameResponse;
+import org.apache.hadoop.ozone.om.response.key.OMKeyRenameResponseV1;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.*;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Map;
+
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.KEY_NOT_FOUND;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK;
+
+/**
+ * Handles rename key request layout version V1.
+ */
+public class OMKeyRenameRequestV1 extends OMKeyRenameRequest {
+
+  private static final Logger LOG =
+          LoggerFactory.getLogger(OMKeyRenameRequestV1.class);
+
+  public OMKeyRenameRequestV1(OMRequest omRequest) {
+    super(omRequest);
+  }
+
+  @Override
+  @SuppressWarnings("methodlength")
+  public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
+      long trxnLogIndex, OzoneManagerDoubleBufferHelper omDoubleBufferHelper) {
+
+    RenameKeyRequest renameKeyRequest = getOmRequest().getRenameKeyRequest();
+    KeyArgs keyArgs = renameKeyRequest.getKeyArgs();
+    Map<String, String> auditMap = buildAuditMap(keyArgs, renameKeyRequest);
+
+    String volumeName = keyArgs.getVolumeName();
+    String bucketName = keyArgs.getBucketName();
+    String fromKeyName = keyArgs.getKeyName();
+    String toKeyName = renameKeyRequest.getToKeyName();
+
+    OMMetrics omMetrics = ozoneManager.getMetrics();
+    omMetrics.incNumKeyRenames();
+
+    AuditLogger auditLogger = ozoneManager.getAuditLogger();
+
+    OMResponse.Builder omResponse = OmResponseUtil.getOMResponseBuilder(
+            getOmRequest());
+
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+    boolean acquiredLock = false;
+    OMClientResponse omClientResponse = null;
+    IOException exception = null;
+    OmKeyInfo fromKeyValue;
+    String fromKey = null;
+    Result result;
+    try {
+      if (toKeyName.length() == 0 || fromKeyName.length() == 0) {
+        throw new OMException("Key name is empty",
+                OMException.ResultCodes.INVALID_KEY_NAME);
+      }
+
+      keyArgs = resolveBucketLink(ozoneManager, keyArgs, auditMap);
+      volumeName = keyArgs.getVolumeName();
+      bucketName = keyArgs.getBucketName();
+
+      // check Acls to see if user has access to perform delete operation on
+      // old key and create operation on new key
+      checkKeyAcls(ozoneManager, volumeName, bucketName, fromKeyName,
+              IAccessAuthorizer.ACLType.DELETE, OzoneObj.ResourceType.KEY);
+      checkKeyAcls(ozoneManager, volumeName, bucketName, toKeyName,
+              IAccessAuthorizer.ACLType.CREATE, OzoneObj.ResourceType.KEY);
+
+      acquiredLock = omMetadataManager.getLock().acquireWriteLock(BUCKET_LOCK,
+              volumeName, bucketName);
+
+      // Validate bucket and volume exists or not.
+      validateBucketAndVolume(omMetadataManager, volumeName, bucketName);
+
+      // Check if toKey exists
+      OzoneFileStatus fromKeyFileStatus =
+              OMFileRequest.getOMKeyInfoIfExists(omMetadataManager, volumeName,
+                      bucketName, fromKeyName, 0);
+      // fromKeyName should exist
+      if (fromKeyFileStatus == null) {
+        // TODO: Add support for renaming open key
+        throw new OMException("Key not found " + fromKey, KEY_NOT_FOUND);
+      }
+
+      // source exists
+      fromKeyValue = fromKeyFileStatus.getKeyInfo();
+      boolean isRenameDirectory = fromKeyFileStatus.isDirectory();
+
+      OzoneFileStatus toKeyFileStatus =
+              OMFileRequest.getOMKeyInfoIfExists(omMetadataManager,
+                      volumeName, bucketName, toKeyName, 0);
+      OmKeyInfo toKeyValue;
+
+      // Destination exists cases:
+      if(toKeyFileStatus != null) {
+
+        toKeyValue = toKeyFileStatus.getKeyInfo();
+
+        if (fromKeyValue.getKeyName().equals(toKeyValue.getKeyName())) {
+          // case-1) src == destin then check source and destin of same type
+          // If dst is a file then return true. Otherwise fail the operation.
+          if (!toKeyFileStatus.isDirectory()) {

Review comment:
       Thanks again for the great help:-)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [ozone] linyiqun commented on a change in pull request #1528: HDDS-4357: Rename : make rename an atomic ops by updating key path entry in dir/file table

Posted by GitBox <gi...@apache.org>.
linyiqun commented on a change in pull request #1528:
URL: https://github.com/apache/ozone/pull/1528#discussion_r518832585



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyRenameRequestV1.java
##########
@@ -0,0 +1,291 @@
+/**
+ * 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.om.request.key;
+
+import com.google.common.base.Optional;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.cache.CacheKey;
+import org.apache.hadoop.hdds.utils.db.cache.CacheValue;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.audit.AuditLogger;
+import org.apache.hadoop.ozone.audit.OMAction;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OMMetrics;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OzoneFSUtils;
+import org.apache.hadoop.ozone.om.helpers.OzoneFileStatus;
+import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.file.OMFileRequest;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.key.OMKeyRenameResponse;
+import org.apache.hadoop.ozone.om.response.key.OMKeyRenameResponseV1;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.*;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Map;
+
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.KEY_NOT_FOUND;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK;
+
+/**
+ * Handles rename key request layout version V1.
+ */
+public class OMKeyRenameRequestV1 extends OMKeyRenameRequest {
+
+  private static final Logger LOG =
+          LoggerFactory.getLogger(OMKeyRenameRequestV1.class);
+
+  public OMKeyRenameRequestV1(OMRequest omRequest) {
+    super(omRequest);
+  }
+
+  @Override
+  @SuppressWarnings("methodlength")
+  public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
+      long trxnLogIndex, OzoneManagerDoubleBufferHelper omDoubleBufferHelper) {
+
+    RenameKeyRequest renameKeyRequest = getOmRequest().getRenameKeyRequest();
+    KeyArgs keyArgs = renameKeyRequest.getKeyArgs();
+    Map<String, String> auditMap = buildAuditMap(keyArgs, renameKeyRequest);
+
+    String volumeName = keyArgs.getVolumeName();
+    String bucketName = keyArgs.getBucketName();
+    String fromKeyName = keyArgs.getKeyName();
+    String toKeyName = renameKeyRequest.getToKeyName();
+
+    OMMetrics omMetrics = ozoneManager.getMetrics();
+    omMetrics.incNumKeyRenames();
+
+    AuditLogger auditLogger = ozoneManager.getAuditLogger();
+
+    OMResponse.Builder omResponse = OmResponseUtil.getOMResponseBuilder(
+            getOmRequest());
+
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+    boolean acquiredLock = false;
+    OMClientResponse omClientResponse = null;
+    IOException exception = null;
+    OmKeyInfo fromKeyValue;
+    String fromKey = null;
+    Result result;
+    try {
+      if (toKeyName.length() == 0 || fromKeyName.length() == 0) {
+        throw new OMException("Key name is empty",
+                OMException.ResultCodes.INVALID_KEY_NAME);
+      }
+
+      keyArgs = resolveBucketLink(ozoneManager, keyArgs, auditMap);
+      volumeName = keyArgs.getVolumeName();
+      bucketName = keyArgs.getBucketName();
+
+      // check Acls to see if user has access to perform delete operation on
+      // old key and create operation on new key
+      checkKeyAcls(ozoneManager, volumeName, bucketName, fromKeyName,
+              IAccessAuthorizer.ACLType.DELETE, OzoneObj.ResourceType.KEY);
+      checkKeyAcls(ozoneManager, volumeName, bucketName, toKeyName,
+              IAccessAuthorizer.ACLType.CREATE, OzoneObj.ResourceType.KEY);
+
+      acquiredLock = omMetadataManager.getLock().acquireWriteLock(BUCKET_LOCK,
+              volumeName, bucketName);
+
+      // Validate bucket and volume exists or not.
+      validateBucketAndVolume(omMetadataManager, volumeName, bucketName);
+
+      // Check if toKey exists
+      OzoneFileStatus fromKeyFileStatus =
+              OMFileRequest.getOMKeyInfoIfExists(omMetadataManager, volumeName,
+                      bucketName, fromKeyName, 0);
+      // fromKeyName should exist
+      if (fromKeyFileStatus == null) {
+        // TODO: Add support for renaming open key
+        throw new OMException("Key not found " + fromKey, KEY_NOT_FOUND);
+      }
+
+      // source exists
+      fromKeyValue = fromKeyFileStatus.getKeyInfo();
+      boolean isRenameDirectory = fromKeyFileStatus.isDirectory();
+
+      OzoneFileStatus toKeyFileStatus =
+              OMFileRequest.getOMKeyInfoIfExists(omMetadataManager,
+                      volumeName, bucketName, toKeyName, 0);
+      OmKeyInfo toKeyValue;
+
+      // Destination exists cases:
+      if(toKeyFileStatus != null) {
+
+        toKeyValue = toKeyFileStatus.getKeyInfo();
+
+        if (fromKeyValue.getKeyName().equals(toKeyValue.getKeyName())) {
+          // case-1) src == destin then check source and destin of same type
+          // If dst is a file then return true. Otherwise fail the operation.
+          if (!toKeyFileStatus.isDirectory()) {

Review comment:
       I was looking at these lines before, [KeyManagerImpl.java#L821](https://github.com/apache/ozone/blob/master/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java#L821)
   
   I'm okay to keep this same with BasicOzoneFileSystem logic did.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [ozone] rakeshadr commented on a change in pull request #1528: HDDS-4357: Rename : make rename an atomic ops by updating key path entry in dir/file table

Posted by GitBox <gi...@apache.org>.
rakeshadr commented on a change in pull request #1528:
URL: https://github.com/apache/ozone/pull/1528#discussion_r518645184



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMFileRequest.java
##########
@@ -583,4 +589,180 @@ public static OmKeyInfo getOmKeyInfoFromFileTable(boolean openFileTable,
     return dbOmKeyInfo;
   }
 
+  /**
+   * Gets OmKeyInfo if exists for the given key name in the DB.
+   *
+   * @param omMetadataMgr metadata manager
+   * @param volumeName    volume name
+   * @param bucketName    bucket name
+   * @param keyName       key name
+   * @param scmBlockSize  scm block size
+   * @return OzoneFileStatus
+   * @throws IOException DB failure
+   */
+  @Nullable
+  public static OzoneFileStatus getOMKeyInfoIfExists(
+      OMMetadataManager omMetadataMgr, String volumeName, String bucketName,
+      String keyName, long scmBlockSize) throws IOException {
+
+    Path keyPath = Paths.get(keyName);
+    Iterator<Path> elements = keyPath.iterator();
+    String bucketKey = omMetadataMgr.getBucketKey(volumeName, bucketName);
+    OmBucketInfo omBucketInfo =
+            omMetadataMgr.getBucketTable().get(bucketKey);
+
+    long lastKnownParentId = omBucketInfo.getObjectID();
+    OmDirectoryInfo omDirInfo = null;
+    while (elements.hasNext()) {
+      String fileName = elements.next().toString();
+
+      // For example, /vol1/buck1/a/b/c/d/e/file1.txt
+      // 1. Do lookup path component on directoryTable starting from bucket
+      // 'buck1' to the leaf node component, which is 'file1.txt'.
+      // 2. If there is no dir exists for the leaf node component 'file1.txt'
+      // then do look it on fileTable.
+      String dbNodeName = omMetadataMgr.getOzonePathKey(
+              lastKnownParentId, fileName);
+      omDirInfo = omMetadataMgr.getDirectoryTable().get(dbNodeName);
+
+      if (omDirInfo != null) {
+        lastKnownParentId = omDirInfo.getObjectID();
+      } else if (!elements.hasNext()) {
+        // reached last path component. Check file exists for the given path.
+        OmKeyInfo omKeyInfo = OMFileRequest.getOmKeyInfoFromFileTable(false,
+                omMetadataMgr, dbNodeName, keyName);
+        if (omKeyInfo != null) {
+          return new OzoneFileStatus(omKeyInfo, scmBlockSize, false);
+        }
+      } else {
+        // Missing intermediate directory and just return null;
+        // key not found in DB
+        return null;
+      }
+    }
+
+    if (omDirInfo != null) {
+      OmKeyInfo omKeyInfo = getOmKeyInfo(volumeName, bucketName, omDirInfo,
+              keyName);
+      return new OzoneFileStatus(omKeyInfo, scmBlockSize, true);
+    }
+
+    // key not found in DB
+    return null;
+  }
+
+  /**
+   * Prepare OmKeyInfo from OmDirectoryInfo.
+   *
+   * @param volumeName volume name
+   * @param bucketName bucket name
+   * @param dirInfo    directory info
+   * @param keyName    user given key name
+   * @return OmKeyInfo object
+   */
+  @NotNull
+  public static OmKeyInfo getOmKeyInfo(String volumeName, String bucketName,
+      OmDirectoryInfo dirInfo, String keyName) {
+
+    OmKeyInfo.Builder builder = new OmKeyInfo.Builder();
+    builder.setParentObjectID(dirInfo.getParentObjectID());
+    builder.setKeyName(keyName);
+    builder.setAcls(dirInfo.getAcls());
+    builder.addAllMetadata(dirInfo.getMetadata());
+    builder.setVolumeName(volumeName);
+    builder.setBucketName(bucketName);
+    builder.setCreationTime(dirInfo.getCreationTime());
+    builder.setModificationTime(dirInfo.getModificationTime());
+    builder.setObjectID(dirInfo.getObjectID());
+    builder.setUpdateID(dirInfo.getUpdateID());
+    builder.setFileName(dirInfo.getName());
+    builder.setReplicationType(HddsProtos.ReplicationType.RATIS);
+    builder.setReplicationFactor(HddsProtos.ReplicationFactor.ONE);
+    builder.setOmKeyLocationInfos(Collections.singletonList(
+            new OmKeyLocationInfoGroup(0, new ArrayList<>())));
+    return builder.build();
+  }
+
+  /**
+   * Build DirectoryInfo from OmKeyInfo.
+   *
+   * @param keyInfo omKeyInfo
+   * @return omDirectoryInfo object
+   */
+  public static OmDirectoryInfo getDirectoryInfo(OmKeyInfo keyInfo){
+    OmDirectoryInfo.Builder builder = new OmDirectoryInfo.Builder();
+    builder.setParentObjectID(keyInfo.getParentObjectID());
+    builder.setAcls(keyInfo.getAcls());
+    builder.addAllMetadata(keyInfo.getMetadata());
+    builder.setCreationTime(keyInfo.getCreationTime());
+    builder.setModificationTime(keyInfo.getModificationTime());
+    builder.setObjectID(keyInfo.getObjectID());
+    builder.setUpdateID(keyInfo.getUpdateID());
+    builder.setName(OzoneFSUtils.getFileName(keyInfo.getKeyName()));
+    return builder.build();
+  }
+
+  /**
+   * Verify that the given toKey directory is a sub directory of fromKey
+   * directory.
+   * <p>
+   * For example, special case of renaming a directory to its own
+   * sub-directory is not allowed.
+   *
+   * @param fromKeyName source path
+   * @param toKeyName destination path
+   */
+  public static void verifyToDirIsASubDirOfFromDirectory(String fromKeyName,
+      String toKeyName, boolean isDir){
+    if (!isDir) {
+      return;
+    }
+    Path dstParent = Paths.get(toKeyName).getParent();
+    while (dstParent != null && !Paths.get(fromKeyName).equals(dstParent)) {
+      dstParent = dstParent.getParent();
+    }
+    Preconditions.checkArgument(dstParent == null,
+            "Cannot rename a directory to its own subdirectory");
+    return;
+  }
+
+  /**
+   * Verify parent exists for the destination path and return destination
+   * path parent Id.
+   * <p>
+   * Check whether dst parent dir exists or not. If the parent exists, then the
+   * source can be renamed to dst path.
+   *
+   * @param volumeName  volume name
+   * @param bucketName  bucket name
+   * @param toKeyName   destination path
+   * @param fromKeyName source path
+   * @param metaMgr     metadata manager
+   * @throws IOException if the destination parent dir doesn't exists.
+   */
+  public static long getToKeyNameParentId(String volumeName,
+      String bucketName, String toKeyName, String fromKeyName,
+      OMMetadataManager metaMgr) throws IOException {
+
+    int totalDirsCount = OzoneFSUtils.getFileCount(toKeyName);
+    // skip parent is root '/'
+    if (totalDirsCount <= 1) {
+      String bucketKey = metaMgr.getBucketKey(volumeName, bucketName);
+      OmBucketInfo omBucketInfo =
+              metaMgr.getBucketTable().get(bucketKey);
+      return omBucketInfo.getObjectID();
+    }
+
+    String toKeyParentDir = OzoneFSUtils.getParentDir(toKeyName);
+
+    OzoneFileStatus toKeyParentDirStatus = getOMKeyInfoIfExists(metaMgr,
+            volumeName, bucketName, toKeyParentDir, 0);
+    // check if the immediate parent exists
+    if (toKeyParentDirStatus == null) {

Review comment:
       Sure, I will do it in new commit
   
   > > ... Please go through my reply.
   > 
   > I didn't see your reply, do you commit the comments?
   > 
   > @rakeshadr , current latest commit include many previous commits? Would you update with the latest commit as [HDDS-4332](https://issues.apache.org/jira/browse/HDDS-4332) already merged.
   
   oops, sorry for that. I had added my comment and shown in my UI as duplicate then I have deleted it. That caused the issue and my entire comments got wiped out:-)
   
   I have added back my comments. Kindly go through it again. Thanks for your time.
   
   I am planning to create new PR as merging takes more time, by addressing all your comments apart from [one comment ](https://github.com/apache/ozone/pull/1528#discussion_r514214305)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [ozone] rakeshadr commented on a change in pull request #1528: HDDS-4357: Rename : make rename an atomic ops by updating key path entry in dir/file table

Posted by GitBox <gi...@apache.org>.
rakeshadr commented on a change in pull request #1528:
URL: https://github.com/apache/ozone/pull/1528#discussion_r518571985



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMFileRequest.java
##########
@@ -583,4 +589,180 @@ public static OmKeyInfo getOmKeyInfoFromFileTable(boolean openFileTable,
     return dbOmKeyInfo;
   }
 
+  /**
+   * Gets OmKeyInfo if exists for the given key name in the DB.
+   *
+   * @param omMetadataMgr metadata manager
+   * @param volumeName    volume name
+   * @param bucketName    bucket name
+   * @param keyName       key name
+   * @param scmBlockSize  scm block size
+   * @return OzoneFileStatus
+   * @throws IOException DB failure
+   */
+  @Nullable
+  public static OzoneFileStatus getOMKeyInfoIfExists(
+      OMMetadataManager omMetadataMgr, String volumeName, String bucketName,
+      String keyName, long scmBlockSize) throws IOException {
+
+    Path keyPath = Paths.get(keyName);
+    Iterator<Path> elements = keyPath.iterator();
+    String bucketKey = omMetadataMgr.getBucketKey(volumeName, bucketName);
+    OmBucketInfo omBucketInfo =
+            omMetadataMgr.getBucketTable().get(bucketKey);
+
+    long lastKnownParentId = omBucketInfo.getObjectID();
+    OmDirectoryInfo omDirInfo = null;
+    while (elements.hasNext()) {
+      String fileName = elements.next().toString();
+
+      // For example, /vol1/buck1/a/b/c/d/e/file1.txt
+      // 1. Do lookup path component on directoryTable starting from bucket
+      // 'buck1' to the leaf node component, which is 'file1.txt'.
+      // 2. If there is no dir exists for the leaf node component 'file1.txt'
+      // then do look it on fileTable.
+      String dbNodeName = omMetadataMgr.getOzonePathKey(
+              lastKnownParentId, fileName);
+      omDirInfo = omMetadataMgr.getDirectoryTable().get(dbNodeName);
+
+      if (omDirInfo != null) {
+        lastKnownParentId = omDirInfo.getObjectID();
+      } else if (!elements.hasNext()) {
+        // reached last path component. Check file exists for the given path.
+        OmKeyInfo omKeyInfo = OMFileRequest.getOmKeyInfoFromFileTable(false,
+                omMetadataMgr, dbNodeName, keyName);
+        if (omKeyInfo != null) {
+          return new OzoneFileStatus(omKeyInfo, scmBlockSize, false);
+        }
+      } else {
+        // Missing intermediate directory and just return null;
+        // key not found in DB
+        return null;
+      }
+    }
+
+    if (omDirInfo != null) {
+      OmKeyInfo omKeyInfo = getOmKeyInfo(volumeName, bucketName, omDirInfo,
+              keyName);
+      return new OzoneFileStatus(omKeyInfo, scmBlockSize, true);
+    }
+
+    // key not found in DB
+    return null;
+  }
+
+  /**
+   * Prepare OmKeyInfo from OmDirectoryInfo.
+   *
+   * @param volumeName volume name
+   * @param bucketName bucket name
+   * @param dirInfo    directory info
+   * @param keyName    user given key name
+   * @return OmKeyInfo object
+   */
+  @NotNull
+  public static OmKeyInfo getOmKeyInfo(String volumeName, String bucketName,
+      OmDirectoryInfo dirInfo, String keyName) {
+
+    OmKeyInfo.Builder builder = new OmKeyInfo.Builder();
+    builder.setParentObjectID(dirInfo.getParentObjectID());
+    builder.setKeyName(keyName);
+    builder.setAcls(dirInfo.getAcls());
+    builder.addAllMetadata(dirInfo.getMetadata());
+    builder.setVolumeName(volumeName);
+    builder.setBucketName(bucketName);
+    builder.setCreationTime(dirInfo.getCreationTime());
+    builder.setModificationTime(dirInfo.getModificationTime());
+    builder.setObjectID(dirInfo.getObjectID());
+    builder.setUpdateID(dirInfo.getUpdateID());
+    builder.setFileName(dirInfo.getName());
+    builder.setReplicationType(HddsProtos.ReplicationType.RATIS);
+    builder.setReplicationFactor(HddsProtos.ReplicationFactor.ONE);
+    builder.setOmKeyLocationInfos(Collections.singletonList(
+            new OmKeyLocationInfoGroup(0, new ArrayList<>())));
+    return builder.build();
+  }
+
+  /**
+   * Build DirectoryInfo from OmKeyInfo.
+   *
+   * @param keyInfo omKeyInfo
+   * @return omDirectoryInfo object
+   */
+  public static OmDirectoryInfo getDirectoryInfo(OmKeyInfo keyInfo){
+    OmDirectoryInfo.Builder builder = new OmDirectoryInfo.Builder();
+    builder.setParentObjectID(keyInfo.getParentObjectID());
+    builder.setAcls(keyInfo.getAcls());
+    builder.addAllMetadata(keyInfo.getMetadata());
+    builder.setCreationTime(keyInfo.getCreationTime());
+    builder.setModificationTime(keyInfo.getModificationTime());
+    builder.setObjectID(keyInfo.getObjectID());
+    builder.setUpdateID(keyInfo.getUpdateID());
+    builder.setName(OzoneFSUtils.getFileName(keyInfo.getKeyName()));
+    return builder.build();
+  }
+
+  /**
+   * Verify that the given toKey directory is a sub directory of fromKey
+   * directory.
+   * <p>
+   * For example, special case of renaming a directory to its own
+   * sub-directory is not allowed.
+   *
+   * @param fromKeyName source path
+   * @param toKeyName destination path
+   */
+  public static void verifyToDirIsASubDirOfFromDirectory(String fromKeyName,
+      String toKeyName, boolean isDir){
+    if (!isDir) {
+      return;
+    }
+    Path dstParent = Paths.get(toKeyName).getParent();
+    while (dstParent != null && !Paths.get(fromKeyName).equals(dstParent)) {
+      dstParent = dstParent.getParent();
+    }
+    Preconditions.checkArgument(dstParent == null,
+            "Cannot rename a directory to its own subdirectory");
+    return;
+  }
+
+  /**
+   * Verify parent exists for the destination path and return destination
+   * path parent Id.
+   * <p>
+   * Check whether dst parent dir exists or not. If the parent exists, then the
+   * source can be renamed to dst path.
+   *
+   * @param volumeName  volume name
+   * @param bucketName  bucket name
+   * @param toKeyName   destination path
+   * @param fromKeyName source path
+   * @param metaMgr     metadata manager
+   * @throws IOException if the destination parent dir doesn't exists.
+   */
+  public static long getToKeyNameParentId(String volumeName,
+      String bucketName, String toKeyName, String fromKeyName,
+      OMMetadataManager metaMgr) throws IOException {
+
+    int totalDirsCount = OzoneFSUtils.getFileCount(toKeyName);
+    // skip parent is root '/'
+    if (totalDirsCount <= 1) {
+      String bucketKey = metaMgr.getBucketKey(volumeName, bucketName);
+      OmBucketInfo omBucketInfo =
+              metaMgr.getBucketTable().get(bucketKey);
+      return omBucketInfo.getObjectID();
+    }
+
+    String toKeyParentDir = OzoneFSUtils.getParentDir(toKeyName);
+
+    OzoneFileStatus toKeyParentDirStatus = getOMKeyInfoIfExists(metaMgr,
+            volumeName, bucketName, toKeyParentDir, 0);
+    // check if the immediate parent exists
+    if (toKeyParentDirStatus == null) {

Review comment:
       Sure, I will do it in latest commit

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyRenameRequestV1.java
##########
@@ -0,0 +1,291 @@
+/**
+ * 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.om.request.key;
+
+import com.google.common.base.Optional;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.cache.CacheKey;
+import org.apache.hadoop.hdds.utils.db.cache.CacheValue;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.audit.AuditLogger;
+import org.apache.hadoop.ozone.audit.OMAction;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OMMetrics;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OzoneFSUtils;
+import org.apache.hadoop.ozone.om.helpers.OzoneFileStatus;
+import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.file.OMFileRequest;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.key.OMKeyRenameResponse;
+import org.apache.hadoop.ozone.om.response.key.OMKeyRenameResponseV1;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.*;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Map;
+
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.KEY_NOT_FOUND;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK;
+
+/**
+ * Handles rename key request layout version V1.
+ */
+public class OMKeyRenameRequestV1 extends OMKeyRenameRequest {
+
+  private static final Logger LOG =
+          LoggerFactory.getLogger(OMKeyRenameRequestV1.class);
+
+  public OMKeyRenameRequestV1(OMRequest omRequest) {
+    super(omRequest);
+  }
+
+  @Override
+  @SuppressWarnings("methodlength")
+  public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
+      long trxnLogIndex, OzoneManagerDoubleBufferHelper omDoubleBufferHelper) {
+
+    RenameKeyRequest renameKeyRequest = getOmRequest().getRenameKeyRequest();
+    KeyArgs keyArgs = renameKeyRequest.getKeyArgs();
+    Map<String, String> auditMap = buildAuditMap(keyArgs, renameKeyRequest);
+
+    String volumeName = keyArgs.getVolumeName();
+    String bucketName = keyArgs.getBucketName();
+    String fromKeyName = keyArgs.getKeyName();
+    String toKeyName = renameKeyRequest.getToKeyName();
+
+    OMMetrics omMetrics = ozoneManager.getMetrics();
+    omMetrics.incNumKeyRenames();
+
+    AuditLogger auditLogger = ozoneManager.getAuditLogger();
+
+    OMResponse.Builder omResponse = OmResponseUtil.getOMResponseBuilder(
+            getOmRequest());
+
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+    boolean acquiredLock = false;
+    OMClientResponse omClientResponse = null;
+    IOException exception = null;
+    OmKeyInfo fromKeyValue;
+    String fromKey = null;
+    Result result;
+    try {
+      if (toKeyName.length() == 0 || fromKeyName.length() == 0) {
+        throw new OMException("Key name is empty",
+                OMException.ResultCodes.INVALID_KEY_NAME);
+      }
+
+      keyArgs = resolveBucketLink(ozoneManager, keyArgs, auditMap);
+      volumeName = keyArgs.getVolumeName();
+      bucketName = keyArgs.getBucketName();
+
+      // check Acls to see if user has access to perform delete operation on
+      // old key and create operation on new key
+      checkKeyAcls(ozoneManager, volumeName, bucketName, fromKeyName,
+              IAccessAuthorizer.ACLType.DELETE, OzoneObj.ResourceType.KEY);
+      checkKeyAcls(ozoneManager, volumeName, bucketName, toKeyName,
+              IAccessAuthorizer.ACLType.CREATE, OzoneObj.ResourceType.KEY);
+
+      acquiredLock = omMetadataManager.getLock().acquireWriteLock(BUCKET_LOCK,
+              volumeName, bucketName);
+
+      // Validate bucket and volume exists or not.
+      validateBucketAndVolume(omMetadataManager, volumeName, bucketName);
+
+      // Check if toKey exists
+      OzoneFileStatus fromKeyFileStatus =
+              OMFileRequest.getOMKeyInfoIfExists(omMetadataManager, volumeName,
+                      bucketName, fromKeyName, 0);
+      // fromKeyName should exist
+      if (fromKeyFileStatus == null) {
+        // TODO: Add support for renaming open key
+        throw new OMException("Key not found " + fromKey, KEY_NOT_FOUND);
+      }
+
+      // source exists
+      fromKeyValue = fromKeyFileStatus.getKeyInfo();
+      boolean isRenameDirectory = fromKeyFileStatus.isDirectory();
+
+      OzoneFileStatus toKeyFileStatus =
+              OMFileRequest.getOMKeyInfoIfExists(omMetadataManager,
+                      volumeName, bucketName, toKeyName, 0);
+      OmKeyInfo toKeyValue;
+
+      // Destination exists cases:
+      if(toKeyFileStatus != null) {
+
+        toKeyValue = toKeyFileStatus.getKeyInfo();
+
+        if (fromKeyValue.getKeyName().equals(toKeyValue.getKeyName())) {
+          // case-1) src == destin then check source and destin of same type
+          // If dst is a file then return true. Otherwise fail the operation.
+          if (!toKeyFileStatus.isDirectory()) {

Review comment:
       I've referred following code in [BasicOzoneFileSystem.java#L369](https://github.com/apache/ozone/blob/master/hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicOzoneFileSystem.java#L369) and IIUC, it is returning false for a directory and true for a file.
   
   Appreciate if you could point me to the code you are referring to understand the case well. Thanks!

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyRenameRequestV1.java
##########
@@ -0,0 +1,291 @@
+/**
+ * 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.om.request.key;
+
+import com.google.common.base.Optional;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.cache.CacheKey;
+import org.apache.hadoop.hdds.utils.db.cache.CacheValue;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.audit.AuditLogger;
+import org.apache.hadoop.ozone.audit.OMAction;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OMMetrics;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OzoneFSUtils;
+import org.apache.hadoop.ozone.om.helpers.OzoneFileStatus;
+import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.file.OMFileRequest;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.key.OMKeyRenameResponse;
+import org.apache.hadoop.ozone.om.response.key.OMKeyRenameResponseV1;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.*;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Map;
+
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.KEY_NOT_FOUND;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK;
+
+/**
+ * Handles rename key request layout version V1.
+ */
+public class OMKeyRenameRequestV1 extends OMKeyRenameRequest {
+
+  private static final Logger LOG =
+          LoggerFactory.getLogger(OMKeyRenameRequestV1.class);
+
+  public OMKeyRenameRequestV1(OMRequest omRequest) {
+    super(omRequest);
+  }
+
+  @Override
+  @SuppressWarnings("methodlength")
+  public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
+      long trxnLogIndex, OzoneManagerDoubleBufferHelper omDoubleBufferHelper) {
+
+    RenameKeyRequest renameKeyRequest = getOmRequest().getRenameKeyRequest();
+    KeyArgs keyArgs = renameKeyRequest.getKeyArgs();
+    Map<String, String> auditMap = buildAuditMap(keyArgs, renameKeyRequest);
+
+    String volumeName = keyArgs.getVolumeName();
+    String bucketName = keyArgs.getBucketName();
+    String fromKeyName = keyArgs.getKeyName();
+    String toKeyName = renameKeyRequest.getToKeyName();
+
+    OMMetrics omMetrics = ozoneManager.getMetrics();
+    omMetrics.incNumKeyRenames();
+
+    AuditLogger auditLogger = ozoneManager.getAuditLogger();
+
+    OMResponse.Builder omResponse = OmResponseUtil.getOMResponseBuilder(
+            getOmRequest());
+
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+    boolean acquiredLock = false;
+    OMClientResponse omClientResponse = null;
+    IOException exception = null;
+    OmKeyInfo fromKeyValue;
+    String fromKey = null;
+    Result result;
+    try {
+      if (toKeyName.length() == 0 || fromKeyName.length() == 0) {
+        throw new OMException("Key name is empty",
+                OMException.ResultCodes.INVALID_KEY_NAME);
+      }
+
+      keyArgs = resolveBucketLink(ozoneManager, keyArgs, auditMap);
+      volumeName = keyArgs.getVolumeName();
+      bucketName = keyArgs.getBucketName();
+
+      // check Acls to see if user has access to perform delete operation on
+      // old key and create operation on new key
+      checkKeyAcls(ozoneManager, volumeName, bucketName, fromKeyName,
+              IAccessAuthorizer.ACLType.DELETE, OzoneObj.ResourceType.KEY);
+      checkKeyAcls(ozoneManager, volumeName, bucketName, toKeyName,
+              IAccessAuthorizer.ACLType.CREATE, OzoneObj.ResourceType.KEY);
+
+      acquiredLock = omMetadataManager.getLock().acquireWriteLock(BUCKET_LOCK,
+              volumeName, bucketName);
+
+      // Validate bucket and volume exists or not.
+      validateBucketAndVolume(omMetadataManager, volumeName, bucketName);
+
+      // Check if toKey exists
+      OzoneFileStatus fromKeyFileStatus =
+              OMFileRequest.getOMKeyInfoIfExists(omMetadataManager, volumeName,
+                      bucketName, fromKeyName, 0);
+      // fromKeyName should exist
+      if (fromKeyFileStatus == null) {
+        // TODO: Add support for renaming open key
+        throw new OMException("Key not found " + fromKey, KEY_NOT_FOUND);
+      }
+
+      // source exists
+      fromKeyValue = fromKeyFileStatus.getKeyInfo();
+      boolean isRenameDirectory = fromKeyFileStatus.isDirectory();
+
+      OzoneFileStatus toKeyFileStatus =
+              OMFileRequest.getOMKeyInfoIfExists(omMetadataManager,
+                      volumeName, bucketName, toKeyName, 0);
+      OmKeyInfo toKeyValue;
+
+      // Destination exists cases:
+      if(toKeyFileStatus != null) {
+
+        toKeyValue = toKeyFileStatus.getKeyInfo();
+
+        if (fromKeyValue.getKeyName().equals(toKeyValue.getKeyName())) {
+          // case-1) src == destin then check source and destin of same type
+          // If dst is a file then return true. Otherwise fail the operation.
+          if (!toKeyFileStatus.isDirectory()) {
+            result = Result.SUCCESS;
+          } else {
+            throw new OMException("Key already exists " + toKeyName,
+                    OMException.ResultCodes.KEY_ALREADY_EXISTS);
+          }
+        } else if (toKeyFileStatus.isDirectory()) {
+          // case-2) If dst is a directory, rename source as sub-path of it.
+          // For example: rename /source to /dst will lead to /dst/source
+          String fromFileName = OzoneFSUtils.getFileName(fromKeyName);

Review comment:
       Good catch. I will do this check.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [ozone] rakeshadr commented on pull request #1528: HDDS-4357: Rename : make rename an atomic ops by updating key path entry in dir/file table

Posted by GitBox <gi...@apache.org>.
rakeshadr commented on pull request #1528:
URL: https://github.com/apache/ozone/pull/1528#issuecomment-723026444


   Opened new PR https://github.com/apache/ozone/pull/1557.
   
   Will close this current PR in few days. Just keeping as a reference now.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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