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 2022/01/04 22:48:39 UTC

[GitHub] [ozone] GeorgeJahad opened a new pull request #2961: HDDS-3231. Cleanup KeyManagerImpl

GeorgeJahad opened a new pull request #2961:
URL: https://github.com/apache/ozone/pull/2961


   ## What changes were proposed in this pull request?
   
   When the OM-HA code was added, the KeyManager tests were not completely updated. Many of them still call the KeyManager "write-path" methods even though those methods have been superceded by the HA methods. For this reason the obsolete methods in the KeyManager class could not be removed.
   
   This PR completes that work and removes the obsolete methods from Keymanager and KeyManagerImpl classes.
   
   #### Replaced keyManager reference with writeClient reference
   Most of the tests were fixed by replacing the keyManager reference to the obsolete methods.  Now, instead of invoking the methods through a keyManager, they are invoked with a reference to an rpcClient, ("writeClient").  This in turn invokes the correct OM write path methods, instead of the obsolete KeyManager ones.
   
   #### OmTestManagers
   To facilitate the creation of this "writeClient", I've added the OmTestManagers class.  It creates the writeClient and the Ozone Manager needed by that client.  It also exposes the other OM managers, (bucketManager, volumeManager, etc,) which are used by the tests.  Since OmTestManagers is used by most of the other classes, it would be good to review that class first.
   
   Most of the changes were fairly straight forword.  The most complicated changes are in TestKeyManagerImpl.java, so it would be good to review those changes after all the others
   
   
   ## What is the link to the Apache JIRA
   
   https://issues.apache.org/jira/browse/HDDS-3231
   
   also:
   https://issues.apache.org/jira/browse/HDDS-3956
   
   ## How was this patch tested?
   All tests have been updated.
   


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

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

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



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


[GitHub] [ozone] GeorgeJahad commented on a change in pull request #2961: HDDS-3231. Cleanup KeyManagerImpl

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



##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestKeyManagerImpl.java
##########
@@ -442,22 +461,17 @@ public void testCheckAccessForFileKey() throws Exception {
     OmKeyArgs keyArgs = createBuilder()
         .setKeyName("testdir/deep/NOTICE.txt")
         .build();
-    OpenKeySession keySession = keyManager.createFile(keyArgs, false, true);
+    OpenKeySession keySession = writeClient.createFile(keyArgs, false, true);
     keyArgs.setLocationInfoList(
         keySession.getKeyInfo().getLatestVersionLocations().getLocationList());
-    keyManager.commitKey(keyArgs, keySession.getId());
+    writeClient.commitKey(keyArgs, keySession.getId());
 
     OzoneObj fileKey = OzoneObjInfo.Builder.fromKeyArgs(keyArgs)
         .setStoreType(OzoneObj.StoreType.OZONE)
         .build();
     RequestContext context = currentUserReads();
     Assert.assertTrue(keyManager.checkAccess(fileKey, context));
 
-    OzoneObj parentDirKey = OzoneObjInfo.Builder.fromKeyArgs(keyArgs)
-        .setStoreType(OzoneObj.StoreType.OZONE)
-        .setKeyName("testdir")
-        .build();
-    Assert.assertTrue(keyManager.checkAccess(parentDirKey, context));

Review comment:
       This test tries to check the access of the parent directory.
   
   Before my changes, the createFile() method directly invoked the keymanager; and the parent dir isn't created.
   Then checkAccess() of the parentDir returns true and the assert passes because of this code:
   https://github.com/apache/ozone/blob/644ef956a0dec62c5427189ed44e293736e7970a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java#L1851
   
   Now the createFile() method goes through the writeClient; the missing parent dir is created by the OmFileCreateRequest. so then the checkAccess() assert of the parentDir fails.
   
   Thus, I think this parentDir test is no longer valid, and I removed it.
   




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

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

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



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


[GitHub] [ozone] GeorgeJahad commented on a change in pull request #2961: HDDS-3231. Cleanup KeyManagerImpl

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



##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestKeyManagerImpl.java
##########
@@ -1238,123 +1260,113 @@ public void testListStatus() throws IOException {
   @Test
   public void testRefreshPipeline() throws Exception {
 
-    MiniOzoneCluster cluster = MiniOzoneCluster.newBuilder(conf).build();
-    try {
-      cluster.waitForClusterToBeReady();
-      OzoneManager ozoneManager = cluster.getOzoneManager();
+    OzoneManager ozoneManager = om;

Review comment:
       This function has only really been changed to remove the cluster, (which has been replaced by the TestOmManager's OzoneManager.)  Removing the try block made it look like there were many other changes but they are just whitespace.
   




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

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

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



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


[GitHub] [ozone] GeorgeJahad commented on a change in pull request #2961: HDDS-3231. Cleanup KeyManagerImpl

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



##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestKeyManagerImpl.java
##########
@@ -442,22 +461,17 @@ public void testCheckAccessForFileKey() throws Exception {
     OmKeyArgs keyArgs = createBuilder()
         .setKeyName("testdir/deep/NOTICE.txt")
         .build();
-    OpenKeySession keySession = keyManager.createFile(keyArgs, false, true);
+    OpenKeySession keySession = writeClient.createFile(keyArgs, false, true);
     keyArgs.setLocationInfoList(
         keySession.getKeyInfo().getLatestVersionLocations().getLocationList());
-    keyManager.commitKey(keyArgs, keySession.getId());
+    writeClient.commitKey(keyArgs, keySession.getId());
 
     OzoneObj fileKey = OzoneObjInfo.Builder.fromKeyArgs(keyArgs)
         .setStoreType(OzoneObj.StoreType.OZONE)
         .build();
     RequestContext context = currentUserReads();
     Assert.assertTrue(keyManager.checkAccess(fileKey, context));
 
-    OzoneObj parentDirKey = OzoneObjInfo.Builder.fromKeyArgs(keyArgs)
-        .setStoreType(OzoneObj.StoreType.OZONE)
-        .setKeyName("testdir")
-        .build();
-    Assert.assertTrue(keyManager.checkAccess(parentDirKey, context));

Review comment:
       Before these changes, the createFile() method directly invoked the keymanager; and the parent dir isn't created.
   Then checkAccess() of the parentDir returns true because of this code:
   https://github.com/apache/ozone/blob/644ef956a0dec62c5427189ed44e293736e7970a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java#L1851
   
   Now the createFile() method goes through the writeClient; the missing parent dir is created by the OmFileCreateRequest.
   so then the checkAccess() assert of the parentDir fails.
   
   Thus, I think this parentDir test is no longer valid, and I removed it.
   




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

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

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



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


[GitHub] [ozone] GeorgeJahad commented on a change in pull request #2961: HDDS-3231. Cleanup KeyManagerImpl

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



##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestKeyManagerImpl.java
##########
@@ -822,7 +836,7 @@ public void testLookupKeyWithLocation() throws IOException {
   @Test
   public void testLatestLocationVersion() throws IOException {
     String keyName = RandomStringUtils.randomAlphabetic(5);
-    OmKeyArgs keyArgs = createBuilder()
+    OmKeyArgs keyArgs = createBuilder(VERSIONED_BUCKET_NAME)

Review comment:
       this test was failing because it expects the old "versioned" style of bucket, which is no longer turned on by default.  So modified this test to use a versioned bucket.  (I also had to mock the Pipeline so that the request would succeed.)
   




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

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

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



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


[GitHub] [ozone] GeorgeJahad commented on a change in pull request #2961: HDDS-3231. Cleanup KeyManagerImpl

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



##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestKeyManagerImpl.java
##########
@@ -338,32 +365,24 @@ public void testCreateDirectory() throws IOException {
     keyArgs = createBuilder()
         .setKeyName(keyName)
         .build();
-    OpenKeySession keySession = keyManager.openKey(keyArgs);
+    OpenKeySession keySession = writeClient.openKey(keyArgs);
     keyArgs.setLocationInfoList(
         keySession.getKeyInfo().getLatestVersionLocations().getLocationList());
-    keyManager.commitKey(keyArgs, keySession.getId());
+    writeClient.commitKey(keyArgs, keySession.getId());
     try {
-      keyManager.createDirectory(keyArgs);
+      writeClient.createDirectory(keyArgs);
       Assert.fail("Creation should fail for directory.");
     } catch (OMException e) {
       Assert.assertEquals(e.getResult(),
           OMException.ResultCodes.FILE_ALREADY_EXISTS);
     }
 
-    // create directory for root directory
-    keyName = "";
-    keyArgs = createBuilder()
-        .setKeyName(keyName)
-        .build();
-    keyManager.createDirectory(keyArgs);
-    Assert.assertTrue(keyManager.getFileStatus(keyArgs).isDirectory());
-

Review comment:
       I removed this test because I think it is invalid.  See [OMDirectoryCreateRequest](https://github.com/apache/ozone/blob/c4579ec5427758f6fa53b555b8de05a9d3f4d222/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMDirectoryCreateRequest.java#L175) (And it fails now that I'm invoking createDirectory through the write-path.)
   




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

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

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



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


[GitHub] [ozone] GeorgeJahad commented on pull request #2961: HDDS-3231. Cleanup KeyManagerImpl

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


   @bharatviswa504 thank you for all the help!


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

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

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



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


[GitHub] [ozone] GeorgeJahad commented on a change in pull request #2961: HDDS-3231. Cleanup KeyManagerImpl

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



##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestKeyManagerImpl.java
##########
@@ -442,22 +461,17 @@ public void testCheckAccessForFileKey() throws Exception {
     OmKeyArgs keyArgs = createBuilder()
         .setKeyName("testdir/deep/NOTICE.txt")
         .build();
-    OpenKeySession keySession = keyManager.createFile(keyArgs, false, true);
+    OpenKeySession keySession = writeClient.createFile(keyArgs, false, true);
     keyArgs.setLocationInfoList(
         keySession.getKeyInfo().getLatestVersionLocations().getLocationList());
-    keyManager.commitKey(keyArgs, keySession.getId());
+    writeClient.commitKey(keyArgs, keySession.getId());
 
     OzoneObj fileKey = OzoneObjInfo.Builder.fromKeyArgs(keyArgs)
         .setStoreType(OzoneObj.StoreType.OZONE)
         .build();
     RequestContext context = currentUserReads();
     Assert.assertTrue(keyManager.checkAccess(fileKey, context));
 
-    OzoneObj parentDirKey = OzoneObjInfo.Builder.fromKeyArgs(keyArgs)
-        .setStoreType(OzoneObj.StoreType.OZONE)
-        .setKeyName("testdir")
-        .build();
-    Assert.assertTrue(keyManager.checkAccess(parentDirKey, context));

Review comment:
       This test tries to check the access of the parent directory, and I think it is invalid.
   
   Before my changes, the createFile() method directly invoked the keymanager; and the parent dir isn't created.
   Then checkAccess() of the parentDir returns true and the assert passes because of this code:
   https://github.com/apache/ozone/blob/644ef956a0dec62c5427189ed44e293736e7970a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java#L1851
   
   Now the createFile() method goes through the writeClient; the missing parent dir is created by the OmFileCreateRequest. so then the checkAccess() assert of the parentDir fails.
   
   Thus, I think this parentDir test is no longer valid, and I removed it.
   




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

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

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



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


[GitHub] [ozone] GeorgeJahad commented on a change in pull request #2961: HDDS-3231. Cleanup KeyManagerImpl

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



##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestKeyManagerImpl.java
##########
@@ -442,22 +461,17 @@ public void testCheckAccessForFileKey() throws Exception {
     OmKeyArgs keyArgs = createBuilder()
         .setKeyName("testdir/deep/NOTICE.txt")
         .build();
-    OpenKeySession keySession = keyManager.createFile(keyArgs, false, true);
+    OpenKeySession keySession = writeClient.createFile(keyArgs, false, true);
     keyArgs.setLocationInfoList(
         keySession.getKeyInfo().getLatestVersionLocations().getLocationList());
-    keyManager.commitKey(keyArgs, keySession.getId());
+    writeClient.commitKey(keyArgs, keySession.getId());
 
     OzoneObj fileKey = OzoneObjInfo.Builder.fromKeyArgs(keyArgs)
         .setStoreType(OzoneObj.StoreType.OZONE)
         .build();
     RequestContext context = currentUserReads();
     Assert.assertTrue(keyManager.checkAccess(fileKey, context));
 
-    OzoneObj parentDirKey = OzoneObjInfo.Builder.fromKeyArgs(keyArgs)
-        .setStoreType(OzoneObj.StoreType.OZONE)
-        .setKeyName("testdir")
-        .build();
-    Assert.assertTrue(keyManager.checkAccess(parentDirKey, context));

Review comment:
       Before these changes, the createFile() method directly invoked the keymanager; and the parent dir isn't created.
   Then checkAccess() of the parentDir returns true and the assert passes because of this code:
   https://github.com/apache/ozone/blob/644ef956a0dec62c5427189ed44e293736e7970a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java#L1851
   
   Now the createFile() method goes through the writeClient; the missing parent dir is created by the OmFileCreateRequest. so then the checkAccess() assert of the parentDir fails.
   
   Thus, I think this parentDir test is no longer valid, and I removed it.
   




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

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

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



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


[GitHub] [ozone] bharatviswa504 merged pull request #2961: HDDS-3231. Cleanup KeyManagerImpl

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


   


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

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

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



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


[GitHub] [ozone] GeorgeJahad commented on pull request #2961: HDDS-3231. Cleanup KeyManagerImpl

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


   FYI @bharatviswa504 this is the PR we've been discussing.  


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

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

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



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


[GitHub] [ozone] GeorgeJahad commented on a change in pull request #2961: HDDS-3231. Cleanup KeyManagerImpl

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



##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestKeyManagerImpl.java
##########
@@ -1238,123 +1260,113 @@ public void testListStatus() throws IOException {
   @Test
   public void testRefreshPipeline() throws Exception {
 
-    MiniOzoneCluster cluster = MiniOzoneCluster.newBuilder(conf).build();
-    try {
-      cluster.waitForClusterToBeReady();
-      OzoneManager ozoneManager = cluster.getOzoneManager();
+    OzoneManager ozoneManager = om;
 
-      StorageContainerLocationProtocol sclProtocolMock = mock(
-          StorageContainerLocationProtocol.class);
+    StorageContainerLocationProtocol sclProtocolMock = mock(
+        StorageContainerLocationProtocol.class);
 
-      List<Long> containerIDs = new ArrayList<>();
-      containerIDs.add(100L);
-      containerIDs.add(200L);
+    List<Long> containerIDs = new ArrayList<>();
+    containerIDs.add(100L);
+    containerIDs.add(200L);
 
-      List<ContainerWithPipeline> cps = new ArrayList<>();
-      for (Long containerID : containerIDs) {
-        ContainerWithPipeline containerWithPipelineMock =
-            mock(ContainerWithPipeline.class);
-        when(containerWithPipelineMock.getPipeline())
-            .thenReturn(getRandomPipeline());
+    List<ContainerWithPipeline> cps = new ArrayList<>();
+    for (Long containerID : containerIDs) {
+      ContainerWithPipeline containerWithPipelineMock =
+          mock(ContainerWithPipeline.class);
+      when(containerWithPipelineMock.getPipeline())
+          .thenReturn(getRandomPipeline());
 
-        ContainerInfo ci = mock(ContainerInfo.class);
-        when(ci.getContainerID()).thenReturn(containerID);
-        when(containerWithPipelineMock.getContainerInfo()).thenReturn(ci);
+      ContainerInfo ci = mock(ContainerInfo.class);
+      when(ci.getContainerID()).thenReturn(containerID);
+      when(containerWithPipelineMock.getContainerInfo()).thenReturn(ci);
 
-        cps.add(containerWithPipelineMock);
-      }
+      cps.add(containerWithPipelineMock);
+    }
 
-      when(sclProtocolMock.getContainerWithPipelineBatch(containerIDs))
-          .thenReturn(cps);
+    when(sclProtocolMock.getContainerWithPipelineBatch(containerIDs))
+        .thenReturn(cps);
 
-      ScmClient scmClientMock = mock(ScmClient.class);
-      when(scmClientMock.getContainerClient()).thenReturn(sclProtocolMock);
+    ScmClient scmClientMock = mock(ScmClient.class);
+    when(scmClientMock.getContainerClient()).thenReturn(sclProtocolMock);
 
-      OmKeyInfo omKeyInfo = TestOMRequestUtils.createOmKeyInfo("v1",
-          "b1", "k1", ReplicationType.RATIS,
-          ReplicationFactor.THREE);
+    OmKeyInfo omKeyInfo = TestOMRequestUtils.createOmKeyInfo("v1",
+        "b1", "k1", ReplicationType.RATIS,
+        ReplicationFactor.THREE);
 
-      // Add block to key.
-      List<OmKeyLocationInfo> omKeyLocationInfoList = new ArrayList<>();
-      Pipeline pipeline = getRandomPipeline();
+    // Add block to key.
+    List<OmKeyLocationInfo> omKeyLocationInfoList = new ArrayList<>();
+    Pipeline pipeline = getRandomPipeline();
 
-      OmKeyLocationInfo omKeyLocationInfo =
-          new OmKeyLocationInfo.Builder().setBlockID(
-              new BlockID(100L, 1000L))
-              .setOffset(0).setLength(100L).setPipeline(pipeline).build();
+    OmKeyLocationInfo omKeyLocationInfo =
+        new OmKeyLocationInfo.Builder().setBlockID(
+            new BlockID(100L, 1000L))
+            .setOffset(0).setLength(100L).setPipeline(pipeline).build();
 
-      omKeyLocationInfoList.add(omKeyLocationInfo);
+    omKeyLocationInfoList.add(omKeyLocationInfo);
 
-      OmKeyLocationInfo omKeyLocationInfo2 =
-          new OmKeyLocationInfo.Builder().setBlockID(
-              new BlockID(200L, 1000L))
-              .setOffset(0).setLength(100L).setPipeline(pipeline).build();
-      omKeyLocationInfoList.add(omKeyLocationInfo2);
+    OmKeyLocationInfo omKeyLocationInfo2 =
+        new OmKeyLocationInfo.Builder().setBlockID(
+            new BlockID(200L, 1000L))
+            .setOffset(0).setLength(100L).setPipeline(pipeline).build();
+    omKeyLocationInfoList.add(omKeyLocationInfo2);
 
-      OmKeyLocationInfo omKeyLocationInfo3 =
-          new OmKeyLocationInfo.Builder().setBlockID(
-              new BlockID(100L, 2000L))
-              .setOffset(0).setLength(100L).setPipeline(pipeline).build();
-      omKeyLocationInfoList.add(omKeyLocationInfo3);
+    OmKeyLocationInfo omKeyLocationInfo3 =
+        new OmKeyLocationInfo.Builder().setBlockID(
+            new BlockID(100L, 2000L))
+            .setOffset(0).setLength(100L).setPipeline(pipeline).build();
+    omKeyLocationInfoList.add(omKeyLocationInfo3);
 
-      omKeyInfo.appendNewBlocks(omKeyLocationInfoList, false);
+    omKeyInfo.appendNewBlocks(omKeyLocationInfoList, false);
 
-      KeyManagerImpl keyManagerImpl =
-          new KeyManagerImpl(ozoneManager, scmClientMock, conf, "om1");
+    KeyManagerImpl keyManagerImpl =
+        new KeyManagerImpl(ozoneManager, scmClientMock, conf, "om1");
 
-      keyManagerImpl.refresh(omKeyInfo);
+    keyManagerImpl.refresh(omKeyInfo);
+
+    verify(sclProtocolMock, times(1))
+        .getContainerWithPipelineBatch(containerIDs);
 
-      verify(sclProtocolMock, times(1))
-          .getContainerWithPipelineBatch(containerIDs);
-    } finally {
-      cluster.shutdown();
-    }
   }
 
 
   @Test
   public void testRefreshPipelineException() throws Exception {
-    MiniOzoneCluster cluster = MiniOzoneCluster.newBuilder(conf).build();

Review comment:
       This function has only really been changed to remove the cluster, (which has been replaced by the TestOmManager's OzoneManager.)  Removing the try block made it look like there were many other changes but they are just whitespace.
   




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

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

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



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


[GitHub] [ozone] GeorgeJahad commented on a change in pull request #2961: HDDS-3231. Cleanup KeyManagerImpl

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



##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestKeyManagerImpl.java
##########
@@ -442,22 +461,17 @@ public void testCheckAccessForFileKey() throws Exception {
     OmKeyArgs keyArgs = createBuilder()
         .setKeyName("testdir/deep/NOTICE.txt")
         .build();
-    OpenKeySession keySession = keyManager.createFile(keyArgs, false, true);
+    OpenKeySession keySession = writeClient.createFile(keyArgs, false, true);
     keyArgs.setLocationInfoList(
         keySession.getKeyInfo().getLatestVersionLocations().getLocationList());
-    keyManager.commitKey(keyArgs, keySession.getId());
+    writeClient.commitKey(keyArgs, keySession.getId());
 
     OzoneObj fileKey = OzoneObjInfo.Builder.fromKeyArgs(keyArgs)
         .setStoreType(OzoneObj.StoreType.OZONE)
         .build();
     RequestContext context = currentUserReads();
     Assert.assertTrue(keyManager.checkAccess(fileKey, context));
 
-    OzoneObj parentDirKey = OzoneObjInfo.Builder.fromKeyArgs(keyArgs)
-        .setStoreType(OzoneObj.StoreType.OZONE)
-        .setKeyName("testdir")
-        .build();
-    Assert.assertTrue(keyManager.checkAccess(parentDirKey, context));

Review comment:
       Before these changes, the createFile() method directly invoked the keymanager; and the parent dir isn't created.
   Then checkAccess() of the parentDir returns true and the assert passes because of this code:
   https://github.com/apache/ozone/blob/644ef956a0dec62c5427189ed44e293736e7970a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java#L1851
   
   Now the createFile() method goes through the writeClient; the missing parent dir is created by the OmFileCreateRequest.
   so then the checkAccess() assert of the parentDir fails.
   
   Thus, I think this parentDir test is no longer valid, and I removed it.
   




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

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

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



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


[GitHub] [ozone] GeorgeJahad commented on a change in pull request #2961: HDDS-3231. Cleanup KeyManagerImpl

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



##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestKeyManagerImpl.java
##########
@@ -338,32 +365,24 @@ public void testCreateDirectory() throws IOException {
     keyArgs = createBuilder()
         .setKeyName(keyName)
         .build();
-    OpenKeySession keySession = keyManager.openKey(keyArgs);
+    OpenKeySession keySession = writeClient.openKey(keyArgs);
     keyArgs.setLocationInfoList(
         keySession.getKeyInfo().getLatestVersionLocations().getLocationList());
-    keyManager.commitKey(keyArgs, keySession.getId());
+    writeClient.commitKey(keyArgs, keySession.getId());
     try {
-      keyManager.createDirectory(keyArgs);
+      writeClient.createDirectory(keyArgs);
       Assert.fail("Creation should fail for directory.");
     } catch (OMException e) {
       Assert.assertEquals(e.getResult(),
           OMException.ResultCodes.FILE_ALREADY_EXISTS);
     }
 
-    // create directory for root directory
-    keyName = "";
-    keyArgs = createBuilder()
-        .setKeyName(keyName)
-        .build();
-    keyManager.createDirectory(keyArgs);
-    Assert.assertTrue(keyManager.getFileStatus(keyArgs).isDirectory());
-

Review comment:
       I removed this test because I think it is invalid.  See [OMDirectoryCreateRequest](https://github.com/apache/ozone/blob/c4579ec5427758f6fa53b555b8de05a9d3f4d222/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/file/OMDirectoryCreateRequest.java#L175)
   




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

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

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



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


[GitHub] [ozone] GeorgeJahad commented on a change in pull request #2961: HDDS-3231. Cleanup KeyManagerImpl

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



##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestKeyManagerImpl.java
##########
@@ -822,7 +836,7 @@ public void testLookupKeyWithLocation() throws IOException {
   @Test
   public void testLatestLocationVersion() throws IOException {
     String keyName = RandomStringUtils.randomAlphabetic(5);
-    OmKeyArgs keyArgs = createBuilder()
+    OmKeyArgs keyArgs = createBuilder(VERSIONED_BUCKET_NAME)

Review comment:
       this test was failing because it expects the old "versioned" style of bucket, which is no longer turned on by default.  So fixed this test to turn it on.  (I also had to mock the Pipeline so that the request would succeed.)
   




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

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

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



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


[GitHub] [ozone] GeorgeJahad commented on a change in pull request #2961: HDDS-3231. Cleanup KeyManagerImpl

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



##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestKeyManagerImpl.java
##########
@@ -1238,123 +1260,113 @@ public void testListStatus() throws IOException {
   @Test
   public void testRefreshPipeline() throws Exception {
 
-    MiniOzoneCluster cluster = MiniOzoneCluster.newBuilder(conf).build();
-    try {
-      cluster.waitForClusterToBeReady();
-      OzoneManager ozoneManager = cluster.getOzoneManager();
+    OzoneManager ozoneManager = om;

Review comment:
       This function has only really been changed to remove the cluster, (which has been replaced by the OmTestManagers's OzoneManager.)  Removing the try block made it look like there were many other changes but they are just whitespace.
   

##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestKeyManagerImpl.java
##########
@@ -1238,123 +1260,113 @@ public void testListStatus() throws IOException {
   @Test
   public void testRefreshPipeline() throws Exception {
 
-    MiniOzoneCluster cluster = MiniOzoneCluster.newBuilder(conf).build();
-    try {
-      cluster.waitForClusterToBeReady();
-      OzoneManager ozoneManager = cluster.getOzoneManager();
+    OzoneManager ozoneManager = om;
 
-      StorageContainerLocationProtocol sclProtocolMock = mock(
-          StorageContainerLocationProtocol.class);
+    StorageContainerLocationProtocol sclProtocolMock = mock(
+        StorageContainerLocationProtocol.class);
 
-      List<Long> containerIDs = new ArrayList<>();
-      containerIDs.add(100L);
-      containerIDs.add(200L);
+    List<Long> containerIDs = new ArrayList<>();
+    containerIDs.add(100L);
+    containerIDs.add(200L);
 
-      List<ContainerWithPipeline> cps = new ArrayList<>();
-      for (Long containerID : containerIDs) {
-        ContainerWithPipeline containerWithPipelineMock =
-            mock(ContainerWithPipeline.class);
-        when(containerWithPipelineMock.getPipeline())
-            .thenReturn(getRandomPipeline());
+    List<ContainerWithPipeline> cps = new ArrayList<>();
+    for (Long containerID : containerIDs) {
+      ContainerWithPipeline containerWithPipelineMock =
+          mock(ContainerWithPipeline.class);
+      when(containerWithPipelineMock.getPipeline())
+          .thenReturn(getRandomPipeline());
 
-        ContainerInfo ci = mock(ContainerInfo.class);
-        when(ci.getContainerID()).thenReturn(containerID);
-        when(containerWithPipelineMock.getContainerInfo()).thenReturn(ci);
+      ContainerInfo ci = mock(ContainerInfo.class);
+      when(ci.getContainerID()).thenReturn(containerID);
+      when(containerWithPipelineMock.getContainerInfo()).thenReturn(ci);
 
-        cps.add(containerWithPipelineMock);
-      }
+      cps.add(containerWithPipelineMock);
+    }
 
-      when(sclProtocolMock.getContainerWithPipelineBatch(containerIDs))
-          .thenReturn(cps);
+    when(sclProtocolMock.getContainerWithPipelineBatch(containerIDs))
+        .thenReturn(cps);
 
-      ScmClient scmClientMock = mock(ScmClient.class);
-      when(scmClientMock.getContainerClient()).thenReturn(sclProtocolMock);
+    ScmClient scmClientMock = mock(ScmClient.class);
+    when(scmClientMock.getContainerClient()).thenReturn(sclProtocolMock);
 
-      OmKeyInfo omKeyInfo = TestOMRequestUtils.createOmKeyInfo("v1",
-          "b1", "k1", ReplicationType.RATIS,
-          ReplicationFactor.THREE);
+    OmKeyInfo omKeyInfo = TestOMRequestUtils.createOmKeyInfo("v1",
+        "b1", "k1", ReplicationType.RATIS,
+        ReplicationFactor.THREE);
 
-      // Add block to key.
-      List<OmKeyLocationInfo> omKeyLocationInfoList = new ArrayList<>();
-      Pipeline pipeline = getRandomPipeline();
+    // Add block to key.
+    List<OmKeyLocationInfo> omKeyLocationInfoList = new ArrayList<>();
+    Pipeline pipeline = getRandomPipeline();
 
-      OmKeyLocationInfo omKeyLocationInfo =
-          new OmKeyLocationInfo.Builder().setBlockID(
-              new BlockID(100L, 1000L))
-              .setOffset(0).setLength(100L).setPipeline(pipeline).build();
+    OmKeyLocationInfo omKeyLocationInfo =
+        new OmKeyLocationInfo.Builder().setBlockID(
+            new BlockID(100L, 1000L))
+            .setOffset(0).setLength(100L).setPipeline(pipeline).build();
 
-      omKeyLocationInfoList.add(omKeyLocationInfo);
+    omKeyLocationInfoList.add(omKeyLocationInfo);
 
-      OmKeyLocationInfo omKeyLocationInfo2 =
-          new OmKeyLocationInfo.Builder().setBlockID(
-              new BlockID(200L, 1000L))
-              .setOffset(0).setLength(100L).setPipeline(pipeline).build();
-      omKeyLocationInfoList.add(omKeyLocationInfo2);
+    OmKeyLocationInfo omKeyLocationInfo2 =
+        new OmKeyLocationInfo.Builder().setBlockID(
+            new BlockID(200L, 1000L))
+            .setOffset(0).setLength(100L).setPipeline(pipeline).build();
+    omKeyLocationInfoList.add(omKeyLocationInfo2);
 
-      OmKeyLocationInfo omKeyLocationInfo3 =
-          new OmKeyLocationInfo.Builder().setBlockID(
-              new BlockID(100L, 2000L))
-              .setOffset(0).setLength(100L).setPipeline(pipeline).build();
-      omKeyLocationInfoList.add(omKeyLocationInfo3);
+    OmKeyLocationInfo omKeyLocationInfo3 =
+        new OmKeyLocationInfo.Builder().setBlockID(
+            new BlockID(100L, 2000L))
+            .setOffset(0).setLength(100L).setPipeline(pipeline).build();
+    omKeyLocationInfoList.add(omKeyLocationInfo3);
 
-      omKeyInfo.appendNewBlocks(omKeyLocationInfoList, false);
+    omKeyInfo.appendNewBlocks(omKeyLocationInfoList, false);
 
-      KeyManagerImpl keyManagerImpl =
-          new KeyManagerImpl(ozoneManager, scmClientMock, conf, "om1");
+    KeyManagerImpl keyManagerImpl =
+        new KeyManagerImpl(ozoneManager, scmClientMock, conf, "om1");
 
-      keyManagerImpl.refresh(omKeyInfo);
+    keyManagerImpl.refresh(omKeyInfo);
+
+    verify(sclProtocolMock, times(1))
+        .getContainerWithPipelineBatch(containerIDs);
 
-      verify(sclProtocolMock, times(1))
-          .getContainerWithPipelineBatch(containerIDs);
-    } finally {
-      cluster.shutdown();
-    }
   }
 
 
   @Test
   public void testRefreshPipelineException() throws Exception {
-    MiniOzoneCluster cluster = MiniOzoneCluster.newBuilder(conf).build();

Review comment:
       This function has only really been changed to remove the cluster, (which has been replaced by the OmTestManagers's OzoneManager.)  Removing the try block made it look like there were many other changes but they are just whitespace.
   




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

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

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



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