You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ozone.apache.org by ck...@apache.org on 2022/10/25 12:09:57 UTC

[ozone] branch master updated: HDDS-7253. Fix exception when '/' in key name (#3774)

This is an automated email from the ASF dual-hosted git repository.

ckj pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ozone.git


The following commit(s) were added to refs/heads/master by this push:
     new b9a47f6bbf HDDS-7253. Fix exception when '/' in key name (#3774)
b9a47f6bbf is described below

commit b9a47f6bbf72605d5391cafc8d382190f822b375
Author: XiChen <32...@users.noreply.github.com>
AuthorDate: Tue Oct 25 20:09:50 2022 +0800

    HDDS-7253. Fix exception when '/' in key name (#3774)
---
 .../hadoop/fs/ozone/TestOzoneFileSystem.java       | 81 ++++++++++++++++++++--
 .../apache/hadoop/ozone/om/TestKeyManagerImpl.java | 70 +++++++++++++++++--
 .../org/apache/hadoop/ozone/om/KeyManagerImpl.java | 68 ++++++++++++++----
 3 files changed, 199 insertions(+), 20 deletions(-)

diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFileSystem.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFileSystem.java
index f3e8cf10be..1f688029b6 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFileSystem.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFileSystem.java
@@ -153,6 +153,7 @@ public class TestOzoneFileSystem {
   private static OzoneManagerProtocol writeClient;
   private static FileSystem fs;
   private static OzoneFileSystem o3fs;
+  private static OzoneBucket ozoneBucket;
   private static String volumeName;
   private static String bucketName;
   private static Trash trash;
@@ -179,10 +180,9 @@ public class TestOzoneFileSystem {
     writeClient = cluster.getRpcClient().getObjectStore()
         .getClientProxy().getOzoneManagerClient();
     // create a volume and a bucket to be used by OzoneFileSystem
-    OzoneBucket bucket =
-        TestDataUtil.createVolumeAndBucket(cluster, bucketLayout);
-    volumeName = bucket.getVolumeName();
-    bucketName = bucket.getName();
+    ozoneBucket = TestDataUtil.createVolumeAndBucket(cluster, bucketLayout);
+    volumeName = ozoneBucket.getVolumeName();
+    bucketName = ozoneBucket.getName();
 
     String rootPath = String.format("%s://%s.%s/",
             OzoneConsts.OZONE_URI_SCHEME, bucketName, volumeName);
@@ -334,6 +334,30 @@ public class TestOzoneFileSystem {
     assertTrue("Shouldn't send error if dir exists", status);
   }
 
+  @Test
+  public void testMakeDirsWithAnFakeDirectory() throws Exception {
+    /*
+     * Op 1. commit a key -> "dir1/dir2/key1"
+     * Op 2. create dir -> "dir1/testDir", the dir1 is a fake dir,
+     *  "dir1/testDir" can be created normal
+     */
+
+    String fakeGrandpaKey = "dir1";
+    String fakeParentKey = fakeGrandpaKey + "/dir2";
+    String fullKeyName = fakeParentKey + "/key1";
+    TestDataUtil.createKey(ozoneBucket, fullKeyName, "");
+
+    // /dir1/dir2 should not exist
+    assertFalse(fs.exists(new Path(fakeParentKey)));
+
+    // /dir1/dir2/key2 should be created because has a fake parent directory
+    Path subdir = new Path(fakeParentKey, "key2");
+    assertTrue(fs.mkdirs(subdir));
+    // the intermediate directories /dir1 and /dir1/dir2 will be created too
+    assertTrue(fs.exists(new Path(fakeGrandpaKey)));
+    assertTrue(fs.exists(new Path(fakeParentKey)));
+  }
+
   @Test
   public void testCreateWithInvalidPaths() throws Exception {
     // Test for path with ..
@@ -727,6 +751,37 @@ public class TestOzoneFileSystem {
     }
   }
 
+  @Test
+  public void testListStatusOnKeyNameContainDelimiter() throws Exception {
+    /*
+    * op1: create a key -> "dir1/dir2/key1"
+    * op2: `ls /` child dir "/dir1/" will be return
+    * op2: `ls /dir1` child dir "/dir1/dir2/" will be return
+    * op3: `ls /dir1/dir2` file "/dir1/dir2/key" will be return
+    *
+    * the "/dir1", "/dir1/dir2/" are fake directory
+    * */
+    String keyName = "dir1/dir2/key1";
+    TestDataUtil.createKey(ozoneBucket, keyName, "");
+    FileStatus[] fileStatuses;
+
+    fileStatuses = fs.listStatus(new Path("/"));
+    assertEquals(1, fileStatuses.length);
+    assertEquals("/dir1", fileStatuses[0].getPath().toUri().getPath());
+    assertTrue(fileStatuses[0].isDirectory());
+
+    fileStatuses = fs.listStatus(new Path("/dir1"));
+    assertEquals(1, fileStatuses.length);
+    assertEquals("/dir1/dir2", fileStatuses[0].getPath().toUri().getPath());
+    assertTrue(fileStatuses[0].isDirectory());
+
+    fileStatuses = fs.listStatus(new Path("/dir1/dir2"));
+    assertEquals(1, fileStatuses.length);
+    assertEquals("/dir1/dir2/key1",
+        fileStatuses[0].getPath().toUri().getPath());
+    assertTrue(fileStatuses[0].isFile());
+  }
+
   /**
    * Cleanup files and directories.
    *
@@ -1273,6 +1328,24 @@ public class TestOzoneFileSystem {
             "file1")));
   }
 
+  @Test
+  public void testRenameContainDelimiterFile() throws Exception {
+    String fakeGrandpaKey = "dir1";
+    String fakeParentKey = fakeGrandpaKey + "/dir2";
+    String sourceKeyName = fakeParentKey + "/key1";
+    String targetKeyName = fakeParentKey +  "/key2";
+    TestDataUtil.createKey(ozoneBucket, sourceKeyName, "");
+
+    Path sourcePath = new Path(fs.getUri().toString() + "/" + sourceKeyName);
+    Path targetPath = new Path(fs.getUri().toString() + "/" + targetKeyName);
+    assertTrue(fs.rename(sourcePath, targetPath));
+    assertFalse(fs.exists(sourcePath));
+    assertTrue(fs.exists(targetPath));
+    // intermediate directories will not be created
+    assertFalse(fs.exists(new Path(fakeGrandpaKey)));
+    assertFalse(fs.exists(new Path(fakeParentKey)));
+  }
+
 
   /**
    * Fails if the (a) parent of dst does not exist or (b) parent is a file.
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestKeyManagerImpl.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestKeyManagerImpl.java
index 492173b71c..e30d27fc74 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestKeyManagerImpl.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestKeyManagerImpl.java
@@ -800,19 +800,20 @@ public class TestKeyManagerImpl {
     List<OmKeyLocationInfo> locationList =
         keySession.getKeyInfo().getLatestVersionLocations().getLocationList();
     Assert.assertEquals(1, locationList.size());
+    long containerID = locationList.get(0).getContainerID();
     locationInfoList.add(
         new OmKeyLocationInfo.Builder().setPipeline(pipeline)
-            .setBlockID(new BlockID(locationList.get(0).getContainerID(),
+            .setBlockID(new BlockID(containerID,
                 locationList.get(0).getLocalID())).build());
     keyArgs.setLocationInfoList(locationInfoList);
 
     writeClient.commitKey(keyArgs, keySession.getId());
-    ContainerInfo containerInfo = new ContainerInfo.Builder().setContainerID(1L)
-        .setPipelineID(pipeline.getId()).build();
+    ContainerInfo containerInfo = new ContainerInfo.Builder()
+        .setContainerID(containerID).setPipelineID(pipeline.getId()).build();
     List<ContainerWithPipeline> containerWithPipelines = Arrays.asList(
         new ContainerWithPipeline(containerInfo, pipeline));
     when(mockScmContainerClient.getContainerWithPipelineBatch(
-        Arrays.asList(1L))).thenReturn(containerWithPipelines);
+        Arrays.asList(containerID))).thenReturn(containerWithPipelines);
 
     OmKeyInfo key = keyManager.lookupKey(keyArgs, null);
     Assert.assertEquals(key.getKeyName(), keyName);
@@ -1273,6 +1274,67 @@ public class TestKeyManagerImpl {
     }
   }
 
+  @Test
+  public void testGetFileStatus() throws IOException {
+    // create a key
+    String keyName = RandomStringUtils.randomAlphabetic(5);
+    OmKeyArgs keyArgs = createBuilder()
+        .setKeyName(keyName)
+        .setLatestVersionLocation(true)
+        .build();
+    writeClient.createFile(keyArgs, false, false);
+    OpenKeySession keySession = writeClient.createFile(keyArgs, true, true);
+    keyArgs.setLocationInfoList(
+        keySession.getKeyInfo().getLatestVersionLocations().getLocationList());
+    writeClient.commitKey(keyArgs, keySession.getId());
+    OzoneFileStatus ozoneFileStatus = keyManager.getFileStatus(keyArgs);
+    Assert.assertEquals(keyName, ozoneFileStatus.getKeyInfo().getFileName());
+  }
+
+  @Test
+  public void testGetFileStatusWithFakeDir() throws IOException {
+    String parentDir = "dir1";
+    String key = "key1";
+    String fullKeyName = parentDir + OZONE_URI_DELIMITER + key;
+    OzoneFileStatus ozoneFileStatus;
+
+    // create a key "dir1/key1"
+    OmKeyArgs keyArgs = createBuilder().setKeyName(fullKeyName).build();
+    OpenKeySession keySession = writeClient.openKey(keyArgs);
+    keyArgs.setLocationInfoList(
+        keySession.getKeyInfo().getLatestVersionLocations().getLocationList());
+    writeClient.commitKey(keyArgs, keySession.getId());
+
+    // verify
+    String keyArg;
+    keyArg = metadataManager.getOzoneKey(VOLUME_NAME, BUCKET_NAME, parentDir);
+    Assert.assertNull(
+        metadataManager.getKeyTable(getDefaultBucketLayout()).get(keyArg));
+    keyArg = metadataManager.getOzoneKey(VOLUME_NAME, BUCKET_NAME, fullKeyName);
+    Assert.assertNotNull(metadataManager.getKeyTable(getDefaultBucketLayout())
+        .get(keyArg));
+
+    // get a non-existing "dir1", since the key is prefixed "dir1/key1",
+    // a fake "/dir1" will be returned
+    keyArgs = createBuilder().setKeyName(parentDir).build();
+    ozoneFileStatus = keyManager.getFileStatus(keyArgs);
+    Assert.assertEquals(parentDir, ozoneFileStatus.getKeyInfo().getFileName());
+    Assert.assertTrue(ozoneFileStatus.isDirectory());
+
+    // get a non-existing "dir", since the key is not prefixed "dir1/key1",
+    // a `OMException` will be thrown
+    keyArgs = createBuilder().setKeyName("dir").build();
+    OmKeyArgs finalKeyArgs = keyArgs;
+    Assert.assertThrows(OMException.class, () -> keyManager.getFileStatus(
+        finalKeyArgs));
+
+    // get a file "dir1/key1"
+    keyArgs = createBuilder().setKeyName(fullKeyName).build();
+    ozoneFileStatus = keyManager.getFileStatus(keyArgs);
+    Assert.assertEquals(key, ozoneFileStatus.getKeyInfo().getFileName());
+    Assert.assertTrue(ozoneFileStatus.isFile());
+  }
+
   @Test
   public void testRefreshPipeline() throws Exception {
 
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java
index dd66c33a1e..483a2e88b0 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java
@@ -1209,6 +1209,8 @@ public class KeyManagerImpl implements KeyManager {
     final String keyName = args.getKeyName();
 
     OmKeyInfo fileKeyInfo = null;
+    OmKeyInfo dirKeyInfo = null;
+    OmKeyInfo fakeDirKeyInfo = null;
     metadataManager.getLock().acquireReadLock(BUCKET_LOCK, volumeName,
         bucketName);
     try {
@@ -1221,28 +1223,27 @@ public class KeyManagerImpl implements KeyManager {
       // Check if the key is a file.
       String fileKeyBytes = metadataManager.getOzoneKey(
               volumeName, bucketName, keyName);
-      fileKeyInfo = metadataManager
-          .getKeyTable(getBucketLayout(metadataManager, volumeName, bucketName))
-          .get(fileKeyBytes);
+      BucketLayout layout =
+          getBucketLayout(metadataManager, volumeName, bucketName);
+      fileKeyInfo = metadataManager.getKeyTable(layout).get(fileKeyBytes);
+      String dirKey = OzoneFSUtils.addTrailingSlashIfNeeded(keyName);
 
       // Check if the key is a directory.
       if (fileKeyInfo == null) {
-        String dirKey = OzoneFSUtils.addTrailingSlashIfNeeded(keyName);
         String dirKeyBytes = metadataManager.getOzoneKey(
                 volumeName, bucketName, dirKey);
-        OmKeyInfo dirKeyInfo = metadataManager.getKeyTable(
-                getBucketLayout(metadataManager, volumeName, bucketName))
-            .get(dirKeyBytes);
-        if (dirKeyInfo != null) {
-          return new OzoneFileStatus(dirKeyInfo, scmBlockSize, true);
+        dirKeyInfo = metadataManager.getKeyTable(layout).get(dirKeyBytes);
+        if (dirKeyInfo == null) {
+          fakeDirKeyInfo =
+              createFakeDirIfShould(volumeName, bucketName, keyName, layout);
         }
       }
     } finally {
       metadataManager.getLock().releaseReadLock(BUCKET_LOCK, volumeName,
               bucketName);
-
-      // if the key is a file then do refresh pipeline info in OM by asking SCM
       if (fileKeyInfo != null) {
+        // if the key is a file
+        // then do refresh pipeline info in OM by asking SCM
         if (args.getLatestVersionLocation()) {
           slimLocationVersion(fileKeyInfo);
         }
@@ -1257,10 +1258,21 @@ public class KeyManagerImpl implements KeyManager {
             sortDatanodes(clientAddress, fileKeyInfo);
           }
         }
-        return new OzoneFileStatus(fileKeyInfo, scmBlockSize, false);
       }
     }
 
+    if (fileKeyInfo != null) {
+      return new OzoneFileStatus(fileKeyInfo, scmBlockSize, false);
+    }
+
+    if (dirKeyInfo != null) {
+      return new OzoneFileStatus(dirKeyInfo, scmBlockSize, true);
+    }
+
+    if (fakeDirKeyInfo != null) {
+      return new OzoneFileStatus(fakeDirKeyInfo, scmBlockSize, true);
+    }
+
     // Key is not found, throws exception
     if (LOG.isDebugEnabled()) {
       LOG.debug("Unable to get file status for the key: volume: {}, bucket:" +
@@ -1272,6 +1284,37 @@ public class KeyManagerImpl implements KeyManager {
             FILE_NOT_FOUND);
   }
 
+  /**
+   * Create a fake directory if the key is a path prefix,
+   * otherwise returns null.
+   * Some keys may contain '/' Ozone will treat '/' as directory separator
+   * such as : key name is 'a/b/c', 'a' and 'b' may not really exist,
+   * but Ozone treats 'a' and 'b' as a directory.
+   * we need create a fake directory 'a' or 'a/b'
+   *
+   * @return OmKeyInfo if the key is a path prefix, otherwise returns null.
+   */
+  private OmKeyInfo createFakeDirIfShould(String volume, String bucket,
+      String keyName, BucketLayout layout) throws IOException {
+    OmKeyInfo fakeDirKeyInfo = null;
+    String dirKey = OzoneFSUtils.addTrailingSlashIfNeeded(keyName);
+    String fileKeyBytes = metadataManager.getOzoneKey(volume, bucket, keyName);
+    Table.KeyValue<String, OmKeyInfo> keyValue =
+            metadataManager.getKeyTable(layout).iterator().seek(fileKeyBytes);
+
+    if (keyValue != null) {
+      Path fullPath = Paths.get(keyValue.getValue().getKeyName());
+      Path subPath = Paths.get(dirKey);
+      OmKeyInfo omKeyInfo = keyValue.getValue();
+      if (fullPath.startsWith(subPath)) {
+        // create fake directory
+        fakeDirKeyInfo = createDirectoryKey(omKeyInfo, dirKey);
+      }
+    }
+
+    return fakeDirKeyInfo;
+  }
+
 
   private OzoneFileStatus getOzoneFileStatusFSO(OmKeyArgs args,
       String clientAddress, boolean skipFileNotFoundError) throws IOException {
@@ -1349,6 +1392,7 @@ public class KeyManagerImpl implements KeyManager {
         .setVolumeName(keyInfo.getVolumeName())
         .setBucketName(keyInfo.getBucketName())
         .setKeyName(dir)
+        .setFileName(OzoneFSUtils.getFileName(keyName))
         .setOmKeyLocationInfos(Collections.singletonList(
             new OmKeyLocationInfoGroup(0, new ArrayList<>())))
         .setCreationTime(Time.now())


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