You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ozone.apache.org by ra...@apache.org on 2021/04/07 03:25:42 UTC

[ozone] 15/29: HDDS-4805. [FSO]Fix findbugs issues after HDDS-2195 (#1906)

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

rakeshr pushed a commit to branch HDDS-2939
in repository https://gitbox.apache.org/repos/asf/ozone.git

commit 68aa4fdf6923b472a4902537037c6da293570994
Author: Rakesh Radhakrishnan <ra...@apache.org>
AuthorDate: Mon Feb 8 19:29:09 2021 +0530

    HDDS-4805. [FSO]Fix findbugs issues after HDDS-2195 (#1906)
---
 .../apache/hadoop/fs/ozone/TestOzoneFileOps.java   | 15 +++-
 .../hadoop/fs/ozone/TestOzoneFileSystem.java       | 60 ++++++++------
 .../hadoop/fs/ozone/TestOzoneFileSystemV1.java     | 94 +++++++++++-----------
 .../hadoop/ozone/client/rpc/TestReadRetries.java   | 10 +--
 .../apache/hadoop/ozone/om/TestObjectStoreV1.java  | 39 ++++++---
 .../file/TestOMDirectoryCreateRequestV1.java       | 20 ++---
 .../om/request/key/TestOMKeyCreateRequestV1.java   |  5 +-
 .../TestS3InitiateMultipartUploadRequestV1.java    |  4 -
 .../file/TestOMDirectoryCreateResponseV1.java      |  2 -
 .../response/file/TestOMFileCreateResponseV1.java  |  6 +-
 .../om/response/key/TestOMKeyCommitResponse.java   |  2 -
 .../om/response/key/TestOMKeyCreateResponseV1.java |  6 +-
 .../TestS3InitiateMultipartUploadResponseV1.java   |  5 --
 13 files changed, 147 insertions(+), 121 deletions(-)

diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFileOps.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFileOps.java
index 12dd51e..176d0c4 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFileOps.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFileOps.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
 import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
 import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
 import org.apache.hadoop.ozone.om.request.TestOMRequestUtils;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.junit.After;
 import org.junit.Before;
@@ -155,12 +156,20 @@ public class TestOzoneFileOps {
     // trigger CommitKeyRequest
     outputStream.close();
 
-    Assert.assertTrue("Failed to commit the open file:" + openFileKey,
-            omMgr.getOpenKeyTable().isEmpty());
-
     OmKeyInfo omKeyInfo = omMgr.getKeyTable().get(openFileKey);
     Assert.assertNotNull("Invalid Key!", omKeyInfo);
     verifyOMFileInfoFormat(omKeyInfo, file.getName(), d2ObjectID);
+
+    // wait for DB updates
+    GenericTestUtils.waitFor(() -> {
+      try {
+        return omMgr.getOpenKeyTable().isEmpty();
+      } catch (IOException e) {
+        LOG.error("DB failure!", e);
+        Assert.fail("DB failure!");
+        return false;
+      }
+    }, 1000, 120000);
   }
 
   private void verifyOMFileInfoFormat(OmKeyInfo omKeyInfo, String fileName,
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 7e91576..a8e89e1 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
@@ -20,6 +20,7 @@ package org.apache.hadoop.fs.ozone;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -125,26 +126,16 @@ public class TestOzoneFileSystem {
   private static final Logger LOG =
       LoggerFactory.getLogger(TestOzoneFileSystem.class);
 
-  @SuppressWarnings("checkstyle:VisibilityModifier")
-  protected static boolean isBucketFSOptimized = false;
-  @SuppressWarnings("checkstyle:VisibilityModifier")
-  protected static boolean enabledFileSystemPaths;
-  @SuppressWarnings("checkstyle:VisibilityModifier")
-  protected static boolean omRatisEnabled;
-  @SuppressWarnings("checkstyle:VisibilityModifier")
-  protected static MiniOzoneCluster cluster;
-  @SuppressWarnings("checkstyle:VisibilityModifier")
-  protected static FileSystem fs;
-  @SuppressWarnings("checkstyle:VisibilityModifier")
-  protected static OzoneFileSystem o3fs;
-  @SuppressWarnings("checkstyle:VisibilityModifier")
-  protected static String volumeName;
-  @SuppressWarnings("checkstyle:VisibilityModifier")
-  protected static String bucketName;
-  @SuppressWarnings("checkstyle:VisibilityModifier")
-  protected static int rootItemCount;
-  @SuppressWarnings("checkstyle:VisibilityModifier")
-  protected static Trash trash;
+  private static boolean isBucketFSOptimized = false;
+  private static boolean enabledFileSystemPaths;
+  private static boolean omRatisEnabled;
+
+  private static MiniOzoneCluster cluster;
+  private static FileSystem fs;
+  private static OzoneFileSystem o3fs;
+  private static String volumeName;
+  private static String bucketName;
+  private static Trash trash;
 
   private void init() throws Exception {
     OzoneConfiguration conf = new OzoneConfiguration();
@@ -203,6 +194,26 @@ public class TestOzoneFileSystem {
     }
   }
 
+  public static FileSystem getFs() {
+    return fs;
+  }
+
+  public static boolean isEnabledFileSystemPaths() {
+    return enabledFileSystemPaths;
+  }
+
+  public static void setIsBucketFSOptimized(boolean isBucketFSO) {
+    isBucketFSOptimized = isBucketFSO;
+  }
+
+  public static String getBucketName() {
+    return bucketName;
+  }
+
+  public static String getVolumeName() {
+    return volumeName;
+  }
+
   @Test
   public void testCreateFileShouldCheckExistenceOfDirWithSameName()
       throws Exception {
@@ -606,7 +617,7 @@ public class TestOzoneFileSystem {
     // Added logs for debugging failures, to check any sub-path mismatches.
     Set<String> actualPaths = new TreeSet<>();
     ArrayList<String> actualPathList = new ArrayList<>();
-    if (rootItemCount != fileStatuses.length) {
+    if (numDirs != fileStatuses.length) {
       for (int i = 0; i < fileStatuses.length; i++) {
         boolean duplicate =
                 actualPaths.add(fileStatuses[i].getPath().getName());
@@ -616,7 +627,7 @@ public class TestOzoneFileSystem {
         }
         actualPathList.add(fileStatuses[i].getPath().getName());
       }
-      if (rootItemCount != actualPathList.size()) {
+      if (numDirs != actualPathList.size()) {
         LOG.info("actualPathsSize: {}", actualPaths.size());
         LOG.info("actualPathListSize: {}", actualPathList.size());
         actualPaths.removeAll(paths);
@@ -643,8 +654,6 @@ public class TestOzoneFileSystem {
     Path root = new Path("/");
     FileStatus[] fileStatuses = fs.listStatus(root);
 
-    rootItemCount = 0; // reset to zero
-
     if (fileStatuses == null) {
       return;
     }
@@ -723,7 +732,7 @@ public class TestOzoneFileSystem {
   public void testAllocateMoreThanOneBlock() throws IOException {
     Path file = new Path("/file");
     String str = "TestOzoneFileSystemV1.testSeekOnFileLength";
-    byte[] strBytes = str.getBytes();
+    byte[] strBytes = str.getBytes(StandardCharsets.UTF_8);
     long numBlockAllocationsOrg =
             cluster.getOzoneManager().getMetrics().getNumBlockAllocates();
 
@@ -1052,7 +1061,6 @@ public class TestOzoneFileSystem {
   @Test
   public void testRenameDir() throws Exception {
     final String dir = "/root_dir/dir1";
-    Path rootDir = new Path(fs.getUri().toString() +  "/root_dir");
     final Path source = new Path(fs.getUri().toString() + dir);
     final Path dest = new Path(source.toString() + ".renamed");
     // Add a sub-dir to the directory to be moved.
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFileSystemV1.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFileSystemV1.java
index ffeb5a3..03846ae 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFileSystemV1.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFileSystemV1.java
@@ -29,9 +29,7 @@ import org.junit.Assert;
 import org.junit.After;
 import org.junit.BeforeClass;
 import org.junit.Ignore;
-import org.junit.Rule;
 import org.junit.Test;
-import org.junit.rules.Timeout;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.slf4j.Logger;
@@ -66,7 +64,7 @@ public class TestOzoneFileSystemV1 extends TestOzoneFileSystem {
 
   @BeforeClass
   public static void init() {
-    isBucketFSOptimized = true;
+    setIsBucketFSOptimized(true);
   }
 
   public TestOzoneFileSystemV1(boolean setDefaultFs, boolean enableOMRatis) {
@@ -85,12 +83,6 @@ public class TestOzoneFileSystemV1 extends TestOzoneFileSystem {
     }
   }
 
-  /**
-   * Set a timeout for each test.
-   */
-  @Rule
-  public Timeout timeout = new Timeout(300000);
-
   private static final Logger LOG =
       LoggerFactory.getLogger(TestOzoneFileSystemV1.class);
 
@@ -106,30 +98,32 @@ public class TestOzoneFileSystemV1 extends TestOzoneFileSystem {
      * Op 7. create file -> /d1/d2/key1
      */
     Path key1 = new Path("/key1");
-    try (FSDataOutputStream outputStream = fs.create(key1, false)) {
+    try (FSDataOutputStream outputStream = getFs().create(key1,
+            false)) {
       assertNotNull("Should be able to create file: key1",
               outputStream);
     }
     Path d1 = new Path("/d1");
     Path dir1Key1 = new Path(d1, "key1");
-    try (FSDataOutputStream outputStream = fs.create(dir1Key1, false)) {
+    try (FSDataOutputStream outputStream = getFs().create(dir1Key1, false)) {
       assertNotNull("Should be able to create file: " + dir1Key1,
               outputStream);
     }
     Path d2 = new Path("/d2");
     Path dir2Key1 = new Path(d2, "key1");
-    try (FSDataOutputStream outputStream = fs.create(dir2Key1, false)) {
+    try (FSDataOutputStream outputStream = getFs().create(dir2Key1, false)) {
       assertNotNull("Should be able to create file: " + dir2Key1,
               outputStream);
     }
     Path dir1Dir2 = new Path("/d1/d2/");
     Path dir1Dir2Key1 = new Path(dir1Dir2, "key1");
-    try (FSDataOutputStream outputStream = fs.create(dir1Dir2Key1, false)) {
+    try (FSDataOutputStream outputStream = getFs().create(dir1Dir2Key1,
+            false)) {
       assertNotNull("Should be able to create file: " + dir1Dir2Key1,
               outputStream);
     }
     Path d1Key2 = new Path(d1, "key2");
-    try (FSDataOutputStream outputStream = fs.create(d1Key2, false)) {
+    try (FSDataOutputStream outputStream = getFs().create(d1Key2, false)) {
       assertNotNull("Should be able to create file: " + d1Key2,
               outputStream);
     }
@@ -137,11 +131,14 @@ public class TestOzoneFileSystemV1 extends TestOzoneFileSystem {
     Path dir1Dir3 = new Path("/d1/d3/");
     Path dir1Dir4 = new Path("/d1/d4/");
 
-    fs.mkdirs(dir1Dir3);
-    fs.mkdirs(dir1Dir4);
+    getFs().mkdirs(dir1Dir3);
+    getFs().mkdirs(dir1Dir4);
+
+    String bucketName = getBucketName();
+    String volumeName = getVolumeName();
 
     // Root Directory
-    FileStatus[] fileStatusList = fs.listStatus(new Path("/"));
+    FileStatus[] fileStatusList = getFs().listStatus(new Path("/"));
     assertEquals("FileStatus should return files and directories",
             3, fileStatusList.length);
     ArrayList<String> expectedPaths = new ArrayList<>();
@@ -155,7 +152,7 @@ public class TestOzoneFileSystemV1 extends TestOzoneFileSystem {
             0, expectedPaths.size());
 
     // level-1 sub-dirs
-    fileStatusList = fs.listStatus(new Path("/d1"));
+    fileStatusList = getFs().listStatus(new Path("/d1"));
     assertEquals("FileStatus should return files and directories",
             5, fileStatusList.length);
     expectedPaths = new ArrayList<>();
@@ -171,7 +168,7 @@ public class TestOzoneFileSystemV1 extends TestOzoneFileSystem {
             0, expectedPaths.size());
 
     // level-2 sub-dirs
-    fileStatusList = fs.listStatus(new Path("/d1/d2"));
+    fileStatusList = getFs().listStatus(new Path("/d1/d2"));
     assertEquals("FileStatus should return files and directories",
             1, fileStatusList.length);
     expectedPaths = new ArrayList<>();
@@ -184,7 +181,7 @@ public class TestOzoneFileSystemV1 extends TestOzoneFileSystem {
             0, expectedPaths.size());
 
     // level-2 key2
-    fileStatusList = fs.listStatus(new Path("/d1/d2/key1"));
+    fileStatusList = getFs().listStatus(new Path("/d1/d2/key1"));
     assertEquals("FileStatus should return files and directories",
             1, fileStatusList.length);
     expectedPaths = new ArrayList<>();
@@ -198,13 +195,13 @@ public class TestOzoneFileSystemV1 extends TestOzoneFileSystem {
 
     // invalid root key
     try {
-      fileStatusList = fs.listStatus(new Path("/key2"));
+      fileStatusList = getFs().listStatus(new Path("/key2"));
       fail("Should throw FileNotFoundException");
     } catch (FileNotFoundException fnfe) {
       // ignore as its expected
     }
     try {
-      fileStatusList = fs.listStatus(new Path("/d1/d2/key2"));
+      fileStatusList = getFs().listStatus(new Path("/d1/d2/key2"));
       fail("Should throw FileNotFoundException");
     } catch (FileNotFoundException fnfe) {
       // ignore as its expected
@@ -221,30 +218,30 @@ public class TestOzoneFileSystemV1 extends TestOzoneFileSystem {
      * Op 4. create dir -> /d1/d2/d1/d2/key1
      */
     Path dir1Dir1Dir2Key1 = new Path("/d1/d1/d2/key1");
-    try (FSDataOutputStream outputStream = fs.create(dir1Dir1Dir2Key1,
+    try (FSDataOutputStream outputStream = getFs().create(dir1Dir1Dir2Key1,
             false)) {
       assertNotNull("Should be able to create file: " + dir1Dir1Dir2Key1,
               outputStream);
     }
     Path key1 = new Path("/key1");
-    try (FSDataOutputStream outputStream = fs.create(key1, false)) {
+    try (FSDataOutputStream outputStream = getFs().create(key1, false)) {
       assertNotNull("Should be able to create file: " + key1,
               outputStream);
     }
     Path key2 = new Path("/key2");
-    try (FSDataOutputStream outputStream = fs.create(key2, false)) {
+    try (FSDataOutputStream outputStream = getFs().create(key2, false)) {
       assertNotNull("Should be able to create file: key2",
               outputStream);
     }
     Path dir1Dir2Dir1Dir2Key1 = new Path("/d1/d2/d1/d2/key1");
-    try (FSDataOutputStream outputStream = fs.create(dir1Dir2Dir1Dir2Key1,
+    try (FSDataOutputStream outputStream = getFs().create(dir1Dir2Dir1Dir2Key1,
             false)) {
       assertNotNull("Should be able to create file: "
               + dir1Dir2Dir1Dir2Key1, outputStream);
     }
-    RemoteIterator<LocatedFileStatus> fileStatusItr = fs.listFiles(new Path(
-            "/"), true);
-    String uriPrefix = "o3fs://" + bucketName + "." + volumeName;
+    RemoteIterator<LocatedFileStatus> fileStatusItr = getFs().listFiles(
+            new Path("/"), true);
+    String uriPrefix = "o3fs://" + getBucketName() + "." + getVolumeName();
     ArrayList<String> expectedPaths = new ArrayList<>();
     expectedPaths.add(uriPrefix + dir1Dir1Dir2Key1.toString());
     expectedPaths.add(uriPrefix + key1.toString());
@@ -263,7 +260,7 @@ public class TestOzoneFileSystemV1 extends TestOzoneFileSystem {
             expectedPaths.size());
 
     // Recursive=false
-    fileStatusItr = fs.listFiles(new Path("/"), false);
+    fileStatusItr = getFs().listFiles(new Path("/"), false);
     expectedPaths.clear();
     expectedPaths.add(uriPrefix + "/key1");
     expectedPaths.add(uriPrefix + "/key2");
@@ -286,23 +283,23 @@ public class TestOzoneFileSystemV1 extends TestOzoneFileSystem {
   @Test
   public void testRenameWithNonExistentSource() throws Exception {
     // Skip as this will run only in new layout
-    if (!enabledFileSystemPaths) {
+    if (!isEnabledFileSystemPaths()) {
       return;
     }
 
     final String root = "/root";
     final String dir1 = root + "/dir1";
     final String dir2 = root + "/dir2";
-    final Path source = new Path(fs.getUri().toString() + dir1);
-    final Path destin = new Path(fs.getUri().toString() + dir2);
+    final Path source = new Path(getFs().getUri().toString() + dir1);
+    final Path destin = new Path(getFs().getUri().toString() + dir2);
 
     // creates destin
-    fs.mkdirs(destin);
+    getFs().mkdirs(destin);
     LOG.info("Created destin dir: {}", destin);
 
     LOG.info("Rename op-> source:{} to destin:{}}", source, destin);
     try {
-      fs.rename(source, destin);
+      getFs().rename(source, destin);
       Assert.fail("Should throw exception : Source doesn't exist!");
     } catch (OMException ome) {
       // expected
@@ -316,22 +313,22 @@ public class TestOzoneFileSystemV1 extends TestOzoneFileSystem {
   @Test
   public void testRenameDirToItsOwnSubDir() throws Exception {
     // Skip as this will run only in new layout
-    if (!enabledFileSystemPaths) {
+    if (!isEnabledFileSystemPaths()) {
       return;
     }
 
     final String root = "/root";
     final String dir1 = root + "/dir1";
-    final Path dir1Path = new Path(fs.getUri().toString() + dir1);
+    final Path dir1Path = new Path(getFs().getUri().toString() + dir1);
     // Add a sub-dir1 to the directory to be moved.
     final Path subDir1 = new Path(dir1Path, "sub_dir1");
-    fs.mkdirs(subDir1);
+    getFs().mkdirs(subDir1);
     LOG.info("Created dir1 {}", subDir1);
 
-    final Path sourceRoot = new Path(fs.getUri().toString() + root);
+    final Path sourceRoot = new Path(getFs().getUri().toString() + root);
     LOG.info("Rename op-> source:{} to destin:{}", sourceRoot, subDir1);
     try {
-      fs.rename(sourceRoot, subDir1);
+      getFs().rename(sourceRoot, subDir1);
       Assert.fail("Should throw exception : Cannot rename a directory to" +
               " its own subdirectory");
     } catch (OMException ome) {
@@ -348,20 +345,21 @@ public class TestOzoneFileSystemV1 extends TestOzoneFileSystem {
   @Test
   public void testRenameDestinationParentDoesntExist() throws Exception {
     // Skip as this will run only in new layout
-    if (!enabledFileSystemPaths) {
+    if (!isEnabledFileSystemPaths()) {
       return;
     }
 
     final String root = "/root_dir";
     final String dir1 = root + "/dir1";
     final String dir2 = dir1 + "/dir2";
-    final Path dir2SourcePath = new Path(fs.getUri().toString() + dir2);
-    fs.mkdirs(dir2SourcePath);
+    final Path dir2SourcePath = new Path(getFs().getUri().toString() + dir2);
+    getFs().mkdirs(dir2SourcePath);
 
     // (a) parent of dst does not exist.  /root_dir/b/c
-    final Path destinPath = new Path(fs.getUri().toString() + root + "/b/c");
+    final Path destinPath = new Path(getFs().getUri().toString()
+            + root + "/b/c");
     try {
-      fs.rename(dir2SourcePath, destinPath);
+      getFs().rename(dir2SourcePath, destinPath);
       Assert.fail("Should fail as parent of dst does not exist!");
     } catch (OMException ome) {
       // expected
@@ -369,12 +367,12 @@ public class TestOzoneFileSystemV1 extends TestOzoneFileSystem {
     }
 
     // (b) parent of dst is a file. /root_dir/file1/c
-    Path filePath = new Path(fs.getUri().toString() + root + "/file1");
-    ContractTestUtils.touch(fs, filePath);
+    Path filePath = new Path(getFs().getUri().toString() + root + "/file1");
+    ContractTestUtils.touch(getFs(), filePath);
 
     Path newDestinPath = new Path(filePath, "c");
     try {
-      fs.rename(dir2SourcePath, newDestinPath);
+      getFs().rename(dir2SourcePath, newDestinPath);
       Assert.fail("Should fail as parent of dst is a file!");
     } catch (OMException ome) {
       // expected
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestReadRetries.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestReadRetries.java
index d729ad3..a61c798 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestReadRetries.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestReadRetries.java
@@ -87,11 +87,11 @@ public class TestReadRetries {
   @Rule
   public ExpectedException thrown = ExpectedException.none();
 
-  private static MiniOzoneCluster cluster = null;
-  private static OzoneClient ozClient = null;
-  private static ObjectStore store = null;
-  private static OzoneManager ozoneManager;
-  private static StorageContainerLocationProtocolClientSideTranslatorPB
+  private MiniOzoneCluster cluster = null;
+  private OzoneClient ozClient = null;
+  private ObjectStore store = null;
+  private OzoneManager ozoneManager;
+  private StorageContainerLocationProtocolClientSideTranslatorPB
       storageContainerLocationClient;
 
   private static final String SCM_ID = UUID.randomUUID().toString();
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestObjectStoreV1.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestObjectStoreV1.java
index d09020e..b877e29 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestObjectStoreV1.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestObjectStoreV1.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.ozone.om.exceptions.OMException;
 import org.apache.hadoop.ozone.om.helpers.OmDirectoryInfo;
 import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
 import org.apache.hadoop.ozone.om.request.TestOMRequestUtils;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.junit.Assert;
 import org.junit.AfterClass;
@@ -50,8 +51,10 @@ import org.junit.Test;
 import org.junit.rules.Timeout;
 
 import java.io.IOException;
+import java.nio.charset.StandardCharsets;
 import java.util.HashMap;
 import java.util.UUID;
+import java.util.concurrent.TimeoutException;
 
 import static org.apache.hadoop.hdds.client.ReplicationFactor.ONE;
 import static org.apache.hadoop.hdds.client.ReplicationType.STAND_ALONE;
@@ -175,7 +178,8 @@ public class TestObjectStoreV1 {
     verifyKeyInOpenFileTable(openFileTable, clientID, file,
             dirPathC.getObjectID(), false);
 
-    ozoneOutputStream.write(data.getBytes(), 0, data.length());
+    ozoneOutputStream.write(data.getBytes(StandardCharsets.UTF_8), 0,
+            data.length());
     ozoneOutputStream.close();
 
     Table<String, OmKeyInfo> fileTable =
@@ -227,7 +231,8 @@ public class TestObjectStoreV1 {
     verifyKeyInOpenFileTable(openFileTable, clientID, fileName,
             dirPathC.getObjectID(), false);
 
-    ozoneOutputStream.write(data.getBytes(), 0, data.length());
+    ozoneOutputStream.write(data.getBytes(StandardCharsets.UTF_8), 0,
+            data.length());
 
     // open key
     try {
@@ -356,22 +361,21 @@ public class TestObjectStoreV1 {
   }
 
   private void assertKeyRenamedEx(OzoneBucket bucket, String keyName)
-      throws Exception {
-    OMException oe = null;
+          throws Exception {
     try {
       bucket.getKey(keyName);
-    } catch (OMException e) {
-      oe = e;
+      fail("Should throw KeyNotFound as the key got renamed!");
+    } catch (OMException ome) {
+      Assert.assertEquals(KEY_NOT_FOUND, ome.getResult());
     }
-    Assert.assertEquals(KEY_NOT_FOUND, oe.getResult());
   }
 
   private void createTestKey(OzoneBucket bucket, String keyName,
       String keyValue) throws IOException {
     OzoneOutputStream out = bucket.createKey(keyName,
-            keyValue.getBytes().length, STAND_ALONE,
+            keyValue.getBytes(StandardCharsets.UTF_8).length, STAND_ALONE,
             ONE, new HashMap<>());
-    out.write(keyValue.getBytes());
+    out.write(keyValue.getBytes(StandardCharsets.UTF_8));
     out.close();
     OzoneKey key = bucket.getKey(keyName);
     Assert.assertEquals(keyName, key.getName());
@@ -416,13 +420,24 @@ public class TestObjectStoreV1 {
 
   private void verifyKeyInOpenFileTable(Table<String, OmKeyInfo> openFileTable,
       long clientID, String fileName, long parentID, boolean isEmpty)
-          throws IOException {
+          throws IOException, TimeoutException, InterruptedException {
     String dbOpenFileKey =
             parentID + OM_KEY_PREFIX + fileName + OM_KEY_PREFIX + clientID;
-    OmKeyInfo omKeyInfo = openFileTable.get(dbOpenFileKey);
+
     if (isEmpty) {
-      Assert.assertNull("Table is not empty!", omKeyInfo);
+      // wait for DB updates
+      GenericTestUtils.waitFor(() -> {
+        try {
+          OmKeyInfo omKeyInfo = openFileTable.get(dbOpenFileKey);
+          return omKeyInfo == null;
+        } catch (IOException e) {
+          Assert.fail("DB failure!");
+          return false;
+        }
+
+      }, 1000, 120000);
     } else {
+      OmKeyInfo omKeyInfo = openFileTable.get(dbOpenFileKey);
       Assert.assertNotNull("Table is empty!", omKeyInfo);
       // used startsWith because the key format is,
       // <parentID>/fileName/<clientID> and clientID is not visible.
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/file/TestOMDirectoryCreateRequestV1.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/file/TestOMDirectoryCreateRequestV1.java
index 454cfbb..d8c8108 100644
--- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/file/TestOMDirectoryCreateRequestV1.java
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/file/TestOMDirectoryCreateRequestV1.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.utils.db.cache.CacheKey;
 import org.apache.hadoop.hdds.utils.db.cache.CacheValue;
+import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.audit.AuditLogger;
 import org.apache.hadoop.ozone.audit.AuditMessage;
 import org.apache.hadoop.ozone.om.OMConfigKeys;
@@ -369,8 +370,9 @@ public class TestOMDirectoryCreateRequestV1 {
             bucketName, keyName, HddsProtos.ReplicationType.RATIS,
             HddsProtos.ReplicationFactor.THREE, objID++);
     String ozoneFileName = parentID + "/" + dirs.get(dirs.size() - 1);
+    ++txnID;
     omMetadataManager.getKeyTable().addCacheEntry(new CacheKey<>(ozoneFileName),
-            new CacheValue<>(Optional.of(omKeyInfo), ++txnID));
+            new CacheValue<>(Optional.of(omKeyInfo), txnID));
     omMetadataManager.getKeyTable().put(ozoneFileName, omKeyInfo);
 
     OMRequest omRequest = createDirectoryRequest(volumeName, bucketName,
@@ -435,10 +437,12 @@ public class TestOMDirectoryCreateRequestV1 {
     // Add a key in second level.
     OmKeyInfo omKeyInfo = TestOMRequestUtils.createOmKeyInfo(volumeName,
             bucketName, keyName, HddsProtos.ReplicationType.RATIS,
-            HddsProtos.ReplicationFactor.THREE, objID++);
+            HddsProtos.ReplicationFactor.THREE, objID);
+
     String ozoneKey = parentID + "/" + dirs.get(1);
+    ++txnID;
     omMetadataManager.getKeyTable().addCacheEntry(new CacheKey<>(ozoneKey),
-            new CacheValue<>(Optional.of(omKeyInfo), ++txnID));
+            new CacheValue<>(Optional.of(omKeyInfo), txnID));
     omMetadataManager.getKeyTable().put(ozoneKey, omKeyInfo);
 
     OMRequest omRequest = createDirectoryRequest(volumeName, bucketName,
@@ -517,10 +521,6 @@ public class TestOMDirectoryCreateRequestV1 {
     // Add volume and bucket entries to DB.
     TestOMRequestUtils.addVolumeAndBucketToDB(volumeName, bucketName,
             omMetadataManager);
-    String bucketKey = omMetadataManager.getBucketKey(volumeName, bucketName);
-    OmBucketInfo omBucketInfo =
-            omMetadataManager.getBucketTable().get(bucketKey);
-    long bucketID = omBucketInfo.getObjectID();
 
     OMRequest omRequest = createDirectoryRequest(volumeName, bucketName,
             OzoneFSUtils.addTrailingSlashIfNeeded(keyName));
@@ -587,12 +587,14 @@ public class TestOMDirectoryCreateRequestV1 {
   private String createDirKey(List<String> dirs, int depth) {
     String keyName = RandomStringUtils.randomAlphabetic(5);
     dirs.add(keyName);
+    StringBuffer buf = new StringBuffer(keyName);
     for (int i = 0; i < depth; i++) {
       String dirName = RandomStringUtils.randomAlphabetic(5);
       dirs.add(dirName);
-      keyName += "/" + dirName;
+      buf.append(OzoneConsts.OM_KEY_PREFIX);
+      buf.append(dirName);
     }
-    return keyName;
+    return buf.toString();
   }
 
   private void verifyDirectoriesInDB(List<String> dirs, long bucketID)
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyCreateRequestV1.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyCreateRequestV1.java
index b65443d..e545dc7 100644
--- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyCreateRequestV1.java
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/key/TestOMKeyCreateRequestV1.java
@@ -75,7 +75,9 @@ public class TestOMKeyCreateRequestV1 extends TestOMKeyCreateRequest {
     long parentID = checkIntermediatePaths(keyPath);
 
     // Check open key entry
-    String fileName = keyPath.getFileName().toString();
+    Path keyPathFileName = keyPath.getFileName();
+    Assert.assertNotNull("Failed to find fileName", keyPathFileName);
+    String fileName = keyPathFileName.toString();
     String openKey = omMetadataManager.getOpenFileName(parentID, fileName,
             omRequest.getCreateKeyRequest().getClientID());
     OmKeyInfo omKeyInfo = omMetadataManager.getOpenKeyTable().get(openKey);
@@ -88,6 +90,7 @@ public class TestOMKeyCreateRequestV1 extends TestOMKeyCreateRequest {
     String bucketKey = omMetadataManager.getBucketKey(volumeName, bucketName);
     OmBucketInfo omBucketInfo =
             omMetadataManager.getBucketTable().get(bucketKey);
+    Assert.assertNotNull("Bucket not found!", omBucketInfo);
     long lastKnownParentId = omBucketInfo.getObjectID();
 
     Iterator<Path> elements = keyPath.iterator();
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/s3/multipart/TestS3InitiateMultipartUploadRequestV1.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/s3/multipart/TestS3InitiateMultipartUploadRequestV1.java
index dac2efe..5fa75ba 100644
--- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/s3/multipart/TestS3InitiateMultipartUploadRequestV1.java
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/s3/multipart/TestS3InitiateMultipartUploadRequestV1.java
@@ -133,10 +133,6 @@ public class TestS3InitiateMultipartUploadRequestV1
     Assert.assertEquals(OzoneManagerProtocolProtos.Status.BUCKET_NOT_FOUND,
         omClientResponse.getOMResponse().getStatus());
 
-    String multipartKey = omMetadataManager.getMultipartKey(volumeName,
-        bucketName, keyName, modifiedRequest.getInitiateMultiPartUploadRequest()
-            .getKeyArgs().getMultipartUploadID());
-
     Assert.assertTrue(omMetadataManager.getOpenKeyTable().isEmpty());
     Assert.assertTrue(omMetadataManager.getMultipartInfoTable().isEmpty());
   }
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/file/TestOMDirectoryCreateResponseV1.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/file/TestOMDirectoryCreateResponseV1.java
index 0a1114a..fb06581 100644
--- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/file/TestOMDirectoryCreateResponseV1.java
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/file/TestOMDirectoryCreateResponseV1.java
@@ -59,9 +59,7 @@ public class TestOMDirectoryCreateResponseV1 {
   @Test
   public void testAddToDBBatch() throws Exception {
 
-    String volumeName = UUID.randomUUID().toString();
     String keyName = UUID.randomUUID().toString();
-    String bucketName = UUID.randomUUID().toString();
 
     long parentID = 100;
     OmDirectoryInfo omDirInfo =
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/file/TestOMFileCreateResponseV1.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/file/TestOMFileCreateResponseV1.java
index e1549e1..a8a0c99 100644
--- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/file/TestOMFileCreateResponseV1.java
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/file/TestOMFileCreateResponseV1.java
@@ -30,6 +30,8 @@ import org.apache.hadoop.util.Time;
 import org.jetbrains.annotations.NotNull;
 import org.junit.Assert;
 
+import java.util.ArrayList;
+
 /**
  * Tests MKeyCreateResponse layout version V1.
  */
@@ -59,8 +61,8 @@ public class TestOMFileCreateResponseV1 extends TestOMKeyCreateResponse {
   protected OMKeyCreateResponse getOmKeyCreateResponse(OmKeyInfo keyInfo,
       OmBucketInfo bucketInfo, OMResponse response) {
 
-    return new OMFileCreateResponseV1(response, keyInfo, null, clientID,
-            bucketInfo);
+    return new OMFileCreateResponseV1(response, keyInfo, new ArrayList<>(),
+            clientID, bucketInfo);
   }
 
   @NotNull
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/key/TestOMKeyCommitResponse.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/key/TestOMKeyCommitResponse.java
index 4d50337..e2a223c 100644
--- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/key/TestOMKeyCommitResponse.java
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/key/TestOMKeyCommitResponse.java
@@ -34,8 +34,6 @@ import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
 @SuppressWarnings("visibilitymodifier")
 public class TestOMKeyCommitResponse extends TestOMKeyResponse {
 
-  protected OmBucketInfo omBucketInfo;
-
   @Test
   public void testAddToDBBatch() throws Exception {
 
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/key/TestOMKeyCreateResponseV1.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/key/TestOMKeyCreateResponseV1.java
index 6299639..834aafa 100644
--- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/key/TestOMKeyCreateResponseV1.java
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/key/TestOMKeyCreateResponseV1.java
@@ -28,6 +28,8 @@ import org.apache.hadoop.util.Time;
 import org.jetbrains.annotations.NotNull;
 import org.junit.Assert;
 
+import java.util.ArrayList;
+
 /**
  * Tests OMKeyCreateResponseV1.
  */
@@ -67,7 +69,7 @@ public class TestOMKeyCreateResponseV1 extends TestOMKeyCreateResponse {
   protected OMKeyCreateResponse getOmKeyCreateResponse(OmKeyInfo keyInfo,
       OmBucketInfo bucketInfo, OMResponse response) {
 
-    return new OMKeyCreateResponseV1(response, keyInfo, null, clientID,
-            bucketInfo);
+    return new OMKeyCreateResponseV1(response, keyInfo,  new ArrayList<>(),
+            clientID, bucketInfo);
   }
 }
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/s3/multipart/TestS3InitiateMultipartUploadResponseV1.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/s3/multipart/TestS3InitiateMultipartUploadResponseV1.java
index 31f9e5a..6dd6aaf 100644
--- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/s3/multipart/TestS3InitiateMultipartUploadResponseV1.java
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/response/s3/multipart/TestS3InitiateMultipartUploadResponseV1.java
@@ -40,11 +40,6 @@ public class TestS3InitiateMultipartUploadResponseV1
     String volumeName = UUID.randomUUID().toString();
     String bucketName = UUID.randomUUID().toString();
     String prefix = "a/b/c/d/";
-    List<String> dirs = new ArrayList<String>();
-    dirs.add("a");
-    dirs.add("b");
-    dirs.add("c");
-    dirs.add("d");
     String fileName = UUID.randomUUID().toString();
     String keyName = prefix + fileName;
 

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