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/12/09 12:20:47 UTC

[GitHub] [ozone] rakeshadr opened a new pull request #1679: HDDS-4514. AllocateBlock : lookup and update open file table for the given path

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


   ## What changes were proposed in this pull request?
   
   This task is to use open file table for the allocate block operations. This has been identified as part of Hive TPCDS benchmark test
   
   ## What is the link to the Apache JIRA
   
   https://issues.apache.org/jira/browse/HDDS-4514
   
   ## How was this patch tested?
   Added UT cases. Also, verified by running hive tpcds test


----------------------------------------------------------------
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 #1679: HDDS-4514. AllocateBlock : lookup and update open file table for the given path

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/key/OMAllocateBlockResponseV1.java
##########
@@ -0,0 +1,63 @@
+/**
+ * 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.response.key;
+
+import org.apache.hadoop.hdds.utils.db.BatchOperation;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.request.file.OMFileRequest;
+import org.apache.hadoop.ozone.om.response.CleanupTableInfo;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+
+import javax.annotation.Nonnull;
+import java.io.IOException;
+
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.OPEN_FILE_TABLE;
+
+/**
+ * Response for AllocateBlock request layout version V1.
+ */
+@CleanupTableInfo(cleanupTables = {OPEN_FILE_TABLE})
+public class OMAllocateBlockResponseV1 extends OMAllocateBlockResponse {
+
+  public OMAllocateBlockResponseV1(@Nonnull OMResponse omResponse,
+      @Nonnull OmKeyInfo omKeyInfo, long clientID,
+      @Nonnull OmVolumeArgs omVolumeArgs, @Nonnull OmBucketInfo omBucketInfo) {
+    super(omResponse, omKeyInfo, clientID, omVolumeArgs, omBucketInfo);
+  }
+
+  @Override
+  public void addToDBBatch(OMMetadataManager omMetadataManager,
+      BatchOperation batchOperation) throws IOException {
+
+    OMFileRequest.addToOpenFileTable(omMetadataManager, batchOperation,
+            getOmKeyInfo(), getClientID());
+
+    // update volume usedBytes.
+    omMetadataManager.getVolumeTable().putWithBatch(batchOperation,
+            omMetadataManager.getVolumeKey(getOmVolumeArgs().getVolume()),
+            getOmVolumeArgs());

Review comment:
       Thanks @linyiqun  for the review comments. Fixed in latest 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 pull request #1679: HDDS-4514. AllocateBlock : lookup and update open file table for the given path

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


   @linyiqun I've addressed your comments. Could you please review it again 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] bharatviswa504 commented on a change in pull request #1679: HDDS-4514. AllocateBlock : lookup and update open file table for the given path

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMAllocateBlockRequestV1.java
##########
@@ -0,0 +1,91 @@
+/**
+ * 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 org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.helpers.OzoneFSUtils;
+import org.apache.hadoop.ozone.om.request.file.OMFileRequest;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.key.OMAllocateBlockResponseV1;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Iterator;
+
+/**
+ * Handles allocate block request layout version V1.
+ */
+public class OMAllocateBlockRequestV1 extends OMAllocateBlockRequest {
+
+  private static final Logger LOG =
+          LoggerFactory.getLogger(OMAllocateBlockRequestV1.class);
+
+  public OMAllocateBlockRequestV1(OMRequest omRequest) {
+    super(omRequest);
+  }
+
+  protected OmKeyInfo getOpenKeyInfo(OMMetadataManager omMetadataManager,
+      String openKeyName, String keyName) throws IOException {
+    String fileName = OzoneFSUtils.getFileName(keyName);
+    return OMFileRequest.getOmKeyInfoFromFileTable(true,
+            omMetadataManager, openKeyName, fileName);
+  }
+
+  protected String getOpenKeyName(String volumeName, String bucketName,
+      String keyName, long clientID, OzoneManager ozoneManager)
+          throws IOException {
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+    String bucketKey = omMetadataManager.getBucketKey(volumeName, bucketName);
+    OmBucketInfo omBucketInfo =
+            omMetadataManager.getBucketTable().get(bucketKey);
+    long bucketId = omBucketInfo.getObjectID();
+    String fileName = OzoneFSUtils.getFileName(keyName);
+    Iterator<Path> pathComponents = Paths.get(keyName).iterator();

Review comment:
       Yes, I agree the ideal thing should be doing this under bucket lock whole operation. But that will have performance impact, as previously allocate block was done with out lock. Previous code has a similar issue, even then during key commit, it might not be found out, and also before we don't have atomic delete, so nothing can be really guaranteed.
   
   In new approach deletes are atomic, and when delete only parent dir is removed, and childs are deleted asynchronously. With new approach, this will be figured out during key commit (As we iterate the entire path again). But it will have impact on performance, and client may write some additional block data even when parent dir's are removed. (This is the only a problem for fs semantics, not object-store semantics)
   
   So, here we need to make a choice, do we want to detect during allocate block or during commit Key.
   
   cc @mukul1987 @arp7 for comments
   




----------------------------------------------------------------
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 #1679: HDDS-4514. AllocateBlock : lookup and update open file table for the given path

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMAllocateBlockRequestV1.java
##########
@@ -0,0 +1,91 @@
+/**
+ * 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 org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.helpers.OzoneFSUtils;
+import org.apache.hadoop.ozone.om.request.file.OMFileRequest;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.key.OMAllocateBlockResponseV1;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Iterator;
+
+/**
+ * Handles allocate block request layout version V1.
+ */
+public class OMAllocateBlockRequestV1 extends OMAllocateBlockRequest {
+
+  private static final Logger LOG =
+          LoggerFactory.getLogger(OMAllocateBlockRequestV1.class);
+
+  public OMAllocateBlockRequestV1(OMRequest omRequest) {
+    super(omRequest);
+  }
+
+  protected OmKeyInfo getOpenKeyInfo(OMMetadataManager omMetadataManager,
+      String openKeyName, String keyName) throws IOException {
+    String fileName = OzoneFSUtils.getFileName(keyName);
+    return OMFileRequest.getOmKeyInfoFromFileTable(true,
+            omMetadataManager, openKeyName, fileName);
+  }
+
+  protected String getOpenKeyName(String volumeName, String bucketName,
+      String keyName, long clientID, OzoneManager ozoneManager)
+          throws IOException {
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+    String bucketKey = omMetadataManager.getBucketKey(volumeName, bucketName);
+    OmBucketInfo omBucketInfo =
+            omMetadataManager.getBucketTable().get(bucketKey);
+    long bucketId = omBucketInfo.getObjectID();
+    String fileName = OzoneFSUtils.getFileName(keyName);
+    Iterator<Path> pathComponents = Paths.get(keyName).iterator();

Review comment:
       Good catch. Yes, during commit it will throw exception. But somehow I feel it is safer to move #getOpenKeyInfo() under BUCKET_LOCK and this would avoid race conditions and will make consistency to the system. What do you say?
   
   On the other side, moving getOpenKeyInfo() inside lock is adding any additional overhead and performance penalty ? If not, then I will move it inside the lock. 




----------------------------------------------------------------
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 #1679: HDDS-4514. AllocateBlock : lookup and update open file table for the given path

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMFileRequest.java
##########
@@ -556,9 +560,10 @@ public static OmKeyInfo getOmKeyInfoFromFileTable(boolean openFileTable,
     // For example, the user given key path is '/a/b/c/d/e/file1', then in DB
     // keyName field stores only the leaf node name, which is 'file1'.
     if (dbOmKeyInfo != null) {
-      dbOmKeyInfo.setKeyName(keyName);

Review comment:
       Agreed, Done the changes!

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMFileRequest.java
##########
@@ -851,4 +856,72 @@ private static boolean checkSubFileExists(OmKeyInfo omKeyInfo,
   public static boolean isImmediateChild(long parentId, long ancestorId) {
     return parentId == ancestorId;
   }
+
+
+  /**
+   * Check for directory exists with same name, if it exists throw error.
+   *
+   * @param keyName                  key name
+   * @param ozoneManager             Ozone Manager
+   * @param reachedLastPathComponent true if the path component is a fileName
+   * @throws IOException if directory exists with same name
+   */
+  private static void checkDirectoryAlreadyExists(String keyName,
+      OzoneManager ozoneManager, boolean reachedLastPathComponent)
+          throws IOException {
+    // Reached last component, which would be a file. Returns its parentID.

Review comment:
       Agreed, Done the changes!




----------------------------------------------------------------
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] bharatviswa504 commented on a change in pull request #1679: HDDS-4514. AllocateBlock : lookup and update open file table for the given path

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMAllocateBlockRequestV1.java
##########
@@ -0,0 +1,91 @@
+/**
+ * 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 org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.helpers.OzoneFSUtils;
+import org.apache.hadoop.ozone.om.request.file.OMFileRequest;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.key.OMAllocateBlockResponseV1;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Iterator;
+
+/**
+ * Handles allocate block request layout version V1.
+ */
+public class OMAllocateBlockRequestV1 extends OMAllocateBlockRequest {
+
+  private static final Logger LOG =
+          LoggerFactory.getLogger(OMAllocateBlockRequestV1.class);
+
+  public OMAllocateBlockRequestV1(OMRequest omRequest) {
+    super(omRequest);
+  }
+
+  protected OmKeyInfo getOpenKeyInfo(OMMetadataManager omMetadataManager,
+      String openKeyName, String keyName) throws IOException {
+    String fileName = OzoneFSUtils.getFileName(keyName);
+    return OMFileRequest.getOmKeyInfoFromFileTable(true,
+            omMetadataManager, openKeyName, fileName);
+  }
+
+  protected String getOpenKeyName(String volumeName, String bucketName,
+      String keyName, long clientID, OzoneManager ozoneManager)
+          throws IOException {
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+    String bucketKey = omMetadataManager.getBucketKey(volumeName, bucketName);
+    OmBucketInfo omBucketInfo =
+            omMetadataManager.getBucketTable().get(bucketKey);
+    long bucketId = omBucketInfo.getObjectID();
+    String fileName = OzoneFSUtils.getFileName(keyName);
+    Iterator<Path> pathComponents = Paths.get(keyName).iterator();

Review comment:
       On a side note anyway I think it will be handled during commit Key anyway.
   Thoughts?




----------------------------------------------------------------
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 #1679: HDDS-4514. AllocateBlock : lookup and update open file table for the given path

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMAllocateBlockRequest.java
##########
@@ -219,14 +219,13 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
       omMetadataManager.getOpenKeyTable().addCacheEntry(
           new CacheKey<>(openKeyName),
           new CacheValue<>(Optional.of(openKeyInfo), trxnLogIndex));
-

Review comment:
       Removed unnecessary spaces!




----------------------------------------------------------------
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] bharatviswa504 commented on a change in pull request #1679: HDDS-4514. AllocateBlock : lookup and update open file table for the given path

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMAllocateBlockRequestV1.java
##########
@@ -0,0 +1,91 @@
+/**
+ * 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 org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.helpers.OzoneFSUtils;
+import org.apache.hadoop.ozone.om.request.file.OMFileRequest;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.key.OMAllocateBlockResponseV1;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Iterator;
+
+/**
+ * Handles allocate block request layout version V1.
+ */
+public class OMAllocateBlockRequestV1 extends OMAllocateBlockRequest {
+
+  private static final Logger LOG =
+          LoggerFactory.getLogger(OMAllocateBlockRequestV1.class);
+
+  public OMAllocateBlockRequestV1(OMRequest omRequest) {
+    super(omRequest);
+  }
+
+  protected OmKeyInfo getOpenKeyInfo(OMMetadataManager omMetadataManager,
+      String openKeyName, String keyName) throws IOException {
+    String fileName = OzoneFSUtils.getFileName(keyName);
+    return OMFileRequest.getOmKeyInfoFromFileTable(true,
+            omMetadataManager, openKeyName, fileName);
+  }
+
+  protected String getOpenKeyName(String volumeName, String bucketName,
+      String keyName, long clientID, OzoneManager ozoneManager)
+          throws IOException {
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+    String bucketKey = omMetadataManager.getBucketKey(volumeName, bucketName);
+    OmBucketInfo omBucketInfo =
+            omMetadataManager.getBucketTable().get(bucketKey);
+    long bucketId = omBucketInfo.getObjectID();
+    String fileName = OzoneFSUtils.getFileName(keyName);
+    Iterator<Path> pathComponents = Paths.get(keyName).iterator();

Review comment:
       Yes, I agree the ideal thing should be doing this under bucket lock whole operation. But that will have performance impact, as previously allocate block was done with out lock. Previous code has a similar issue, even then during key commit, it might not be found out, and also before we don't have atomic delete, so nothing can be really guaranteed.
   
   In new approach deletes are atomic, and when delete only parent dir is removed, and childs are deleted asynchronously. With new approach, this will be figured out during key commit (As we iterate the entire path again). But it will have impact on performance, and client may write some additional block data even when parent dir's are removed. (This is the only a problem for fs semantics, not object-store semantics). This will be eventually figured out during the next allocateblock, keycommit.
   
   So, here we need to make a choice, do we want to detect during allocate block or during commit Key.
   
   cc @mukul1987 @arp7 for comments
   




----------------------------------------------------------------
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 #1679: HDDS-4514. AllocateBlock : lookup and update open file table for the given path

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMAllocateBlockRequestV1.java
##########
@@ -0,0 +1,91 @@
+/**
+ * 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 org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.helpers.OzoneFSUtils;
+import org.apache.hadoop.ozone.om.request.file.OMFileRequest;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.key.OMAllocateBlockResponseV1;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Iterator;
+
+/**
+ * Handles allocate block request layout version V1.
+ */
+public class OMAllocateBlockRequestV1 extends OMAllocateBlockRequest {
+
+  private static final Logger LOG =
+          LoggerFactory.getLogger(OMAllocateBlockRequestV1.class);
+
+  public OMAllocateBlockRequestV1(OMRequest omRequest) {
+    super(omRequest);
+  }
+
+  protected OmKeyInfo getOpenKeyInfo(OMMetadataManager omMetadataManager,
+      String openKeyName, String keyName) throws IOException {
+    String fileName = OzoneFSUtils.getFileName(keyName);
+    return OMFileRequest.getOmKeyInfoFromFileTable(true,
+            omMetadataManager, openKeyName, fileName);
+  }
+
+  protected String getOpenKeyName(String volumeName, String bucketName,
+      String keyName, long clientID, OzoneManager ozoneManager)
+          throws IOException {
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+    String bucketKey = omMetadataManager.getBucketKey(volumeName, bucketName);
+    OmBucketInfo omBucketInfo =
+            omMetadataManager.getBucketTable().get(bucketKey);
+    long bucketId = omBucketInfo.getObjectID();
+    String fileName = OzoneFSUtils.getFileName(keyName);
+    Iterator<Path> pathComponents = Paths.get(keyName).iterator();

Review comment:
       Thanks @bharatviswa504 for the quick response.




----------------------------------------------------------------
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 merged pull request #1679: HDDS-4514. AllocateBlock : lookup and update open file table for the given path

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


   


----------------------------------------------------------------
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] bharatviswa504 commented on a change in pull request #1679: HDDS-4514. AllocateBlock : lookup and update open file table for the given path

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMAllocateBlockRequestV1.java
##########
@@ -0,0 +1,91 @@
+/**
+ * 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 org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.helpers.OzoneFSUtils;
+import org.apache.hadoop.ozone.om.request.file.OMFileRequest;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.key.OMAllocateBlockResponseV1;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Iterator;
+
+/**
+ * Handles allocate block request layout version V1.
+ */
+public class OMAllocateBlockRequestV1 extends OMAllocateBlockRequest {
+
+  private static final Logger LOG =
+          LoggerFactory.getLogger(OMAllocateBlockRequestV1.class);
+
+  public OMAllocateBlockRequestV1(OMRequest omRequest) {
+    super(omRequest);
+  }
+
+  protected OmKeyInfo getOpenKeyInfo(OMMetadataManager omMetadataManager,
+      String openKeyName, String keyName) throws IOException {
+    String fileName = OzoneFSUtils.getFileName(keyName);
+    return OMFileRequest.getOmKeyInfoFromFileTable(true,
+            omMetadataManager, openKeyName, fileName);
+  }
+
+  protected String getOpenKeyName(String volumeName, String bucketName,
+      String keyName, long clientID, OzoneManager ozoneManager)
+          throws IOException {
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+    String bucketKey = omMetadataManager.getBucketKey(volumeName, bucketName);
+    OmBucketInfo omBucketInfo =
+            omMetadataManager.getBucketTable().get(bucketKey);
+    long bucketId = omBucketInfo.getObjectID();
+    String fileName = OzoneFSUtils.getFileName(keyName);
+    Iterator<Path> pathComponents = Paths.get(keyName).iterator();

Review comment:
       Not holding bucket lock during traversing of key to get parentID
   Do you think we will have a race condition with delete or any parent directories?
   
   




----------------------------------------------------------------
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] bharatviswa504 commented on a change in pull request #1679: HDDS-4514. AllocateBlock : lookup and update open file table for the given path

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMAllocateBlockRequestV1.java
##########
@@ -0,0 +1,91 @@
+/**
+ * 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 org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.helpers.OzoneFSUtils;
+import org.apache.hadoop.ozone.om.request.file.OMFileRequest;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.key.OMAllocateBlockResponseV1;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Iterator;
+
+/**
+ * Handles allocate block request layout version V1.
+ */
+public class OMAllocateBlockRequestV1 extends OMAllocateBlockRequest {

Review comment:
       Yes. I will take one final look at it (as previously skimmed the entire patch) and post my comments.




----------------------------------------------------------------
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 #1679: HDDS-4514. AllocateBlock : lookup and update open file table for the given path

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


   > Please address below minor comment and fix failed unit test, : ).
   
   TestOzoneFileSystemV1#testListStatusOnLargeDirectory is failing in the build, interestingly it is passing consistently in my local env. Added log messages to debug it. Will monitor the results in next build.


----------------------------------------------------------------
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] bharatviswa504 commented on a change in pull request #1679: HDDS-4514. AllocateBlock : lookup and update open file table for the given path

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMAllocateBlockRequestV1.java
##########
@@ -0,0 +1,91 @@
+/**
+ * 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 org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.helpers.OzoneFSUtils;
+import org.apache.hadoop.ozone.om.request.file.OMFileRequest;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.key.OMAllocateBlockResponseV1;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Iterator;
+
+/**
+ * Handles allocate block request layout version V1.
+ */
+public class OMAllocateBlockRequestV1 extends OMAllocateBlockRequest {
+
+  private static final Logger LOG =
+          LoggerFactory.getLogger(OMAllocateBlockRequestV1.class);
+
+  public OMAllocateBlockRequestV1(OMRequest omRequest) {
+    super(omRequest);
+  }
+
+  protected OmKeyInfo getOpenKeyInfo(OMMetadataManager omMetadataManager,
+      String openKeyName, String keyName) throws IOException {
+    String fileName = OzoneFSUtils.getFileName(keyName);
+    return OMFileRequest.getOmKeyInfoFromFileTable(true,
+            omMetadataManager, openKeyName, fileName);
+  }
+
+  protected String getOpenKeyName(String volumeName, String bucketName,
+      String keyName, long clientID, OzoneManager ozoneManager)
+          throws IOException {
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+    String bucketKey = omMetadataManager.getBucketKey(volumeName, bucketName);
+    OmBucketInfo omBucketInfo =
+            omMetadataManager.getBucketTable().get(bucketKey);
+    long bucketId = omBucketInfo.getObjectID();
+    String fileName = OzoneFSUtils.getFileName(keyName);
+    Iterator<Path> pathComponents = Paths.get(keyName).iterator();

Review comment:
       I am fine with that, if we have consensus.
   As, previous thought of not acquiring bucket lock for allocateBlock is the same reason.




----------------------------------------------------------------
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 #1679: HDDS-4514. AllocateBlock : lookup and update open file table for the given path

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



##########
File path: hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/key/TestOMAllocateBlockResponseV1.java
##########
@@ -0,0 +1,84 @@
+/**
+ * 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.response.key;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.om.OMConfigKeys;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerRatisUtils;
+import org.apache.hadoop.ozone.om.request.TestOMRequestUtils;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.apache.hadoop.util.Time;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * Tests OMAllocateBlockResponse layout version V1.
+ */
+public class TestOMAllocateBlockResponseV1
+        extends TestOMAllocateBlockResponse {
+
+  // logical ID, which really doesn't exist in dirTable
+  private long parentID = 10;
+  private String fileName = "file1";
+
+  protected OmKeyInfo createOmKeyInfo() throws Exception {
+    // need to initialize parentID
+    String parentDir = keyName;
+    keyName = parentDir + OzoneConsts.OM_KEY_PREFIX + fileName;
+
+    long txnId = 50;
+    long objectId = parentID + 1;
+
+    OmKeyInfo omKeyInfoV1 =
+            TestOMRequestUtils.createOmKeyInfo(volumeName, bucketName, keyName,
+                    HddsProtos.ReplicationType.RATIS,
+                    HddsProtos.ReplicationFactor.ONE, objectId, parentID, txnId,
+                    Time.now());
+    return omKeyInfoV1;
+  }
+
+  protected String getOpenKey() throws Exception {
+    return omMetadataManager.getOpenFileName(
+            parentID, fileName, clientID);
+  }
+
+  @NotNull
+  protected OMAllocateBlockResponse getOmAllocateBlockResponse(
+          OmKeyInfo omKeyInfo, OmVolumeArgs omVolumeArgs,
+          OmBucketInfo omBucketInfo, OMResponse omResponse) {
+    return new OMAllocateBlockResponseV1(omResponse, omKeyInfo, clientID,
+            omVolumeArgs, omBucketInfo);
+  }
+
+  @NotNull
+  @Override
+  protected OzoneConfiguration getOzoneConfiguration() {
+    OzoneConfiguration config = super.getOzoneConfiguration();
+    config.set(OMConfigKeys.OZONE_OM_LAYOUT_VERSION, "V1");
+    // OzoneManager#start() is not invoked in this test cases. Hence need to

Review comment:
       Yes, its basically for initializing prefix tables. I've updated comment, hope its clear 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


[GitHub] [ozone] bharatviswa504 commented on a change in pull request #1679: HDDS-4514. AllocateBlock : lookup and update open file table for the given path

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMFileRequest.java
##########
@@ -556,9 +560,10 @@ public static OmKeyInfo getOmKeyInfoFromFileTable(boolean openFileTable,
     // For example, the user given key path is '/a/b/c/d/e/file1', then in DB
     // keyName field stores only the leaf node name, which is 'file1'.
     if (dbOmKeyInfo != null) {
-      dbOmKeyInfo.setKeyName(keyName);

Review comment:
       Why do we need this copyObject here?
   Because get() always returns new object. (if it exists in cache, it returns copyObject, if in DB, get new parsed Object from bytes.
   
   
   ```
     public VALUE get(KEY key) throws IOException {
       // Here the metadata lock will guarantee that cache is not updated for same
       // key during get key.
   
       CacheResult<CacheValue<VALUE>> cacheResult =
           cache.lookup(new CacheKey<>(key));
   
       if (cacheResult.getCacheStatus() == EXISTS) {
         return codecRegistry.copyObject(cacheResult.getValue().getCacheValue(),
             valueType);
       } else if (cacheResult.getCacheStatus() == NOT_EXIST) {
         return null;
       } else {
         return getFromTable(key);
       }
     }
   ```

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMFileRequest.java
##########
@@ -851,4 +856,72 @@ private static boolean checkSubFileExists(OmKeyInfo omKeyInfo,
   public static boolean isImmediateChild(long parentId, long ancestorId) {
     return parentId == ancestorId;
   }
+
+
+  /**
+   * Check for directory exists with same name, if it exists throw error.
+   *
+   * @param keyName                  key name
+   * @param ozoneManager             Ozone Manager
+   * @param reachedLastPathComponent true if the path component is a fileName
+   * @throws IOException if directory exists with same name
+   */
+  private static void checkDirectoryAlreadyExists(String keyName,
+      OzoneManager ozoneManager, boolean reachedLastPathComponent)
+          throws IOException {
+    // Reached last component, which would be a file. Returns its parentID.

Review comment:
       Minor: Comment says Returns its parentID.
   
   In code we just check if lastComponent is true and fileSystem enabled, we return exception.
   
   And also as current V1 always assumes fs enabled true, do we need && ozoneManager.getEnableFileSystemPaths() check?
   
   And also these 3 lines can be in the calling method, as it is not a utility method which is called by different classes.

##########
File path: hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/key/TestOMAllocateBlockResponseV1.java
##########
@@ -0,0 +1,84 @@
+/**
+ * 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.response.key;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.om.OMConfigKeys;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerRatisUtils;
+import org.apache.hadoop.ozone.om.request.TestOMRequestUtils;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.apache.hadoop.util.Time;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * Tests OMAllocateBlockResponse layout version V1.
+ */
+public class TestOMAllocateBlockResponseV1
+        extends TestOMAllocateBlockResponse {
+
+  // logical ID, which really doesn't exist in dirTable
+  private long parentID = 10;
+  private String fileName = "file1";
+
+  protected OmKeyInfo createOmKeyInfo() throws Exception {
+    // need to initialize parentID
+    String parentDir = keyName;
+    keyName = parentDir + OzoneConsts.OM_KEY_PREFIX + fileName;
+
+    long txnId = 50;
+    long objectId = parentID + 1;
+
+    OmKeyInfo omKeyInfoV1 =
+            TestOMRequestUtils.createOmKeyInfo(volumeName, bucketName, keyName,
+                    HddsProtos.ReplicationType.RATIS,
+                    HddsProtos.ReplicationFactor.ONE, objectId, parentID, txnId,
+                    Time.now());
+    return omKeyInfoV1;
+  }
+
+  protected String getOpenKey() throws Exception {
+    return omMetadataManager.getOpenFileName(
+            parentID, fileName, clientID);
+  }
+
+  @NotNull
+  protected OMAllocateBlockResponse getOmAllocateBlockResponse(
+          OmKeyInfo omKeyInfo, OmVolumeArgs omVolumeArgs,
+          OmBucketInfo omBucketInfo, OMResponse omResponse) {
+    return new OMAllocateBlockResponseV1(omResponse, omKeyInfo, clientID,
+            omVolumeArgs, omBucketInfo);
+  }
+
+  @NotNull
+  @Override
+  protected OzoneConfiguration getOzoneConfiguration() {
+    OzoneConfiguration config = super.getOzoneConfiguration();
+    config.set(OMConfigKeys.OZONE_OM_LAYOUT_VERSION, "V1");
+    // OzoneManager#start() is not invoked in this test cases. Hence need to

Review comment:
       Is this for getKeyTable where we used this in metadatamanager?

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMAllocateBlockRequest.java
##########
@@ -216,17 +216,15 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
       openKeyInfo.setUpdateID(trxnLogIndex, ozoneManager.isRatisEnabled());
 
       // Add to cache.
-      omMetadataManager.getOpenKeyTable().addCacheEntry(
-          new CacheKey<>(openKeyName),
-          new CacheValue<>(Optional.of(openKeyInfo), trxnLogIndex));
-
+      addOpenTableCacheEntry(trxnLogIndex, omMetadataManager, openKeyName,
+          openKeyInfo);
       omBucketInfo.incrUsedBytes(preAllocatedSpace);
 
       omResponse.setAllocateBlockResponse(AllocateBlockResponse.newBuilder()
           .setKeyLocation(blockLocation).build());
-      omClientResponse = new OMAllocateBlockResponse(omResponse.build(),
-          openKeyInfo, clientID, omVolumeArgs, omBucketInfo.copyObject());
 
+      omClientResponse = getOmClientResponse(clientID, omResponse,

Review comment:
       We don't update omVolumeArgs, but still reading and passing to Response, which is not required.
   Not related to your patch BTW, we can fix this in master and get to branch.

##########
File path: hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/key/TestOMAllocateBlockResponseV1.java
##########
@@ -0,0 +1,84 @@
+/**
+ * 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.response.key;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.om.OMConfigKeys;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerRatisUtils;
+import org.apache.hadoop.ozone.om.request.TestOMRequestUtils;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.apache.hadoop.util.Time;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * Tests OMAllocateBlockResponse layout version V1.
+ */
+public class TestOMAllocateBlockResponseV1
+        extends TestOMAllocateBlockResponse {
+
+  // logical ID, which really doesn't exist in dirTable
+  private long parentID = 10;
+  private String fileName = "file1";
+
+  protected OmKeyInfo createOmKeyInfo() throws Exception {
+    // need to initialize parentID
+    String parentDir = keyName;
+    keyName = parentDir + OzoneConsts.OM_KEY_PREFIX + fileName;
+
+    long txnId = 50;
+    long objectId = parentID + 1;
+
+    OmKeyInfo omKeyInfoV1 =
+            TestOMRequestUtils.createOmKeyInfo(volumeName, bucketName, keyName,
+                    HddsProtos.ReplicationType.RATIS,
+                    HddsProtos.ReplicationFactor.ONE, objectId, parentID, txnId,
+                    Time.now());
+    return omKeyInfoV1;
+  }
+
+  protected String getOpenKey() throws Exception {
+    return omMetadataManager.getOpenFileName(
+            parentID, fileName, clientID);
+  }
+
+  @NotNull
+  protected OMAllocateBlockResponse getOmAllocateBlockResponse(
+          OmKeyInfo omKeyInfo, OmVolumeArgs omVolumeArgs,
+          OmBucketInfo omBucketInfo, OMResponse omResponse) {
+    return new OMAllocateBlockResponseV1(omResponse, omKeyInfo, clientID,
+            omVolumeArgs, omBucketInfo);
+  }
+
+  @NotNull
+  @Override
+  protected OzoneConfiguration getOzoneConfiguration() {
+    OzoneConfiguration config = super.getOzoneConfiguration();
+    config.set(OMConfigKeys.OZONE_OM_LAYOUT_VERSION, "V1");
+    // OzoneManager#start() is not invoked in this test cases. Hence need to

Review comment:
       Not understood this comment and why we need to do this, and is there in many tests, can you provide info why we need to do this?




----------------------------------------------------------------
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] bharatviswa504 commented on a change in pull request #1679: HDDS-4514. AllocateBlock : lookup and update open file table for the given path

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMAllocateBlockRequestV1.java
##########
@@ -0,0 +1,91 @@
+/**
+ * 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 org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.helpers.OzoneFSUtils;
+import org.apache.hadoop.ozone.om.request.file.OMFileRequest;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.key.OMAllocateBlockResponseV1;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Iterator;
+
+/**
+ * Handles allocate block request layout version V1.
+ */
+public class OMAllocateBlockRequestV1 extends OMAllocateBlockRequest {
+
+  private static final Logger LOG =
+          LoggerFactory.getLogger(OMAllocateBlockRequestV1.class);
+
+  public OMAllocateBlockRequestV1(OMRequest omRequest) {
+    super(omRequest);
+  }
+
+  protected OmKeyInfo getOpenKeyInfo(OMMetadataManager omMetadataManager,
+      String openKeyName, String keyName) throws IOException {
+    String fileName = OzoneFSUtils.getFileName(keyName);
+    return OMFileRequest.getOmKeyInfoFromFileTable(true,
+            omMetadataManager, openKeyName, fileName);
+  }
+
+  protected String getOpenKeyName(String volumeName, String bucketName,
+      String keyName, long clientID, OzoneManager ozoneManager)
+          throws IOException {
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+    String bucketKey = omMetadataManager.getBucketKey(volumeName, bucketName);
+    OmBucketInfo omBucketInfo =
+            omMetadataManager.getBucketTable().get(bucketKey);
+    long bucketId = omBucketInfo.getObjectID();
+    String fileName = OzoneFSUtils.getFileName(keyName);
+    Iterator<Path> pathComponents = Paths.get(keyName).iterator();

Review comment:
       Yes, I agree the ideal thing should be doing this under bucket lock whole operation. But that will have performance impact, as previously allocate block was done with out lock. Previous code has a similar issue, even then during key commit, it might not be found out, and also before we don't have atomic delete, so nothing can be really guaranteed.
   
   In new approach deletes are atomic, and when delete only parent dir is removed, and childs are deleted asynchronously. With new approach, this will be figured out during key commit (As we iterate the entire path again). But it will have impact on performance, and client may write some additional block data even when parent dir's are removed. (This is the only a problem for fs semantics, not object-store semantics). This will be eventually figured out during the next allocateblock, keycommit.
   
   So, here we need to make a choice, do we want to detect during allocate block or during commit Key.
   
   cc @mukul1987 @arp7 for comments
   
   Reading code more, only bucket lock is not there when key check, so we should be okay I believe holding lock during traversal? ( I think this is changed during quota change, previously in allocate block we are not holding lock
   




----------------------------------------------------------------
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] bharatviswa504 commented on a change in pull request #1679: HDDS-4514. AllocateBlock : lookup and update open file table for the given path

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMAllocateBlockRequest.java
##########
@@ -219,14 +219,13 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
       omMetadataManager.getOpenKeyTable().addCacheEntry(
           new CacheKey<>(openKeyName),
           new CacheValue<>(Optional.of(openKeyInfo), trxnLogIndex));
-

Review comment:
       Just space only changes.




----------------------------------------------------------------
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 #1679: HDDS-4514. AllocateBlock : lookup and update open file table for the given path

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


   Thank you @linyiqun , @bharatviswa504 for the useful review comments. Merging it to the feature 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: 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 #1679: HDDS-4514. AllocateBlock : lookup and update open file table for the given path

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMAllocateBlockRequestV1.java
##########
@@ -0,0 +1,91 @@
+/**
+ * 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 org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.helpers.OzoneFSUtils;
+import org.apache.hadoop.ozone.om.request.file.OMFileRequest;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.key.OMAllocateBlockResponseV1;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Iterator;
+
+/**
+ * Handles allocate block request layout version V1.
+ */
+public class OMAllocateBlockRequestV1 extends OMAllocateBlockRequest {

Review comment:
       Thanks @bharatviswa504  for the review comments.
   
   I've HDDS-4486 jira task to make proper feature configuration. Presently, I followed same pattern for the V1 classes. Hope thats fine for you?




----------------------------------------------------------------
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] bharatviswa504 commented on a change in pull request #1679: HDDS-4514. AllocateBlock : lookup and update open file table for the given path

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMAllocateBlockRequestV1.java
##########
@@ -0,0 +1,91 @@
+/**
+ * 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 org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.helpers.OzoneFSUtils;
+import org.apache.hadoop.ozone.om.request.file.OMFileRequest;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.key.OMAllocateBlockResponseV1;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Iterator;
+
+/**
+ * Handles allocate block request layout version V1.
+ */
+public class OMAllocateBlockRequestV1 extends OMAllocateBlockRequest {

Review comment:
       Just a question:
   V1 only supports filesystem.paths.enabled true only and old code will be used when filesystem.paths.enabled false.
   I think, we had a discussion some time back, can you provide info.
   
   Is that is the reason only code for true mode is only handled in V1?




----------------------------------------------------------------
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] bharatviswa504 commented on a change in pull request #1679: HDDS-4514. AllocateBlock : lookup and update open file table for the given path

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMAllocateBlockRequestV1.java
##########
@@ -0,0 +1,91 @@
+/**
+ * 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 org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.helpers.OzoneFSUtils;
+import org.apache.hadoop.ozone.om.request.file.OMFileRequest;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.key.OMAllocateBlockResponseV1;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Iterator;
+
+/**
+ * Handles allocate block request layout version V1.
+ */
+public class OMAllocateBlockRequestV1 extends OMAllocateBlockRequest {
+
+  private static final Logger LOG =
+          LoggerFactory.getLogger(OMAllocateBlockRequestV1.class);
+
+  public OMAllocateBlockRequestV1(OMRequest omRequest) {
+    super(omRequest);
+  }
+
+  protected OmKeyInfo getOpenKeyInfo(OMMetadataManager omMetadataManager,
+      String openKeyName, String keyName) throws IOException {
+    String fileName = OzoneFSUtils.getFileName(keyName);
+    return OMFileRequest.getOmKeyInfoFromFileTable(true,
+            omMetadataManager, openKeyName, fileName);
+  }
+
+  protected String getOpenKeyName(String volumeName, String bucketName,
+      String keyName, long clientID, OzoneManager ozoneManager)
+          throws IOException {
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+    String bucketKey = omMetadataManager.getBucketKey(volumeName, bucketName);
+    OmBucketInfo omBucketInfo =
+            omMetadataManager.getBucketTable().get(bucketKey);
+    long bucketId = omBucketInfo.getObjectID();
+    String fileName = OzoneFSUtils.getFileName(keyName);
+    Iterator<Path> pathComponents = Paths.get(keyName).iterator();

Review comment:
       Yes, I agree the ideal thing should be doing this under bucket lock whole operation. But that will have performance impact, as previously allocate block was done with out lock. Previous code has a similar issue, even then during key commit, it might not be found out, and also before we don't have atomic delete, so nothing can be really guaranteed.
   
   In new approach deletes are atomic, and when delete only parent dir is removed, and childs are deleted asynchronously. With new approach, this will be figured out during key commit (As we iterate the entire path again). But it will have impact on performance, and client may write some additional block data even when parent dir's are removed. (This is the only a problem for fs semantics, not object-store semantics). This will be eventually figured out during the next allocateblock, keycommit.
   
   So, here we need to make a choice, do we want to detect during allocate block or during commit Key.
   
   cc @mukul1987 @arp7 for comments
   
   Reading code more, only bucket lock is not there when key check, so we should be okay I believe holding lock during traversal?
   




----------------------------------------------------------------
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 #1679: HDDS-4514. AllocateBlock : lookup and update open file table for the given path

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/key/OMAllocateBlockResponseV1.java
##########
@@ -0,0 +1,63 @@
+/**
+ * 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.response.key;
+
+import org.apache.hadoop.hdds.utils.db.BatchOperation;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.request.file.OMFileRequest;
+import org.apache.hadoop.ozone.om.response.CleanupTableInfo;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+
+import javax.annotation.Nonnull;
+import java.io.IOException;
+
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.OPEN_FILE_TABLE;
+
+/**
+ * Response for AllocateBlock request layout version V1.
+ */
+@CleanupTableInfo(cleanupTables = {OPEN_FILE_TABLE})
+public class OMAllocateBlockResponseV1 extends OMAllocateBlockResponse {
+
+  public OMAllocateBlockResponseV1(@Nonnull OMResponse omResponse,
+      @Nonnull OmKeyInfo omKeyInfo, long clientID,
+      @Nonnull OmVolumeArgs omVolumeArgs, @Nonnull OmBucketInfo omBucketInfo) {
+    super(omResponse, omKeyInfo, clientID, omVolumeArgs, omBucketInfo);
+  }
+
+  @Override
+  public void addToDBBatch(OMMetadataManager omMetadataManager,
+      BatchOperation batchOperation) throws IOException {
+
+    OMFileRequest.addToOpenFileTable(omMetadataManager, batchOperation,
+            getOmKeyInfo(), getClientID());
+
+    // update volume usedBytes.
+    omMetadataManager.getVolumeTable().putWithBatch(batchOperation,
+            omMetadataManager.getVolumeKey(getOmVolumeArgs().getVolume()),
+            getOmVolumeArgs());

Review comment:
       volume usedbytes update can be removed 




----------------------------------------------------------------
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] bharatviswa504 commented on a change in pull request #1679: HDDS-4514. AllocateBlock : lookup and update open file table for the given path

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMAllocateBlockRequestV1.java
##########
@@ -0,0 +1,91 @@
+/**
+ * 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 org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.helpers.OzoneFSUtils;
+import org.apache.hadoop.ozone.om.request.file.OMFileRequest;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.key.OMAllocateBlockResponseV1;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Iterator;
+
+/**
+ * Handles allocate block request layout version V1.
+ */
+public class OMAllocateBlockRequestV1 extends OMAllocateBlockRequest {

Review comment:
       Question:
   V1 only supports filesystem.paths.enabled true only and old code will be used when filesystem.paths.enabled false.
   I think, we had a discussion some time back, can you provide info.
   
   Is that is the reason only code for true mode is only handled in V1?




----------------------------------------------------------------
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 #1679: HDDS-4514. AllocateBlock : lookup and update open file table for the given path

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMAllocateBlockRequestV1.java
##########
@@ -0,0 +1,91 @@
+/**
+ * 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 org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.helpers.OzoneFSUtils;
+import org.apache.hadoop.ozone.om.request.file.OMFileRequest;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.key.OMAllocateBlockResponseV1;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Iterator;
+
+/**
+ * Handles allocate block request layout version V1.
+ */
+public class OMAllocateBlockRequestV1 extends OMAllocateBlockRequest {
+
+  private static final Logger LOG =
+          LoggerFactory.getLogger(OMAllocateBlockRequestV1.class);
+
+  public OMAllocateBlockRequestV1(OMRequest omRequest) {
+    super(omRequest);
+  }
+
+  protected OmKeyInfo getOpenKeyInfo(OMMetadataManager omMetadataManager,
+      String openKeyName, String keyName) throws IOException {
+    String fileName = OzoneFSUtils.getFileName(keyName);
+    return OMFileRequest.getOmKeyInfoFromFileTable(true,
+            omMetadataManager, openKeyName, fileName);
+  }
+
+  protected String getOpenKeyName(String volumeName, String bucketName,
+      String keyName, long clientID, OzoneManager ozoneManager)
+          throws IOException {
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+    String bucketKey = omMetadataManager.getBucketKey(volumeName, bucketName);
+    OmBucketInfo omBucketInfo =
+            omMetadataManager.getBucketTable().get(bucketKey);
+    long bucketId = omBucketInfo.getObjectID();
+    String fileName = OzoneFSUtils.getFileName(keyName);
+    Iterator<Path> pathComponents = Paths.get(keyName).iterator();

Review comment:
       As per the offline discussion with @mukul1987 and @arp7, it is not required to do the get parentID check under BUCKET_LOCK. With this current code flow, it won't make fast fail during race condition case, anyway later it will fail at the key commit operation time and there won't be any correctness issues. Basically it would be good to avoid any extra performance overhead and in future this part can be revisited if someone has any potential issue and required fast fail in allocate block request.
   
   Following is the expected behavior during race condition. Here op-2 arrived first in OM server and simultaneously op-3 also arrived in OM server.
   ```
       op-1:   client-1) Creates openFile  - /a/b/c/d/file1 and written block1
       op-2:   client-2) Invoked delete /a/b/c. This acquired BUCKET_LOCK and performing deletion.
       op-3:   client-1) Invoked AllocateBlock - block2 for /a/b/c/d/file1. Since getParent("/a/b/c/d/file1") is not under BUCKET_LOCK, it saw "/a/b/c/d" parent ID and proceeds to successfully allocate block2.
       op-4:   client-1) CommitKey /a/b/c/d/file1 - Fail as parent doesn't exist.
   ```
   @bharatviswa504, does this sound good to you?
   




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