Browse Source

HADOOP-18671. Add recoverLease(), setSafeMode(), isFileClosed() as interfaces to hadoop-common (#5553)

The HDFS lease APIs have been replicated as interfaces in hadoop-common so other filesystems can
also implement them.  Applications which use the leasing APIs should migrate to the new
interface where possible.

Contributed by Stephen Wu
Tak Lon (Stephen) Wu 2 years ago
parent
commit
0e46388474
68 changed files with 967 additions and 298 deletions
  1. 6 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonPathCapabilities.java
  2. 46 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LeaseRecoverable.java
  3. 50 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/SafeMode.java
  4. 41 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/SafeModeAction.java
  5. 3 1
      hadoop-common-project/hadoop-common/src/site/markdown/filesystem/index.md
  6. 52 0
      hadoop-common-project/hadoop-common/src/site/markdown/filesystem/leaserecoverable.md
  7. 45 0
      hadoop-common-project/hadoop-common/src/site/markdown/filesystem/safemode.md
  8. 90 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractLeaseRecoveryTest.java
  9. 54 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractSafeModeTest.java
  10. 75 7
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
  11. 2 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ViewDistributedFileSystem.java
  12. 1 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsUtils.java
  13. 14 14
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpc.java
  14. 2 2
      hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpcSingleNS.java
  15. 3 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java
  16. 49 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/contract/hdfs/TestHDFSContractLeaseRecovery.java
  17. 49 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/contract/hdfs/TestHDFSContractSafeMode.java
  18. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java
  19. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
  20. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgrade.java
  21. 8 8
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java
  22. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java
  23. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicyWithSnapshot.java
  24. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestGetBlocks.java
  25. 18 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHDFSFileSystemContract.java
  26. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestMiniDFSCluster.java
  27. 14 14
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRollingUpgrade.java
  28. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRollingUpgradeDowngrade.java
  29. 5 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRollingUpgradeRollback.java
  30. 14 14
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java
  31. 99 11
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestViewDistributedFileSystem.java
  32. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
  33. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeRollingUpgrade.java
  34. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyPersistPolicy.java
  35. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
  36. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCacheDirectives.java
  37. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckPointForSecurityTokens.java
  38. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java
  39. 25 26
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
  40. 5 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImageWithAcl.java
  41. 23 23
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImageWithSnapshot.java
  42. 4 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImageWithXAttr.java
  43. 6 6
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
  44. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestLeaseManager.java
  45. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeRetryCacheMetrics.java
  46. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNestedEncryptionZones.java
  47. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaByStorageType.java
  48. 9 9
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestReencryption.java
  49. 18 17
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSaveNamespace.java
  50. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHAMetrics.java
  51. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestNNHealthCheck.java
  52. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestObserverNode.java
  53. 17 17
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestFSImageWithOrderedSnapshotDeletion.java
  54. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestOrderedSnapshotDeletion.java
  55. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRandomOpsWithSnapshots.java
  56. 7 7
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java
  57. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshot.java
  58. 13 13
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java
  59. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestXAttrWithSnapshot.java
  60. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java
  61. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerForAcl.java
  62. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerForContentSummary.java
  63. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerForErasureCodingPolicy.java
  64. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerForStoragePolicy.java
  65. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerForXAttr.java
  66. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerWithStripedBlocks.java
  67. 10 10
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/TestHistoryFileManager.java
  68. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/TestHistoryFileManagerInitWithNonRunningDFS.java

+ 6 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonPathCapabilities.java

@@ -163,5 +163,11 @@ public final class CommonPathCapabilities {
   public static final String ETAGS_PRESERVED_IN_RENAME =
   public static final String ETAGS_PRESERVED_IN_RENAME =
       "fs.capability.etags.preserved.in.rename";
       "fs.capability.etags.preserved.in.rename";
 
 
+  /**
+   * Does this Filesystem support lease recovery operations such as
+   * {@link LeaseRecoverable#recoverLease(Path)} and {@link LeaseRecoverable#isFileClosed(Path)}}?
+   * Value: {@value}.
+   */
+  public static final String LEASE_RECOVERABLE = "fs.capability.lease.recoverable";
 
 
 }
 }

+ 46 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LeaseRecoverable.java

@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs;
+
+import java.io.IOException;
+
+/**
+ * Whether the given Path of the FileSystem has the capability to perform lease recovery.
+ */
+public interface LeaseRecoverable {
+
+  /**
+   * Start the lease recovery of a file.
+   *
+   * @param file path to a file.
+   * @return true if the file is already closed, and it does not require lease recovery.
+   * @throws IOException if an error occurs during lease recovery.
+   * @throws UnsupportedOperationException if lease recovery is not supported by this filesystem.
+   */
+  boolean recoverLease(Path file) throws IOException;
+
+  /**
+   * Get the close status of a file.
+   * @param file The string representation of the path to the file
+   * @return return true if file is closed
+   * @throws IOException If an I/O error occurred
+   * @throws UnsupportedOperationException if isFileClosed is not supported by this filesystem.
+   */
+  boolean isFileClosed(Path file) throws IOException;
+}

+ 50 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/SafeMode.java

@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs;
+
+import java.io.IOException;
+
+/**
+ * Whether the given filesystem is in any status of safe mode.
+ */
+public interface SafeMode {
+
+  /**
+   * Enter, leave, or get safe mode.
+   *
+   * @param action One of {@link SafeModeAction} LEAVE, ENTER, GET, FORCE_EXIT.
+   * @throws IOException if set safe mode fails to proceed.
+   * @return true if the action is successfully accepted, otherwise false means rejected.
+   */
+  default boolean setSafeMode(SafeModeAction action) throws IOException {
+    return setSafeMode(action, false);
+  }
+
+  /**
+   * Enter, leave, or get safe mode.
+   *
+   * @param action    One of {@link SafeModeAction} LEAVE, ENTER, GET, FORCE_EXIT.
+   * @param isChecked If true check only for Active metadata node / NameNode's status,
+   *                  else check first metadata node / NameNode's status.
+   * @throws IOException if set safe mode fails to proceed.
+   * @return true if the action is successfully accepted, otherwise false means rejected.
+   */
+  boolean setSafeMode(SafeModeAction action, boolean isChecked) throws IOException;
+
+}

+ 41 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/SafeModeAction.java

@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs;
+
+/**
+ * An identical copy from org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction, that helps
+ * the other file system implementation to define {@link SafeMode}.
+ */
+public enum SafeModeAction {
+  /**
+   * Starting entering into safe mode.
+   */
+  ENTER,
+  /**
+   * Gracefully exit from safe mode.
+   */
+  LEAVE,
+  /**
+   * Force Exit from safe mode.
+   */
+  FORCE_EXIT,
+  /**
+   * Get the status of the safe mode.
+   */
+  GET;
+}

+ 3 - 1
hadoop-common-project/hadoop-common/src/site/markdown/filesystem/index.md

@@ -41,4 +41,6 @@ HDFS as these are commonly expected by Hadoop client applications.
 2. [Extending the specification and its tests](extending.html)
 2. [Extending the specification and its tests](extending.html)
 1. [Uploading a file using Multiple Parts](multipartuploader.html)
 1. [Uploading a file using Multiple Parts](multipartuploader.html)
 1. [IOStatistics](iostatistics.html)
 1. [IOStatistics](iostatistics.html)
-1. [openFile()](openfile.html).
+1. [openFile()](openfile.html)
+1. [SafeMode](safemode.html)
+1. [LeaseRecoverable](leaserecoverable.html)

+ 52 - 0
hadoop-common-project/hadoop-common/src/site/markdown/filesystem/leaserecoverable.md

@@ -0,0 +1,52 @@
+<!---
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+   http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License. See accompanying LICENSE file.
+-->
+
+# <a name="LeaseRecoverable"></a> interface `LeaseRecoverable`
+
+The `LeaseRecoverable` interface tells whether a given path of current filesystem can perform lease
+recovery for open file that a lease is not explicitly renewed or the client holding it goes away.
+
+This interface should be implemented accordingly when necessary to any Filesystem that supports
+lease recovery, e.g. `DistributedFileSystem` (HDFS) and `ViewDistributedFileSystem`.
+
+```java
+public interface LeaseRecoverable {
+  boolean recoverLease(Path file) throws IOException;
+  boolean isFileClosed(Path file) throws IOException;
+}
+```
+
+There are two main functions of this interface, one performs lease recovery and another one
+verifies if a file has been closed.
+
+### boolean recoverLease(Path file)
+
+This function performs the lease recovery for the given file path, and it does not support
+directory path recovery.
+1. Return `true`, if the file has already closed, or does not require lease recovery.
+1. Return `false`, if the lease recovery is yet completed.
+1. Throw `IOException` if a directory path is given as input.
+
+### boolean isFileClosed(Path file)
+
+This function only checks if the give file path has been closed, and it does not support directory
+verification.
+1. Return `true`, if the file has been closed.
+1. Return `false`, if the file is still open.
+1. Throw `IOException` if a directory path is given as input.
+
+### Path Capabilities SHOULD BE declared
+
+If a filesystem supports `LeaseRecoverable`, it should return `true` to
+`PathCapabilities.hasPathCapability(path, "fs.capability.lease.recoverable")` for a given path.

+ 45 - 0
hadoop-common-project/hadoop-common/src/site/markdown/filesystem/safemode.md

@@ -0,0 +1,45 @@
+<!---
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+   http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License. See accompanying LICENSE file.
+-->
+
+# <a name="SafeMode"></a> interface `SafeMode`
+
+The `SafeMode` interface provides a way to perform safe mode actions and obtain the
+status after such actions performed to the `FileSystem`.
+
+This is admin only interface, should be implemented accordingly when necessary to
+Filesystem that support safe mode, e.g. `DistributedFileSystem` (HDFS) and
+`ViewDistributedFileSystem`.
+
+```java
+public interface SafeMode {
+  default boolean setSafeMode(SafeModeAction action) throws IOException {
+    return setSafeMode(action, false);
+  }
+  boolean setSafeMode(SafeModeAction action, boolean isChecked) throws IOException;
+}
+```
+
+The goals of this interface is allow any file system implementation to share the
+same concept of safe mode with the following actions and states
+
+### Safe mode actions
+1. `GET`, get the safe mode status of the file system.
+1. `ENTER`, enter the safe mode for the file system.
+1. `LEAVE`, exit safe mode for the file system gracefully.
+1. `FORCE_EXIT`, exit safe mode for the file system even if there is any ongoing data process.
+
+### Safe mode states
+1. return true, when safe mode is on.
+1. return false, when safe mode is off, usually it's the result of safe mode actions
+with `GET`, `LEAVE`, `FORCE_EXIT`.

+ 90 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractLeaseRecoveryTest.java

@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.contract;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LeaseRecoverable;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.test.LambdaTestUtils;
+
+import static org.apache.hadoop.fs.CommonPathCapabilities.LEASE_RECOVERABLE;
+
+public abstract class AbstractContractLeaseRecoveryTest extends
+    AbstractFSContractTestBase {
+
+  @Test
+  public void testLeaseRecovery() throws Throwable {
+    final Path path = methodPath();
+    final FileSystem fs = getFileSystem();
+    ContractTestUtils.touch(fs, path);
+    LeaseRecoverable leaseRecoverableFs = verifyAndGetLeaseRecoverableInstance(fs, path);
+
+    Assertions.assertThat(leaseRecoverableFs.recoverLease(path))
+        .describedAs("Issuing lease recovery on a closed file must be successful")
+        .isTrue();
+
+    Assertions.assertThat(leaseRecoverableFs.isFileClosed(path))
+        .describedAs("Get the isFileClose status on a closed file must be successful")
+        .isTrue();
+  }
+
+  @Test
+  public void testLeaseRecoveryFileNotExist() throws Throwable {
+    final Path path = new Path("notExist");
+    final FileSystem fs = getFileSystem();
+    LeaseRecoverable leaseRecoverableFs = verifyAndGetLeaseRecoverableInstance(fs, path);
+
+    LambdaTestUtils.intercept(FileNotFoundException.class, "File does not exist",
+        () -> leaseRecoverableFs.recoverLease(path));
+
+    LambdaTestUtils.intercept(FileNotFoundException.class, "File does not exist",
+        () -> leaseRecoverableFs.isFileClosed(path));
+  }
+
+  @Test
+  public void testLeaseRecoveryFileOnDirectory() throws Throwable {
+    final Path path = methodPath();
+    final FileSystem fs = getFileSystem();
+    LeaseRecoverable leaseRecoverableFs = verifyAndGetLeaseRecoverableInstance(fs, path);
+    final Path parentDirectory = path.getParent();
+
+    LambdaTestUtils.intercept(FileNotFoundException.class, "Path is not a file",
+        () -> leaseRecoverableFs.recoverLease(parentDirectory));
+
+    LambdaTestUtils.intercept(FileNotFoundException.class, "Path is not a file",
+        () -> leaseRecoverableFs.isFileClosed(parentDirectory));
+  }
+
+  private LeaseRecoverable verifyAndGetLeaseRecoverableInstance(FileSystem fs, Path path)
+      throws IOException {
+    Assertions.assertThat(fs.hasPathCapability(path, LEASE_RECOVERABLE))
+        .describedAs("path capability %s of %s", LEASE_RECOVERABLE, path)
+        .isTrue();
+    Assertions.assertThat(fs)
+        .describedAs("filesystem %s", fs)
+        .isInstanceOf(LeaseRecoverable.class);
+    return (LeaseRecoverable) fs;
+  }
+}

+ 54 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractSafeModeTest.java

@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.contract;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.SafeMode;
+import org.apache.hadoop.fs.SafeModeAction;
+
+public abstract class AbstractContractSafeModeTest extends AbstractFSContractTestBase {
+
+  @Test
+  public void testSafeMode() throws Throwable {
+    final FileSystem fs = getFileSystem();
+    SafeMode fsWithSafeMode = verifyAndGetSafeModeInstance(fs);
+    Assertions.assertThat(fsWithSafeMode.setSafeMode(SafeModeAction.GET))
+        .describedAs("Getting the status of safe mode before entering should be off.").isFalse();
+    Assertions.assertThat(fsWithSafeMode.setSafeMode(SafeModeAction.ENTER))
+        .describedAs("Entering Safe mode and safe mode turns on.").isTrue();
+    Assertions.assertThat(fsWithSafeMode.setSafeMode(SafeModeAction.GET))
+        .describedAs("Getting the status of safe mode after entering, safe mode should be on.")
+        .isTrue();
+    Assertions.assertThat(fsWithSafeMode.setSafeMode(SafeModeAction.LEAVE))
+        .describedAs("Leaving safe mode, and safe mode switches off.").isFalse();
+    Assertions.assertThat(fsWithSafeMode.setSafeMode(SafeModeAction.FORCE_EXIT))
+        .describedAs("Force exist safe mode at any time, safe mode should always switches off.")
+        .isFalse();
+  }
+
+  private SafeMode verifyAndGetSafeModeInstance(FileSystem fs) {
+    Assertions.assertThat(fs)
+        .describedAs("File system %s must be an instance of %s", fs, SafeMode.class.getClass())
+        .isInstanceOf(SafeMode.class);
+    return (SafeMode) fs;
+  }
+}

+ 75 - 7
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java

@@ -18,6 +18,8 @@
 
 
 package org.apache.hadoop.hdfs;
 package org.apache.hadoop.hdfs;
 
 
+import org.apache.hadoop.fs.LeaseRecoverable;
+import org.apache.hadoop.fs.SafeMode;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.classification.VisibleForTesting;
 import org.apache.hadoop.classification.VisibleForTesting;
@@ -61,6 +63,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.fs.QuotaUsage;
 import org.apache.hadoop.fs.QuotaUsage;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.SafeModeAction;
 import org.apache.hadoop.fs.StorageStatistics;
 import org.apache.hadoop.fs.StorageStatistics;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.UnresolvedLinkException;
@@ -95,7 +98,6 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.ReencryptAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.ReencryptAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsPathHandle;
 import org.apache.hadoop.hdfs.protocol.HdfsPathHandle;
 import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
@@ -144,7 +146,7 @@ import static org.apache.hadoop.fs.impl.PathCapabilitiesSupport.validatePathCapa
 @InterfaceAudience.LimitedPrivate({ "MapReduce", "HBase" })
 @InterfaceAudience.LimitedPrivate({ "MapReduce", "HBase" })
 @InterfaceStability.Unstable
 @InterfaceStability.Unstable
 public class DistributedFileSystem extends FileSystem
 public class DistributedFileSystem extends FileSystem
-    implements KeyProviderTokenIssuer, BatchListingOperations {
+    implements KeyProviderTokenIssuer, BatchListingOperations, LeaseRecoverable, SafeMode {
   private Path workingDir;
   private Path workingDir;
   private URI uri;
   private URI uri;
 
 
@@ -306,6 +308,7 @@ public class DistributedFileSystem extends FileSystem
    * @return true if the file is already closed
    * @return true if the file is already closed
    * @throws IOException if an error occurs
    * @throws IOException if an error occurs
    */
    */
+  @Override
   public boolean recoverLease(final Path f) throws IOException {
   public boolean recoverLease(final Path f) throws IOException {
     Path absF = fixRelativePart(f);
     Path absF = fixRelativePart(f);
     return new FileSystemLinkResolver<Boolean>() {
     return new FileSystemLinkResolver<Boolean>() {
@@ -1633,6 +1636,63 @@ public class DistributedFileSystem extends FileSystem
    * @see org.apache.hadoop.hdfs.protocol.ClientProtocol#setSafeMode(
    * @see org.apache.hadoop.hdfs.protocol.ClientProtocol#setSafeMode(
    *    HdfsConstants.SafeModeAction,boolean)
    *    HdfsConstants.SafeModeAction,boolean)
    */
    */
+  @Override
+  public boolean setSafeMode(SafeModeAction action)
+      throws IOException {
+    return setSafeMode(action, false);
+  }
+
+  /**
+   * Enter, leave or get safe mode.
+   *
+   * @param action
+   *          One of SafeModeAction.ENTER, SafeModeAction.LEAVE and
+   *          SafeModeAction.GET.
+   * @param isChecked
+   *          If true check only for Active NNs status, else check first NN's
+   *          status.
+   */
+  @Override
+  @SuppressWarnings("deprecation")
+  public boolean setSafeMode(SafeModeAction action, boolean isChecked)
+      throws IOException {
+    return this.setSafeMode(convertToClientProtocolSafeModeAction(action), isChecked);
+  }
+
+  /**
+   * Translating the {@link SafeModeAction} into {@link HdfsConstants.SafeModeAction}
+   * that is used by {@link DFSClient#setSafeMode(HdfsConstants.SafeModeAction, boolean)}.
+   *
+   * @param action any supported action listed in {@link SafeModeAction}.
+   * @return the converted {@link HdfsConstants.SafeModeAction}.
+   * @throws UnsupportedOperationException if the provided {@link SafeModeAction} cannot be
+   *           translated.
+   */
+  private static HdfsConstants.SafeModeAction convertToClientProtocolSafeModeAction(
+      SafeModeAction action) {
+    switch (action) {
+    case ENTER:
+      return HdfsConstants.SafeModeAction.SAFEMODE_ENTER;
+    case LEAVE:
+      return HdfsConstants.SafeModeAction.SAFEMODE_LEAVE;
+    case FORCE_EXIT:
+      return HdfsConstants.SafeModeAction.SAFEMODE_FORCE_EXIT;
+    case GET:
+      return HdfsConstants.SafeModeAction.SAFEMODE_GET;
+    default:
+      throw new UnsupportedOperationException("Unsupported safe mode action " + action);
+    }
+  }
+
+  /**
+   * Enter, leave or get safe mode.
+   *
+   * @see org.apache.hadoop.hdfs.protocol.ClientProtocol#setSafeMode(HdfsConstants.SafeModeAction,
+   * boolean)
+   *
+   * @deprecated please instead use {@link #setSafeMode(SafeModeAction)}.
+   */
+  @Deprecated
   public boolean setSafeMode(HdfsConstants.SafeModeAction action)
   public boolean setSafeMode(HdfsConstants.SafeModeAction action)
       throws IOException {
       throws IOException {
     return setSafeMode(action, false);
     return setSafeMode(action, false);
@@ -1643,12 +1703,18 @@ public class DistributedFileSystem extends FileSystem
    *
    *
    * @param action
    * @param action
    *          One of SafeModeAction.ENTER, SafeModeAction.LEAVE and
    *          One of SafeModeAction.ENTER, SafeModeAction.LEAVE and
-   *          SafeModeAction.GET
+   *          SafeModeAction.GET.
    * @param isChecked
    * @param isChecked
    *          If true check only for Active NNs status, else check first NN's
    *          If true check only for Active NNs status, else check first NN's
-   *          status
-   * @see org.apache.hadoop.hdfs.protocol.ClientProtocol#setSafeMode(SafeModeAction, boolean)
+   *          status.
+   *
+   * @see org.apache.hadoop.hdfs.protocol.ClientProtocol#setSafeMode(HdfsConstants.SafeModeAction,
+   * boolean)
+   *
+   * @deprecated please instead use
+   *               {@link DistributedFileSystem#setSafeMode(SafeModeAction, boolean)}.
    */
    */
+  @Deprecated
   public boolean setSafeMode(HdfsConstants.SafeModeAction action,
   public boolean setSafeMode(HdfsConstants.SafeModeAction action,
       boolean isChecked) throws IOException {
       boolean isChecked) throws IOException {
     return dfs.setSafeMode(action, isChecked);
     return dfs.setSafeMode(action, isChecked);
@@ -1685,7 +1751,7 @@ public class DistributedFileSystem extends FileSystem
   }
   }
 
 
   /**
   /**
-   * enable/disable/check restoreFaileStorage
+   * enable/disable/check restoreFaileStorage.
    *
    *
    * @see org.apache.hadoop.hdfs.protocol.ClientProtocol#restoreFailedStorage(String arg)
    * @see org.apache.hadoop.hdfs.protocol.ClientProtocol#restoreFailedStorage(String arg)
    */
    */
@@ -2048,7 +2114,7 @@ public class DistributedFileSystem extends FileSystem
    *           when there is an issue communicating with the NameNode
    *           when there is an issue communicating with the NameNode
    */
    */
   public boolean isInSafeMode() throws IOException {
   public boolean isInSafeMode() throws IOException {
-    return setSafeMode(SafeModeAction.SAFEMODE_GET, true);
+    return setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_GET, true);
   }
   }
 
 
   /**
   /**
@@ -2464,6 +2530,7 @@ public class DistributedFileSystem extends FileSystem
    * @throws FileNotFoundException if the file does not exist.
    * @throws FileNotFoundException if the file does not exist.
    * @throws IOException If an I/O error occurred
    * @throws IOException If an I/O error occurred
    */
    */
+  @Override
   public boolean isFileClosed(final Path src) throws IOException {
   public boolean isFileClosed(final Path src) throws IOException {
     Path absF = fixRelativePart(src);
     Path absF = fixRelativePart(src);
     return new FileSystemLinkResolver<Boolean>() {
     return new FileSystemLinkResolver<Boolean>() {
@@ -3885,6 +3952,7 @@ public class DistributedFileSystem extends FileSystem
     // (yet/ever) in the WebHDFS API.
     // (yet/ever) in the WebHDFS API.
     switch (validatePathCapabilityArgs(path, capability)) {
     switch (validatePathCapabilityArgs(path, capability)) {
     case CommonPathCapabilities.FS_EXPERIMENTAL_BATCH_LISTING:
     case CommonPathCapabilities.FS_EXPERIMENTAL_BATCH_LISTING:
+    case CommonPathCapabilities.LEASE_RECOVERABLE:
       return true;
       return true;
     default:
     default:
       // fall through
       // fall through

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ViewDistributedFileSystem.java

@@ -801,6 +801,7 @@ public class ViewDistributedFileSystem extends DistributedFileSystem {
   }
   }
 
 
   @Override
   @Override
+  @SuppressWarnings("deprecation")
   public boolean setSafeMode(HdfsConstants.SafeModeAction action)
   public boolean setSafeMode(HdfsConstants.SafeModeAction action)
       throws IOException {
       throws IOException {
     if (this.vfs == null) {
     if (this.vfs == null) {
@@ -811,6 +812,7 @@ public class ViewDistributedFileSystem extends DistributedFileSystem {
   }
   }
 
 
   @Override
   @Override
+  @SuppressWarnings("deprecation")
   public boolean setSafeMode(HdfsConstants.SafeModeAction action,
   public boolean setSafeMode(HdfsConstants.SafeModeAction action,
       boolean isChecked) throws IOException {
       boolean isChecked) throws IOException {
     if (this.vfs == null) {
     if (this.vfs == null) {

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsUtils.java

@@ -46,6 +46,7 @@ public class HdfsUtils {
    * @param uri the HDFS URI.  Note that the URI path is ignored.
    * @param uri the HDFS URI.  Note that the URI path is ignored.
    * @return true if HDFS is healthy; false, otherwise.
    * @return true if HDFS is healthy; false, otherwise.
    */
    */
+  @SuppressWarnings("deprecation")
   public static boolean isHealthy(URI uri) {
   public static boolean isHealthy(URI uri) {
     //check scheme
     //check scheme
     final String scheme = uri.getScheme();
     final String scheme = uri.getScheme();

+ 14 - 14
hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpc.java

@@ -66,6 +66,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.SafeModeAction;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSClient;
@@ -99,7 +100,6 @@ import org.apache.hadoop.hdfs.protocol.SnapshotException;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.SnapshotStatus;
 import org.apache.hadoop.hdfs.protocol.SnapshotStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
@@ -1432,27 +1432,27 @@ public class TestRouterRpc {
   @Test
   @Test
   public void testProxySetSafemode() throws Exception {
   public void testProxySetSafemode() throws Exception {
     boolean routerSafemode =
     boolean routerSafemode =
-        routerProtocol.setSafeMode(SafeModeAction.SAFEMODE_GET, false);
+        routerProtocol.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_GET, false);
     boolean nnSafemode =
     boolean nnSafemode =
-        nnProtocol.setSafeMode(SafeModeAction.SAFEMODE_GET, false);
+        nnProtocol.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_GET, false);
     assertEquals(nnSafemode, routerSafemode);
     assertEquals(nnSafemode, routerSafemode);
 
 
     routerSafemode =
     routerSafemode =
-        routerProtocol.setSafeMode(SafeModeAction.SAFEMODE_GET, true);
+        routerProtocol.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_GET, true);
     nnSafemode =
     nnSafemode =
-        nnProtocol.setSafeMode(SafeModeAction.SAFEMODE_GET, true);
+        nnProtocol.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_GET, true);
     assertEquals(nnSafemode, routerSafemode);
     assertEquals(nnSafemode, routerSafemode);
 
 
     assertFalse(routerProtocol.setSafeMode(
     assertFalse(routerProtocol.setSafeMode(
-        SafeModeAction.SAFEMODE_GET, false));
+        HdfsConstants.SafeModeAction.SAFEMODE_GET, false));
     assertTrue(routerProtocol.setSafeMode(
     assertTrue(routerProtocol.setSafeMode(
-        SafeModeAction.SAFEMODE_ENTER, false));
+        HdfsConstants.SafeModeAction.SAFEMODE_ENTER, false));
     assertTrue(routerProtocol.setSafeMode(
     assertTrue(routerProtocol.setSafeMode(
-        SafeModeAction.SAFEMODE_GET, false));
+        HdfsConstants.SafeModeAction.SAFEMODE_GET, false));
     assertFalse(routerProtocol.setSafeMode(
     assertFalse(routerProtocol.setSafeMode(
-        SafeModeAction.SAFEMODE_LEAVE, false));
+        HdfsConstants.SafeModeAction.SAFEMODE_LEAVE, false));
     assertFalse(routerProtocol.setSafeMode(
     assertFalse(routerProtocol.setSafeMode(
-        SafeModeAction.SAFEMODE_GET, false));
+        HdfsConstants.SafeModeAction.SAFEMODE_GET, false));
   }
   }
 
 
   @Test
   @Test
@@ -1797,18 +1797,18 @@ public class TestRouterRpc {
   @Test
   @Test
   public void testSaveNamespace() throws IOException {
   public void testSaveNamespace() throws IOException {
     cluster.getCluster().getFileSystem(0)
     cluster.getCluster().getFileSystem(0)
-        .setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER);
+        .setSafeMode(SafeModeAction.ENTER);
     cluster.getCluster().getFileSystem(1)
     cluster.getCluster().getFileSystem(1)
-        .setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER);
+        .setSafeMode(SafeModeAction.ENTER);
 
 
     Boolean saveNamespace = routerProtocol.saveNamespace(0, 0);
     Boolean saveNamespace = routerProtocol.saveNamespace(0, 0);
 
 
     assertTrue(saveNamespace);
     assertTrue(saveNamespace);
 
 
     cluster.getCluster().getFileSystem(0)
     cluster.getCluster().getFileSystem(0)
-        .setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
+        .setSafeMode(SafeModeAction.LEAVE);
     cluster.getCluster().getFileSystem(1)
     cluster.getCluster().getFileSystem(1)
-        .setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
+        .setSafeMode(SafeModeAction.LEAVE);
   }
   }
 
 
   /*
   /*

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpcSingleNS.java

@@ -19,9 +19,9 @@ package org.apache.hadoop.hdfs.server.federation.router;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.SafeModeAction;
 import org.apache.hadoop.hdfs.NameNodeProxies;
 import org.apache.hadoop.hdfs.NameNodeProxies;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.federation.MiniRouterDFSCluster;
 import org.apache.hadoop.hdfs.server.federation.MiniRouterDFSCluster;
 import org.apache.hadoop.hdfs.server.federation.RouterConfigBuilder;
 import org.apache.hadoop.hdfs.server.federation.RouterConfigBuilder;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
@@ -203,7 +203,7 @@ public class TestRouterRpcSingleNS {
   @Test
   @Test
   public void testSaveNamespace() throws IOException {
   public void testSaveNamespace() throws IOException {
     cluster.getCluster().getFileSystem()
     cluster.getCluster().getFileSystem()
-        .setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER);
+        .setSafeMode(SafeModeAction.ENTER);
     Boolean saveNamespace = routerProtocol.saveNamespace(0, 0);
     Boolean saveNamespace = routerProtocol.saveNamespace(0, 0);
 
 
     assertTrue(saveNamespace);
     assertTrue(saveNamespace);

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java

@@ -494,6 +494,7 @@ public class DFSAdmin extends FsShell {
    * Gives a report on how the FileSystem is doing.
    * Gives a report on how the FileSystem is doing.
    * @exception IOException if the filesystem does not exist.
    * @exception IOException if the filesystem does not exist.
    */
    */
+  @SuppressWarnings("deprecation")
   public void report(String[] argv, int i) throws IOException {
   public void report(String[] argv, int i) throws IOException {
     DistributedFileSystem dfs = getDFS();
     DistributedFileSystem dfs = getDFS();
     FsStatus ds = dfs.getStatus();
     FsStatus ds = dfs.getStatus();
@@ -660,6 +661,7 @@ public class DFSAdmin extends FsShell {
    * @param idx The index of the command that is being processed.
    * @param idx The index of the command that is being processed.
    * @exception IOException if the filesystem does not exist.
    * @exception IOException if the filesystem does not exist.
    */
    */
+  @SuppressWarnings("deprecation")
   public void setSafeMode(String[] argv, int idx) throws IOException {
   public void setSafeMode(String[] argv, int idx) throws IOException {
     if (idx != argv.length - 1) {
     if (idx != argv.length - 1) {
       printUsage("-safemode");
       printUsage("-safemode");
@@ -713,6 +715,7 @@ public class DFSAdmin extends FsShell {
 
 
   }
   }
 
 
+  @SuppressWarnings("deprecation")
   private boolean waitExitSafeMode(DistributedFileSystem dfs, boolean inSafeMode)
   private boolean waitExitSafeMode(DistributedFileSystem dfs, boolean inSafeMode)
       throws IOException {
       throws IOException {
     while (inSafeMode) {
     while (inSafeMode) {

+ 49 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/contract/hdfs/TestHDFSContractLeaseRecovery.java

@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.fs.contract.hdfs;
+
+import java.io.IOException;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractLeaseRecoveryTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * Test lease recovery on HDFS.
+ */
+public class TestHDFSContractLeaseRecovery extends AbstractContractLeaseRecoveryTest {
+
+  @BeforeClass
+  public static void createCluster() throws IOException {
+    HDFSContract.createCluster();
+  }
+
+  @AfterClass
+  public static void teardownCluster() throws IOException {
+    HDFSContract.destroyCluster();
+  }
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new HDFSContract(conf);
+  }
+}

+ 49 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/contract/hdfs/TestHDFSContractSafeMode.java

@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.fs.contract.hdfs;
+
+import java.io.IOException;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.contract.AbstractContractSafeModeTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+
+/**
+ * Test safe mode interface on HDFS.
+ */
+public class TestHDFSContractSafeMode extends AbstractContractSafeModeTest {
+
+  @BeforeClass
+  public static void createCluster() throws IOException {
+    HDFSContract.createCluster();
+  }
+
+  @AfterClass
+  public static void teardownCluster() throws IOException {
+    HDFSContract.destroyCluster();
+  }
+
+  @Override
+  protected AbstractFSContract createContract(Configuration conf) {
+    return new HDFSContract(conf);
+  }
+}

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java

@@ -29,9 +29,9 @@ import org.apache.hadoop.fs.BlockStoragePolicySpi;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.SafeModeAction;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.protocol.*;
 import org.apache.hadoop.hdfs.protocol.*;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.*;
 import org.apache.hadoop.hdfs.server.blockmanagement.*;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
@@ -1009,9 +1009,9 @@ public class TestBlockStoragePolicy {
       checkDirectoryListing(barList, WARM, HOT);
       checkDirectoryListing(barList, WARM, HOT);
 
 
       // restart namenode with checkpoint to make sure the fsimage is correct
       // restart namenode with checkpoint to make sure the fsimage is correct
-      fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+      fs.setSafeMode(SafeModeAction.ENTER);
       fs.saveNamespace();
       fs.saveNamespace();
-      fs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+      fs.setSafeMode(SafeModeAction.LEAVE);
       cluster.restartNameNode(true);
       cluster.restartNameNode(true);
       dirList = fs.getClient().listPaths(dir.toString(),
       dirList = fs.getClient().listPaths(dir.toString(),
           HdfsFileStatus.EMPTY_NAME).getPartialListing();
           HdfsFileStatus.EMPTY_NAME).getPartialListing();

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java

@@ -60,6 +60,7 @@ import org.apache.hadoop.fs.FileChecksum;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.SafeModeAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.client.HdfsUtils;
 import org.apache.hadoop.hdfs.client.HdfsUtils;
@@ -69,7 +70,6 @@ import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
@@ -1120,7 +1120,7 @@ public class TestDFSClientRetries {
 
 
       //enter safe mode
       //enter safe mode
       assertTrue(HdfsUtils.isHealthy(uri));
       assertTrue(HdfsUtils.isHealthy(uri));
-      dfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+      dfs.setSafeMode(SafeModeAction.ENTER);
       assertFalse(HdfsUtils.isHealthy(uri));
       assertFalse(HdfsUtils.isHealthy(uri));
       
       
       //leave safe mode in a new thread
       //leave safe mode in a new thread
@@ -1131,7 +1131,7 @@ public class TestDFSClientRetries {
             //sleep and then leave safe mode
             //sleep and then leave safe mode
             TimeUnit.SECONDS.sleep(30);
             TimeUnit.SECONDS.sleep(30);
             assertFalse(HdfsUtils.isHealthy(uri));
             assertFalse(HdfsUtils.isHealthy(uri));
-            dfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+            dfs.setSafeMode(SafeModeAction.LEAVE);
             assertTrue(HdfsUtils.isHealthy(uri));
             assertTrue(HdfsUtils.isHealthy(uri));
           } catch (Exception e) {
           } catch (Exception e) {
             exceptions.add(e);
             exceptions.add(e);

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgrade.java

@@ -34,8 +34,8 @@ import java.util.regex.Pattern;
 import org.slf4j.Logger;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
 import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
@@ -239,7 +239,7 @@ public class TestDFSUpgrade {
       // make sure that rolling upgrade cannot be started
       // make sure that rolling upgrade cannot be started
       try {
       try {
         final DistributedFileSystem dfs = cluster.getFileSystem();
         final DistributedFileSystem dfs = cluster.getFileSystem();
-        dfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+        dfs.setSafeMode(SafeModeAction.ENTER);
         dfs.rollingUpgrade(RollingUpgradeAction.PREPARE);
         dfs.rollingUpgrade(RollingUpgradeAction.PREPARE);
         fail();
         fail();
       } catch(RemoteException re) {
       } catch(RemoteException re) {
@@ -384,7 +384,7 @@ public class TestDFSUpgrade {
       // make sure that rolling upgrade cannot be started
       // make sure that rolling upgrade cannot be started
       try {
       try {
         final DistributedFileSystem dfs = cluster.getFileSystem();
         final DistributedFileSystem dfs = cluster.getFileSystem();
-        dfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+        dfs.setSafeMode(SafeModeAction.ENTER);
         dfs.rollingUpgrade(RollingUpgradeAction.PREPARE);
         dfs.rollingUpgrade(RollingUpgradeAction.PREPARE);
         fail();
         fail();
       } catch(RemoteException re) {
       } catch(RemoteException re) {

+ 8 - 8
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java

@@ -73,11 +73,11 @@ import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.FsShell;
 import org.apache.hadoop.fs.FsShell;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.SafeModeAction;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.client.CreateEncryptionZoneFlag;
 import org.apache.hadoop.hdfs.client.CreateEncryptionZoneFlag;
 import org.apache.hadoop.hdfs.client.HdfsAdmin;
 import org.apache.hadoop.hdfs.client.HdfsAdmin;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
@@ -573,9 +573,9 @@ public class TestEncryptionZones {
       assertZonePresent(null, zonePath.toString());
       assertZonePresent(null, zonePath.toString());
     }
     }
 
 
-    fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+    fs.setSafeMode(SafeModeAction.ENTER);
     fs.saveNamespace();
     fs.saveNamespace();
-    fs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+    fs.setSafeMode(SafeModeAction.LEAVE);
     cluster.restartNameNode(true);
     cluster.restartNameNode(true);
     assertNumZones(numZones);
     assertNumZones(numZones);
     assertEquals("Unexpected number of encryption zones!", numZones, cluster
     assertEquals("Unexpected number of encryption zones!", numZones, cluster
@@ -608,9 +608,9 @@ public class TestEncryptionZones {
 
 
     // Verify rootDir ez is present after restarting the NameNode
     // Verify rootDir ez is present after restarting the NameNode
     // and saving/loading from fsimage.
     // and saving/loading from fsimage.
-    fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+    fs.setSafeMode(SafeModeAction.ENTER);
     fs.saveNamespace();
     fs.saveNamespace();
-    fs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+    fs.setSafeMode(SafeModeAction.LEAVE);
     cluster.restartNameNode(true);
     cluster.restartNameNode(true);
     assertNumZones(numZones);
     assertNumZones(numZones);
     assertZonePresent(null, rootDir.toString());
     assertZonePresent(null, rootDir.toString());
@@ -1203,9 +1203,9 @@ public class TestEncryptionZones {
         fs.getSnapshotDiffReport(snapshottable, "snap1", "");
         fs.getSnapshotDiffReport(snapshottable, "snap1", "");
     System.out.println(report);
     System.out.println(report);
     Assert.assertEquals(0, report.getDiffList().size());
     Assert.assertEquals(0, report.getDiffList().size());
-    fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+    fs.setSafeMode(SafeModeAction.ENTER);
     fs.saveNamespace();
     fs.saveNamespace();
-    fs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+    fs.setSafeMode(SafeModeAction.LEAVE);
     cluster.restartNameNode(true);
     cluster.restartNameNode(true);
     report =
     report =
         fs.getSnapshotDiffReport(snapshottable, "snap1", "");
         fs.getSnapshotDiffReport(snapshottable, "snap1", "");
@@ -1719,7 +1719,7 @@ public class TestEncryptionZones {
     fsWrapper.mkdir(zone1, FsPermission.getDirDefault(), true);
     fsWrapper.mkdir(zone1, FsPermission.getDirDefault(), true);
     dfsAdmin.createEncryptionZone(zone1, TEST_KEY, NO_TRASH);
     dfsAdmin.createEncryptionZone(zone1, TEST_KEY, NO_TRASH);
     DFSTestUtil.createFile(fs, zone1File, len, (short) 1, 0xFEED);
     DFSTestUtil.createFile(fs, zone1File, len, (short) 1, 0xFEED);
-    fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER, false);
+    fs.setSafeMode(SafeModeAction.ENTER, false);
     fs.saveNamespace();
     fs.saveNamespace();
 
 
     File originalFsimage = FSImageTestUtil.findLatestImageFile(FSImageTestUtil
     File originalFsimage = FSImageTestUtil.findLatestImageFile(FSImageTestUtil

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java

@@ -23,12 +23,12 @@ import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.SafeModeAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.protocol.AddErasureCodingPolicyResponse;
 import org.apache.hadoop.hdfs.protocol.AddErasureCodingPolicyResponse;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicyInfo;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicyInfo;
 import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
 import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.server.namenode.ErasureCodingPolicyManager;
 import org.apache.hadoop.hdfs.server.namenode.ErasureCodingPolicyManager;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
@@ -236,9 +236,9 @@ public class TestErasureCodingPolicies {
         fs.getErasureCodingPolicy(disabledPolicy));
         fs.getErasureCodingPolicy(disabledPolicy));
 
 
     // Also check loading disabled EC policies from fsimage
     // Also check loading disabled EC policies from fsimage
-    fs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER);
+    fs.setSafeMode(SafeModeAction.ENTER);
     fs.saveNamespace();
     fs.saveNamespace();
-    fs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
+    fs.setSafeMode(SafeModeAction.LEAVE);
     cluster.restartNameNodes();
     cluster.restartNameNodes();
 
 
     Assert.assertEquals("Dir does not have policy set",
     Assert.assertEquals("Dir does not have policy set",

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicyWithSnapshot.java

@@ -25,10 +25,10 @@ import java.io.IOException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FsShell;
 import org.apache.hadoop.fs.FsShell;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.SafeModeAction;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
 import org.apache.hadoop.fs.contract.ContractTestUtils;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
 import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.hadoop.util.ToolRunner;
 import org.junit.After;
 import org.junit.After;
@@ -168,9 +168,9 @@ public class TestErasureCodingPolicyWithSnapshot {
         ecSnap);
         ecSnap);
 
 
     // save namespace, restart namenode, and check ec policy correctness.
     // save namespace, restart namenode, and check ec policy correctness.
-    fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+    fs.setSafeMode(SafeModeAction.ENTER);
     fs.saveNamespace();
     fs.saveNamespace();
-    fs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+    fs.setSafeMode(SafeModeAction.LEAVE);
     cluster.restartNameNode(true);
     cluster.restartNameNode(true);
 
 
     ErasureCodingPolicy ecSnap1 = fs.getErasureCodingPolicy(snap1);
     ErasureCodingPolicy ecSnap1 = fs.getErasureCodingPolicy(snap1);

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestGetBlocks.java

@@ -34,6 +34,7 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.SafeModeAction;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
 import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
@@ -41,7 +42,6 @@ import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
@@ -287,13 +287,13 @@ public class TestGetBlocks {
       assertEquals(blkLocsSize, locs.length);
       assertEquals(blkLocsSize, locs.length);
       assertFalse(fs.isInSafeMode());
       assertFalse(fs.isInSafeMode());
       LOG.info("Entering safe mode");
       LOG.info("Entering safe mode");
-      fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+      fs.setSafeMode(SafeModeAction.ENTER);
       LOG.info("Entered safe mode");
       LOG.info("Entered safe mode");
       assertTrue(fs.isInSafeMode());
       assertTrue(fs.isInSafeMode());
       getBlocksWithException(namenode, info, replicationFactor, 0,
       getBlocksWithException(namenode, info, replicationFactor, 0,
           RemoteException.class,
           RemoteException.class,
           "Cannot execute getBlocks. Name node is in safe mode.");
           "Cannot execute getBlocks. Name node is in safe mode.");
-      fs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+      fs.setSafeMode(SafeModeAction.LEAVE);
       assertFalse(fs.isInSafeMode());
       assertFalse(fs.isInSafeMode());
     }  finally {
     }  finally {
       if (fs != null) {
       if (fs != null) {

+ 18 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHDFSFileSystemContract.java

@@ -24,13 +24,18 @@ import java.io.IOException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.FileSystemContractBaseTest;
 import org.apache.hadoop.fs.FileSystemContractBaseTest;
+import org.apache.hadoop.fs.LeaseRecoverable;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.SafeMode;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.After;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.Test;
 
 
+import static org.apache.hadoop.fs.CommonPathCapabilities.LEASE_RECOVERABLE;
+import static org.assertj.core.api.Assertions.assertThat;
+
 public class TestHDFSFileSystemContract extends FileSystemContractBaseTest {
 public class TestHDFSFileSystemContract extends FileSystemContractBaseTest {
   
   
   private MiniDFSCluster cluster;
   private MiniDFSCluster cluster;
@@ -72,4 +77,17 @@ public class TestHDFSFileSystemContract extends FileSystemContractBaseTest {
   public void testAppend() throws IOException {
   public void testAppend() throws IOException {
     AppendTestUtil.testAppend(fs, new Path("/testAppend/f"));
     AppendTestUtil.testAppend(fs, new Path("/testAppend/f"));
   }
   }
+
+  @Test
+  public void testFileSystemCapabilities() throws Exception {
+    final Path p = new Path("testFileSystemCapabilities");
+    // ViewFileSystem does not support LeaseRecoverable and SafeMode.
+    if (fs instanceof DistributedFileSystem) {
+      final boolean leaseRecovery = fs.hasPathCapability(p, LEASE_RECOVERABLE);
+      assertThat(leaseRecovery).describedAs("path capabilities %s=%s in %s", LEASE_RECOVERABLE,
+          leaseRecovery, fs).isTrue();
+      assertThat(fs).describedAs("filesystem %s", fs).isInstanceOf(LeaseRecoverable.class);
+      assertThat(fs).describedAs("filesystem %s", fs).isInstanceOf(SafeMode.class);
+    }
+  }
 }
 }

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestMiniDFSCluster.java

@@ -34,9 +34,9 @@ import java.util.concurrent.TimeoutException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.SafeModeAction;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.MiniDFSCluster.NameNodeInfo;
 import org.apache.hadoop.hdfs.MiniDFSCluster.NameNodeInfo;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsVolumeImpl;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsVolumeImpl;
@@ -219,7 +219,7 @@ public class TestMiniDFSCluster {
         new MiniDFSCluster.Builder(conf, testDataCluster4).build();
         new MiniDFSCluster.Builder(conf, testDataCluster4).build();
     try {
     try {
       DistributedFileSystem dfs = cluster4.getFileSystem();
       DistributedFileSystem dfs = cluster4.getFileSystem();
-      dfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER);
+      dfs.setSafeMode(SafeModeAction.ENTER);
       cluster4.shutdown();
       cluster4.shutdown();
     } finally {
     } finally {
       while(cluster4.isClusterUp()){
       while(cluster4.isClusterUp()){

+ 14 - 14
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRollingUpgrade.java

@@ -45,9 +45,9 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.SafeModeAction;
 import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
 import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.qjournal.MiniJournalCluster;
 import org.apache.hadoop.hdfs.qjournal.MiniJournalCluster;
 import org.apache.hadoop.hdfs.qjournal.MiniQJMHACluster;
 import org.apache.hadoop.hdfs.qjournal.MiniQJMHACluster;
@@ -136,9 +136,9 @@ public class TestRollingUpgrade {
         runCmd(dfsadmin, true, "-rollingUpgrade");
         runCmd(dfsadmin, true, "-rollingUpgrade");
 
 
         //start rolling upgrade
         //start rolling upgrade
-        dfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+        dfs.setSafeMode(SafeModeAction.ENTER);
         runCmd(dfsadmin, true, "-rollingUpgrade", "prepare");
         runCmd(dfsadmin, true, "-rollingUpgrade", "prepare");
-        dfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+        dfs.setSafeMode(SafeModeAction.LEAVE);
 
 
         //query rolling upgrade
         //query rolling upgrade
         runCmd(dfsadmin, true, "-rollingUpgrade", "query");
         runCmd(dfsadmin, true, "-rollingUpgrade", "query");
@@ -163,9 +163,9 @@ public class TestRollingUpgrade {
         Assert.assertTrue(dfs.exists(bar));
         Assert.assertTrue(dfs.exists(bar));
         Assert.assertTrue(dfs.exists(baz));
         Assert.assertTrue(dfs.exists(baz));
 
 
-        dfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+        dfs.setSafeMode(SafeModeAction.ENTER);
         dfs.saveNamespace();
         dfs.saveNamespace();
-        dfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+        dfs.setSafeMode(SafeModeAction.LEAVE);
       }
       }
 
 
       // Ensure directories exist after restart
       // Ensure directories exist after restart
@@ -241,9 +241,9 @@ public class TestRollingUpgrade {
           dfs.mkdirs(foo);
           dfs.mkdirs(foo);
 
 
           //start rolling upgrade
           //start rolling upgrade
-          dfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+          dfs.setSafeMode(SafeModeAction.ENTER);
           info1 = dfs.rollingUpgrade(RollingUpgradeAction.PREPARE);
           info1 = dfs.rollingUpgrade(RollingUpgradeAction.PREPARE);
-          dfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+          dfs.setSafeMode(SafeModeAction.LEAVE);
           LOG.info("START\n" + info1);
           LOG.info("START\n" + info1);
 
 
           //query rolling upgrade
           //query rolling upgrade
@@ -397,9 +397,9 @@ public class TestRollingUpgrade {
     final DistributedFileSystem dfs = cluster.getFileSystem();
     final DistributedFileSystem dfs = cluster.getFileSystem();
 
 
     //start rolling upgrade
     //start rolling upgrade
-    dfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+    dfs.setSafeMode(SafeModeAction.ENTER);
     dfs.rollingUpgrade(RollingUpgradeAction.PREPARE);
     dfs.rollingUpgrade(RollingUpgradeAction.PREPARE);
-    dfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+    dfs.setSafeMode(SafeModeAction.LEAVE);
 
 
     dfs.mkdirs(bar);
     dfs.mkdirs(bar);
     Assert.assertTrue(dfs.exists(foo));
     Assert.assertTrue(dfs.exists(foo));
@@ -587,13 +587,13 @@ public class TestRollingUpgrade {
       cluster.waitActive();
       cluster.waitActive();
       DistributedFileSystem dfs = cluster.getFileSystem();
       DistributedFileSystem dfs = cluster.getFileSystem();
 
 
-      dfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+      dfs.setSafeMode(SafeModeAction.ENTER);
       // start rolling upgrade
       // start rolling upgrade
       dfs.rollingUpgrade(RollingUpgradeAction.PREPARE);
       dfs.rollingUpgrade(RollingUpgradeAction.PREPARE);
       queryForPreparation(dfs);
       queryForPreparation(dfs);
-      dfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+      dfs.setSafeMode(SafeModeAction.ENTER);
       dfs.saveNamespace();
       dfs.saveNamespace();
-      dfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+      dfs.setSafeMode(SafeModeAction.LEAVE);
 
 
       cluster.restartNameNodes();
       cluster.restartNameNodes();
       dfs.rollingUpgrade(RollingUpgradeAction.QUERY);
       dfs.rollingUpgrade(RollingUpgradeAction.QUERY);
@@ -781,9 +781,9 @@ public class TestRollingUpgrade {
       snn.doCheckpoint();
       snn.doCheckpoint();
 
 
       //start rolling upgrade
       //start rolling upgrade
-      dfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+      dfs.setSafeMode(SafeModeAction.ENTER);
       dfs.rollingUpgrade(RollingUpgradeAction.PREPARE);
       dfs.rollingUpgrade(RollingUpgradeAction.PREPARE);
-      dfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+      dfs.setSafeMode(SafeModeAction.LEAVE);
 
 
       dfs.mkdirs(new Path("/test/bar"));
       dfs.mkdirs(new Path("/test/bar"));
       // do checkpoint in SNN again
       // do checkpoint in SNN again

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRollingUpgradeDowngrade.java

@@ -24,8 +24,8 @@ import java.io.IOException;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.qjournal.MiniQJMHACluster;
 import org.apache.hadoop.hdfs.qjournal.MiniQJMHACluster;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage;
@@ -99,9 +99,9 @@ public class TestRollingUpgradeDowngrade {
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).build();
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).build();
       cluster.waitActive();
       cluster.waitActive();
       DistributedFileSystem fs = cluster.getFileSystem();
       DistributedFileSystem fs = cluster.getFileSystem();
-      fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+      fs.setSafeMode(SafeModeAction.ENTER);
       fs.saveNamespace();
       fs.saveNamespace();
-      fs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+      fs.setSafeMode(SafeModeAction.LEAVE);
       NNStorage storage = spy(cluster.getNameNode().getFSImage().getStorage());
       NNStorage storage = spy(cluster.getNameNode().getFSImage().getStorage());
       int futureVersion = NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION - 1;
       int futureVersion = NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION - 1;
       doReturn(futureVersion).when(storage).getServiceLayoutVersion();
       doReturn(futureVersion).when(storage).getServiceLayoutVersion();

+ 5 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRollingUpgradeRollback.java

@@ -22,8 +22,8 @@ import java.util.List;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.qjournal.MiniJournalCluster;
 import org.apache.hadoop.hdfs.qjournal.MiniJournalCluster;
 import org.apache.hadoop.hdfs.qjournal.MiniQJMHACluster;
 import org.apache.hadoop.hdfs.qjournal.MiniQJMHACluster;
@@ -102,10 +102,10 @@ public class TestRollingUpgradeRollback {
       dfs.mkdirs(foo);
       dfs.mkdirs(foo);
 
 
       // start rolling upgrade
       // start rolling upgrade
-      dfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+      dfs.setSafeMode(SafeModeAction.ENTER);
       Assert.assertEquals(0,
       Assert.assertEquals(0,
           dfsadmin.run(new String[] { "-rollingUpgrade", "prepare" }));
           dfsadmin.run(new String[] { "-rollingUpgrade", "prepare" }));
-      dfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+      dfs.setSafeMode(SafeModeAction.LEAVE);
       // create new directory
       // create new directory
       dfs.mkdirs(bar);
       dfs.mkdirs(bar);
 
 
@@ -164,10 +164,10 @@ public class TestRollingUpgradeRollback {
       dfs.mkdirs(foo);
       dfs.mkdirs(foo);
 
 
       // start rolling upgrade
       // start rolling upgrade
-      dfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+      dfs.setSafeMode(SafeModeAction.ENTER);
       Assert.assertEquals(0,
       Assert.assertEquals(0,
           dfsadmin.run(new String[] { "-rollingUpgrade", "prepare" }));
           dfsadmin.run(new String[] { "-rollingUpgrade", "prepare" }));
-      dfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+      dfs.setSafeMode(SafeModeAction.LEAVE);
       // create new directory
       // create new directory
       dfs.mkdirs(bar);
       dfs.mkdirs(bar);
       dfs.close();
       dfs.close();

+ 14 - 14
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java

@@ -37,12 +37,12 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.SafeModeAction;
 import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
 import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
@@ -134,10 +134,10 @@ public class TestSafeMode {
     dfs = cluster.getFileSystem();
     dfs = cluster.getFileSystem();
     
     
     assertTrue("No datanode is started. Should be in SafeMode", 
     assertTrue("No datanode is started. Should be in SafeMode", 
-               dfs.setSafeMode(SafeModeAction.SAFEMODE_GET));
+               dfs.setSafeMode(SafeModeAction.GET));
     
     
     // manually set safemode.
     // manually set safemode.
-    dfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+    dfs.setSafeMode(SafeModeAction.ENTER);
     
     
     // now bring up the datanode and wait for it to be active.
     // now bring up the datanode and wait for it to be active.
     cluster.startDataNodes(conf, 1, true, null, null);
     cluster.startDataNodes(conf, 1, true, null, null);
@@ -149,9 +149,9 @@ public class TestSafeMode {
     } catch (InterruptedException ignored) {}
     } catch (InterruptedException ignored) {}
 
 
     assertTrue("should still be in SafeMode",
     assertTrue("should still be in SafeMode",
-        dfs.setSafeMode(SafeModeAction.SAFEMODE_GET));
+        dfs.setSafeMode(SafeModeAction.GET));
     assertFalse("should not be in SafeMode", 
     assertFalse("should not be in SafeMode", 
-        dfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE));
+        dfs.setSafeMode(SafeModeAction.LEAVE));
   }
   }
 
 
   /**
   /**
@@ -311,7 +311,7 @@ public class TestSafeMode {
     final Path file1 = new Path("/file1");
     final Path file1 = new Path("/file1");
     DFSTestUtil.createFile(fs, file1, 1024, (short)1, 0);
     DFSTestUtil.createFile(fs, file1, 1024, (short)1, 0);
     assertTrue("Could not enter SM",
     assertTrue("Could not enter SM",
-        dfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER));
+        dfs.setSafeMode(SafeModeAction.ENTER));
     try {
     try {
       FSRun fsRun = new FSRun() {
       FSRun fsRun = new FSRun() {
         @Override
         @Override
@@ -339,10 +339,10 @@ public class TestSafeMode {
       InterruptedException {
       InterruptedException {
     final Path file1 = new Path("/file1");
     final Path file1 = new Path("/file1");
 
 
-    assertFalse(dfs.setSafeMode(SafeModeAction.SAFEMODE_GET));
+    assertFalse(dfs.setSafeMode(SafeModeAction.GET));
     DFSTestUtil.createFile(fs, file1, 1024, (short)1, 0);
     DFSTestUtil.createFile(fs, file1, 1024, (short)1, 0);
     assertTrue("Could not enter SM", 
     assertTrue("Could not enter SM", 
-        dfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER));
+        dfs.setSafeMode(SafeModeAction.ENTER));
 
 
     runFsFun("Set quota while in SM", new FSRun() { 
     runFsFun("Set quota while in SM", new FSRun() { 
       @Override
       @Override
@@ -492,7 +492,7 @@ public class TestSafeMode {
     }
     }
 
 
     assertFalse("Could not leave SM",
     assertFalse("Could not leave SM",
-        dfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE));
+        dfs.setSafeMode(SafeModeAction.LEAVE));
   }
   }
 
 
   /**
   /**
@@ -536,11 +536,11 @@ public class TestSafeMode {
     dfs = cluster.getFileSystem();
     dfs = cluster.getFileSystem();
 
 
     // Enter safemode.
     // Enter safemode.
-    dfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+    dfs.setSafeMode(SafeModeAction.ENTER);
     assertTrue("State was expected to be in safemode.", dfs.isInSafeMode());
     assertTrue("State was expected to be in safemode.", dfs.isInSafeMode());
 
 
     // Exit safemode.
     // Exit safemode.
-    dfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+    dfs.setSafeMode(SafeModeAction.LEAVE);
     assertFalse("State was expected to be out of safemode.", dfs.isInSafeMode());
     assertFalse("State was expected to be out of safemode.", dfs.isInSafeMode());
   }
   }
   
   
@@ -561,11 +561,11 @@ public class TestSafeMode {
       NameNode namenode = cluster.getNameNode();
       NameNode namenode = cluster.getNameNode();
 
 
       // manually set safemode.
       // manually set safemode.
-      dfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+      dfs.setSafeMode(SafeModeAction.ENTER);
       assertTrue("should still be in SafeMode", namenode.isInSafeMode());
       assertTrue("should still be in SafeMode", namenode.isInSafeMode());
       // getBlock locations should still work since block locations exists
       // getBlock locations should still work since block locations exists
       checkGetBlockLocationsWorks(fs, file1);
       checkGetBlockLocationsWorks(fs, file1);
-      dfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+      dfs.setSafeMode(SafeModeAction.LEAVE);
       assertFalse("should not be in SafeMode", namenode.isInSafeMode());
       assertFalse("should not be in SafeMode", namenode.isInSafeMode());
       
       
       
       
@@ -595,7 +595,7 @@ public class TestSafeMode {
       }
       }
 
 
 
 
-      dfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);      
+      dfs.setSafeMode(SafeModeAction.LEAVE);
       assertFalse("Should not be in safemode", namenode.isInSafeMode());
       assertFalse("Should not be in safemode", namenode.isInSafeMode());
       checkGetBlockLocationsWorks(fs, file1);
       checkGetBlockLocationsWorks(fs, file1);
 
 

+ 99 - 11
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestViewDistributedFileSystem.java

@@ -21,18 +21,26 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.CommonPathCapabilities;
 import org.apache.hadoop.fs.CommonPathCapabilities;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LeaseRecoverable;
 import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathHandle;
 import org.apache.hadoop.fs.PathHandle;
+import org.apache.hadoop.fs.SafeMode;
+import org.apache.hadoop.fs.SafeModeAction;
 import org.apache.hadoop.fs.viewfs.ConfigUtil;
 import org.apache.hadoop.fs.viewfs.ConfigUtil;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.test.Whitebox;
 import org.apache.hadoop.test.Whitebox;
+import org.apache.hadoop.util.functional.ConsumerRaisingIOE;
+
+import org.assertj.core.api.Assertions;
 import org.junit.Assert;
 import org.junit.Assert;
 import org.junit.Test;
 import org.junit.Test;
 
 
 import java.io.IOException;
 import java.io.IOException;
 import java.net.URI;
 import java.net.URI;
 
 
+import static org.apache.hadoop.fs.CommonPathCapabilities.LEASE_RECOVERABLE;
+import static org.assertj.core.api.Assertions.assertThat;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertTrue;
@@ -195,18 +203,98 @@ public class TestViewDistributedFileSystem extends TestDistributedFileSystem{
 
 
   @Test
   @Test
   public void testPathCapabilities() throws IOException {
   public void testPathCapabilities() throws IOException {
-    Configuration conf = getTestConfiguration();
-    try (MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).build()) {
-      URI defaultUri = URI.create(conf.get(CommonConfigurationKeys.FS_DEFAULT_NAME_KEY));
-      conf.set("fs.viewfs.mounttable." + defaultUri.getHost() + ".linkFallback",
-          defaultUri.toString());
-      try (ViewDistributedFileSystem fileSystem = (ViewDistributedFileSystem) FileSystem.get(
-          conf)) {
-        final Path testFile = new Path("/test");
-        assertTrue("ViewDfs supports truncate",
-            fileSystem.hasPathCapability(testFile, CommonPathCapabilities.FS_TRUNCATE));
-      }
+    try (MiniDFSCluster cluster = new MiniDFSCluster.Builder(getViewFsConfiguration())
+        .numDataNodes(0).build();
+        ViewDistributedFileSystem fileSystem = (ViewDistributedFileSystem) FileSystem.get(
+            cluster.getConfiguration(0))) {
+      final Path testFile = new Path("/test");
+      assertTrue("ViewDfs supports truncate",
+          fileSystem.hasPathCapability(testFile, CommonPathCapabilities.FS_TRUNCATE));
+      final boolean isLeaseRecoverable = fileSystem.hasPathCapability(testFile, LEASE_RECOVERABLE);
+      assertThat(isLeaseRecoverable).describedAs("path capabilities %s=%s in %s",
+          LEASE_RECOVERABLE, fileSystem.hasPathCapability(testFile, LEASE_RECOVERABLE),
+          fileSystem).isTrue();
+      assertThat(fileSystem).describedAs("filesystem %s", fileSystem)
+          .isInstanceOf(LeaseRecoverable.class);
+      assertThat(fileSystem).describedAs("filesystem %s", fileSystem).isInstanceOf(SafeMode.class);
+    }
+  }
+
+  @Test
+  public void testSafeMode() throws IOException {
+    testSafeMode(this::executeAssertionsWithSafeMode);
+  }
+
+  @Test
+  public void testSafeModeWithDeprecatedAPIs() throws IOException {
+    testSafeMode(this::executeAssertionsWithDeprecatedAPIs);
+  }
+
+  private void testSafeMode(ConsumerRaisingIOE<ViewDistributedFileSystem> executeAssertionsFunction)
+      throws IOException {
+    try (MiniDFSCluster cluster = new MiniDFSCluster.Builder(getViewFsConfiguration())
+        .numDataNodes(0).build();
+        ViewDistributedFileSystem fileSystem = (ViewDistributedFileSystem) FileSystem.get(
+            cluster.getConfiguration(0))) {
+      executeAssertionsFunction.accept(fileSystem);
     }
     }
   }
   }
 
 
+  private SafeMode verifyAndGetSafeModeInstance(FileSystem fs) {
+    Assertions.assertThat(fs)
+        .describedAs("File system %s must be an instance of %s", fs, SafeMode.class.getClass())
+        .isInstanceOf(SafeMode.class);
+    return (SafeMode) fs;
+  }
+
+  private void executeAssertionsWithSafeMode(ViewDistributedFileSystem fileSystem)
+      throws IOException {
+    SafeMode fsWithSafeMode = verifyAndGetSafeModeInstance(fileSystem);
+    assertSafeModeStatus(fsWithSafeMode, SafeModeAction.GET, false,
+        "Getting the status of safe mode before entering should be off.");
+    assertSafeModeStatus(fsWithSafeMode, SafeModeAction.ENTER, true,
+        "Entering Safe mode and safe mode turns on.");
+    assertSafeModeStatus(fsWithSafeMode, SafeModeAction.GET, true,
+        "Getting the status of safe mode after entering, safe mode should be on.");
+    assertSafeModeStatus(fsWithSafeMode, SafeModeAction.LEAVE, false,
+        "Leaving safe mode, and safe mode switches off.");
+    assertSafeModeStatus(fsWithSafeMode, SafeModeAction.FORCE_EXIT, false,
+        "Force exist safe mode at any time, safe mode should always switches off.");
+  }
+
+  private void executeAssertionsWithDeprecatedAPIs(ViewDistributedFileSystem fileSystem)
+      throws IOException {
+    assertSafeModeStatus(fileSystem, HdfsConstants.SafeModeAction.SAFEMODE_GET, false,
+        "Getting the status of safe mode before entering should be off.");
+    assertSafeModeStatus(fileSystem, HdfsConstants.SafeModeAction.SAFEMODE_ENTER, true,
+        "Entering Safe mode and safe mode turns on.");
+    assertSafeModeStatus(fileSystem, HdfsConstants.SafeModeAction.SAFEMODE_GET, true,
+        "Getting the status of safe mode after entering, safe mode should be on.");
+    assertSafeModeStatus(fileSystem, HdfsConstants.SafeModeAction.SAFEMODE_LEAVE, false,
+        "Leaving safe mode, and safe mode switches off.");
+    assertSafeModeStatus(fileSystem, HdfsConstants.SafeModeAction.SAFEMODE_FORCE_EXIT, false,
+        "Force exist safe mode at any time, safe mode should always switches off.");
+  }
+
+  private void assertSafeModeStatus(SafeMode fsWithSafeMode, SafeModeAction action,
+      boolean expectedStatus, String message) throws IOException {
+    Assertions.assertThat(fsWithSafeMode.setSafeMode(action)).describedAs(message)
+        .isEqualTo(expectedStatus);
+  }
+
+  private void assertSafeModeStatus(ViewDistributedFileSystem fileSystem,
+      HdfsConstants.SafeModeAction action, boolean expectedStatus, String message)
+      throws IOException {
+    Assertions.assertThat(fileSystem.setSafeMode(action)).describedAs(message)
+        .isEqualTo(expectedStatus);
+  }
+
+  private Configuration getViewFsConfiguration() {
+    Configuration conf = getTestConfiguration();
+    URI defaultUri = URI.create(conf.get(CommonConfigurationKeys.FS_DEFAULT_NAME_KEY));
+    conf.set("fs.viewfs.mounttable." + defaultUri.getHost() + ".linkFallback",
+        defaultUri.toString());
+    return conf;
+  }
+
 }
 }

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java

@@ -83,6 +83,7 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.SafeModeAction;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DFSTestUtil;
@@ -1527,9 +1528,9 @@ public class TestBalancer {
     // Run balancer
     // Run balancer
     final BalancerParameters p = BalancerParameters.DEFAULT;
     final BalancerParameters p = BalancerParameters.DEFAULT;
 
 
-    fs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER);
+    fs.setSafeMode(SafeModeAction.ENTER);
     fs.rollingUpgrade(HdfsConstants.RollingUpgradeAction.PREPARE);
     fs.rollingUpgrade(HdfsConstants.RollingUpgradeAction.PREPARE);
-    fs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
+    fs.setSafeMode(SafeModeAction.LEAVE);
 
 
     // Rolling upgrade should abort the balancer
     // Rolling upgrade should abort the balancer
     assertEquals(ExitStatus.UNFINALIZED_UPGRADE.getExitCode(),
     assertEquals(ExitStatus.UNFINALIZED_UPGRADE.getExitCode(),

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeRollingUpgrade.java

@@ -31,6 +31,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.SafeModeAction;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSOutputStream;
 import org.apache.hadoop.hdfs.DFSOutputStream;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DFSTestUtil;
@@ -45,7 +46,6 @@ import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.tools.DFSAdmin;
 import org.apache.hadoop.hdfs.tools.DFSAdmin;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.GenericTestUtils;
@@ -170,7 +170,7 @@ public class TestDataNodeRollingUpgrade {
 
 
   private void startRollingUpgrade() throws Exception {
   private void startRollingUpgrade() throws Exception {
     LOG.info("Starting rolling upgrade");
     LOG.info("Starting rolling upgrade");
-    fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+    fs.setSafeMode(SafeModeAction.ENTER);
     final DFSAdmin dfsadmin = new DFSAdmin(conf);
     final DFSAdmin dfsadmin = new DFSAdmin(conf);
     TestRollingUpgrade.runCmd(dfsadmin, true, "-rollingUpgrade", "prepare");
     TestRollingUpgrade.runCmd(dfsadmin, true, "-rollingUpgrade", "prepare");
     triggerHeartBeats();
     triggerHeartBeats();

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestLazyPersistPolicy.java

@@ -19,7 +19,7 @@
 package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
 package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
 
 
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.fs.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.Test;
 import org.junit.Test;
@@ -79,9 +79,9 @@ public class TestLazyPersistPolicy extends LazyPersistTestCase {
 
 
     makeTestFile(path, 0, true);
     makeTestFile(path, 0, true);
     // checkpoint
     // checkpoint
-    fs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER);
+    fs.setSafeMode(SafeModeAction.ENTER);
     fs.saveNamespace();
     fs.saveNamespace();
-    fs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
+    fs.setSafeMode(SafeModeAction.LEAVE);
     cluster.restartNameNode(true);
     cluster.restartNameNode(true);
 
 
     // Stat the file and check that the lazyPersist flag is returned back.
     // Stat the file and check that the lazyPersist flag is returned back.

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.SafeModeAction;
 import org.apache.hadoop.hdfs.DFSStripedOutputStream;
 import org.apache.hadoop.hdfs.DFSStripedOutputStream;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
@@ -31,7 +32,6 @@ import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
@@ -194,9 +194,9 @@ public class TestAddStripedBlocks {
 
 
       // save namespace, restart namenode, and check
       // save namespace, restart namenode, and check
       dfs = cluster.getFileSystem();
       dfs = cluster.getFileSystem();
-      dfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER);
+      dfs.setSafeMode(SafeModeAction.ENTER);
       dfs.saveNamespace();
       dfs.saveNamespace();
-      dfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
+      dfs.setSafeMode(SafeModeAction.LEAVE);
       cluster.restartNameNode(true);
       cluster.restartNameNode(true);
       fsdir = cluster.getNamesystem().getFSDirectory();
       fsdir = cluster.getNamesystem().getFSDirectory();
       fileNode = fsdir.getINode4Write(file.toString()).asFile();
       fileNode = fsdir.getINode4Write(file.toString()).asFile();

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCacheDirectives.java

@@ -55,6 +55,7 @@ import org.apache.hadoop.fs.FileSystemTestHelper;
 import org.apache.hadoop.fs.FsTracer;
 import org.apache.hadoop.fs.FsTracer;
 import org.apache.hadoop.fs.InvalidRequestException;
 import org.apache.hadoop.fs.InvalidRequestException;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.SafeModeAction;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.client.impl.BlockReaderTestUtil;
 import org.apache.hadoop.hdfs.client.impl.BlockReaderTestUtil;
@@ -75,7 +76,6 @@ import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.CachePoolStats;
 import org.apache.hadoop.hdfs.protocol.CachePoolStats;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.CachedBlocksList.Type;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.CachedBlocksList.Type;
@@ -693,9 +693,9 @@ public class TestCacheDirectives {
         .setPath(new Path("/image")).setPool(imagePool).build());
         .setPath(new Path("/image")).setPool(imagePool).build());
 
 
       // Save a new image to force a fresh fsimage download
       // Save a new image to force a fresh fsimage download
-      dfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+      dfs.setSafeMode(SafeModeAction.ENTER);
       dfs.saveNamespace();
       dfs.saveNamespace();
-      dfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+      dfs.setSafeMode(SafeModeAction.LEAVE);
 
 
       // Checkpoint again forcing a reload of FSN state
       // Checkpoint again forcing a reload of FSN state
       boolean fetchImage = secondary.doCheckpoint();
       boolean fetchImage = secondary.doCheckpoint();

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckPointForSecurityTokens.java

@@ -24,11 +24,11 @@ import static org.junit.Assert.fail;
 import java.io.IOException;
 import java.io.IOException;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.SafeModeAction;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.namenode.FileJournalManager.EditLogFile;
 import org.apache.hadoop.hdfs.server.namenode.FileJournalManager.EditLogFile;
@@ -95,7 +95,7 @@ public class TestCheckPointForSecurityTokens {
       }
       }
 
 
       // Saving image in safe mode should succeed
       // Saving image in safe mode should succeed
-      fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+      fs.setSafeMode(SafeModeAction.ENTER);
       try {
       try {
         admin.run(args);
         admin.run(args);
       } catch(Exception e) {
       } catch(Exception e) {

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java

@@ -1140,6 +1140,7 @@ public class TestCheckpoint {
    * Tests save namespace.
    * Tests save namespace.
    */
    */
   @Test
   @Test
+  @SuppressWarnings("deprecation")
   public void testSaveNamespace() throws IOException {
   public void testSaveNamespace() throws IOException {
     MiniDFSCluster cluster = null;
     MiniDFSCluster cluster = null;
     DistributedFileSystem fs = null;
     DistributedFileSystem fs = null;

+ 25 - 26
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java

@@ -62,6 +62,7 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.SafeModeAction;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSOutputStream;
 import org.apache.hadoop.hdfs.DFSOutputStream;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DFSTestUtil;
@@ -69,8 +70,6 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease;
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
@@ -147,9 +146,9 @@ public class TestFSImage {
           .of(SyncFlag.UPDATE_LENGTH));
           .of(SyncFlag.UPDATE_LENGTH));
 
 
       // checkpoint
       // checkpoint
-      fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+      fs.setSafeMode(SafeModeAction.ENTER);
       fs.saveNamespace();
       fs.saveNamespace();
-      fs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+      fs.setSafeMode(SafeModeAction.LEAVE);
 
 
       cluster.restartNameNode();
       cluster.restartNameNode();
       cluster.waitActive();
       cluster.waitActive();
@@ -363,9 +362,9 @@ public class TestFSImage {
     try {
     try {
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).build();
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).build();
       DistributedFileSystem fs = cluster.getFileSystem();
       DistributedFileSystem fs = cluster.getFileSystem();
-      fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+      fs.setSafeMode(SafeModeAction.ENTER);
       fs.saveNamespace();
       fs.saveNamespace();
-      fs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+      fs.setSafeMode(SafeModeAction.LEAVE);
       File currentDir = FSImageTestUtil.getNameNodeCurrentDirs(cluster, 0).get(
       File currentDir = FSImageTestUtil.getNameNodeCurrentDirs(cluster, 0).get(
           0);
           0);
       File fsimage = FSImageTestUtil.findNewestImageFile(currentDir
       File fsimage = FSImageTestUtil.findNewestImageFile(currentDir
@@ -405,9 +404,9 @@ public class TestFSImage {
       long atimeLink = hdfs.getFileLinkStatus(link).getAccessTime();
       long atimeLink = hdfs.getFileLinkStatus(link).getAccessTime();
 
 
       // save namespace and restart cluster
       // save namespace and restart cluster
-      hdfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER);
+      hdfs.setSafeMode(SafeModeAction.ENTER);
       hdfs.saveNamespace();
       hdfs.saveNamespace();
-      hdfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
+      hdfs.setSafeMode(SafeModeAction.LEAVE);
       cluster.shutdown();
       cluster.shutdown();
       cluster = new MiniDFSCluster.Builder(conf).format(false)
       cluster = new MiniDFSCluster.Builder(conf).format(false)
           .numDataNodes(1).build();
           .numDataNodes(1).build();
@@ -526,9 +525,9 @@ public class TestFSImage {
       DFSTestUtil.writeFile(fs, file_3_2, new String(bytes));
       DFSTestUtil.writeFile(fs, file_3_2, new String(bytes));
 
 
       // Save namespace and restart NameNode
       // Save namespace and restart NameNode
-      fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+      fs.setSafeMode(SafeModeAction.ENTER);
       fs.saveNamespace();
       fs.saveNamespace();
-      fs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+      fs.setSafeMode(SafeModeAction.LEAVE);
 
 
       cluster.restartNameNodes();
       cluster.restartNameNodes();
       fs = cluster.getFileSystem();
       fs = cluster.getFileSystem();
@@ -805,9 +804,9 @@ public class TestFSImage {
           .of(SyncFlag.UPDATE_LENGTH));
           .of(SyncFlag.UPDATE_LENGTH));
 
 
       // checkpoint
       // checkpoint
-      fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+      fs.setSafeMode(SafeModeAction.ENTER);
       fs.saveNamespace();
       fs.saveNamespace();
-      fs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+      fs.setSafeMode(SafeModeAction.LEAVE);
 
 
       cluster.restartNameNode();
       cluster.restartNameNode();
       cluster.waitActive();
       cluster.waitActive();
@@ -864,9 +863,9 @@ public class TestFSImage {
       DFSTestUtil.enableAllECPolicies(fs);
       DFSTestUtil.enableAllECPolicies(fs);
 
 
       // Save namespace and restart NameNode
       // Save namespace and restart NameNode
-      fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+      fs.setSafeMode(SafeModeAction.ENTER);
       fs.saveNamespace();
       fs.saveNamespace();
-      fs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+      fs.setSafeMode(SafeModeAction.LEAVE);
 
 
       cluster.restartNameNodes();
       cluster.restartNameNodes();
       cluster.waitActive();
       cluster.waitActive();
@@ -887,9 +886,9 @@ public class TestFSImage {
       newPolicy = ret[0].getPolicy();
       newPolicy = ret[0].getPolicy();
 
 
       // Save namespace and restart NameNode
       // Save namespace and restart NameNode
-      fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+      fs.setSafeMode(SafeModeAction.ENTER);
       fs.saveNamespace();
       fs.saveNamespace();
-      fs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+      fs.setSafeMode(SafeModeAction.LEAVE);
 
 
       cluster.restartNameNodes();
       cluster.restartNameNodes();
       cluster.waitActive();
       cluster.waitActive();
@@ -935,9 +934,9 @@ public class TestFSImage {
 
 
 
 
     // Save namespace and restart NameNode
     // Save namespace and restart NameNode
-    fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+    fs.setSafeMode(SafeModeAction.ENTER);
     fs.saveNamespace();
     fs.saveNamespace();
-    fs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+    fs.setSafeMode(SafeModeAction.LEAVE);
 
 
     cluster.restartNameNodes();
     cluster.restartNameNodes();
     cluster.waitActive();
     cluster.waitActive();
@@ -957,9 +956,9 @@ public class TestFSImage {
     // 2. Disable an erasure coding policy
     // 2. Disable an erasure coding policy
     fs.disableErasureCodingPolicy(ecPolicy.getName());
     fs.disableErasureCodingPolicy(ecPolicy.getName());
     // Save namespace and restart NameNode
     // Save namespace and restart NameNode
-    fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+    fs.setSafeMode(SafeModeAction.ENTER);
     fs.saveNamespace();
     fs.saveNamespace();
-    fs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+    fs.setSafeMode(SafeModeAction.LEAVE);
 
 
     cluster.restartNameNodes();
     cluster.restartNameNodes();
     cluster.waitActive();
     cluster.waitActive();
@@ -995,9 +994,9 @@ public class TestFSImage {
 
 
     fs.removeErasureCodingPolicy(ecPolicy.getName());
     fs.removeErasureCodingPolicy(ecPolicy.getName());
     // Save namespace and restart NameNode
     // Save namespace and restart NameNode
-    fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+    fs.setSafeMode(SafeModeAction.ENTER);
     fs.saveNamespace();
     fs.saveNamespace();
-    fs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+    fs.setSafeMode(SafeModeAction.LEAVE);
 
 
     cluster.restartNameNodes();
     cluster.restartNameNodes();
     cluster.waitActive();
     cluster.waitActive();
@@ -1059,9 +1058,9 @@ public class TestFSImage {
     }
     }
 
 
     // checkpoint
     // checkpoint
-    fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+    fs.setSafeMode(SafeModeAction.ENTER);
     fs.saveNamespace();
     fs.saveNamespace();
-    fs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+    fs.setSafeMode(SafeModeAction.LEAVE);
 
 
     cluster.restartNameNode();
     cluster.restartNameNode();
     cluster.waitActive();
     cluster.waitActive();
@@ -1202,9 +1201,9 @@ public class TestFSImage {
     SnapshotTestHelper.dumpTree2File(fsdir, preRestartTree);
     SnapshotTestHelper.dumpTree2File(fsdir, preRestartTree);
 
 
     // checkpoint
     // checkpoint
-    fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+    fs.setSafeMode(SafeModeAction.ENTER);
     fs.saveNamespace();
     fs.saveNamespace();
-    fs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+    fs.setSafeMode(SafeModeAction.LEAVE);
 
 
     cluster.restartNameNode();
     cluster.restartNameNode();
     cluster.waitActive();
     cluster.waitActive();

+ 5 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImageWithAcl.java

@@ -27,12 +27,12 @@ import java.util.List;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.SafeModeAction;
 import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.util.Lists;
 import org.apache.hadoop.util.Lists;
 import org.junit.AfterClass;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.Assert;
@@ -80,9 +80,9 @@ public class TestFSImageWithAcl {
     fs.removeAcl(p);
     fs.removeAcl(p);
 
 
     if (persistNamespace) {
     if (persistNamespace) {
-      fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+      fs.setSafeMode(SafeModeAction.ENTER);
       fs.saveNamespace();
       fs.saveNamespace();
-      fs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+      fs.setSafeMode(SafeModeAction.LEAVE);
     }
     }
 
 
     cluster.restartNameNode();
     cluster.restartNameNode();
@@ -247,9 +247,9 @@ public class TestFSImageWithAcl {
   private void restart(DistributedFileSystem fs, boolean persistNamespace)
   private void restart(DistributedFileSystem fs, boolean persistNamespace)
       throws IOException {
       throws IOException {
     if (persistNamespace) {
     if (persistNamespace) {
-      fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+      fs.setSafeMode(SafeModeAction.ENTER);
       fs.saveNamespace();
       fs.saveNamespace();
-      fs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+      fs.setSafeMode(SafeModeAction.LEAVE);
     }
     }
 
 
     cluster.restartNameNode();
     cluster.restartNameNode();

+ 23 - 23
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImageWithSnapshot.java

@@ -21,6 +21,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.SafeModeAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
@@ -28,7 +29,6 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.DiffList;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.DiffList;
@@ -187,9 +187,9 @@ public class TestFSImageWithSnapshot {
     hdfs = cluster.getFileSystem();
     hdfs = cluster.getFileSystem();
     
     
     // save namespace and restart cluster
     // save namespace and restart cluster
-    hdfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+    hdfs.setSafeMode(SafeModeAction.ENTER);
     hdfs.saveNamespace();
     hdfs.saveNamespace();
-    hdfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+    hdfs.setSafeMode(SafeModeAction.LEAVE);
     cluster.shutdown();
     cluster.shutdown();
     cluster = new MiniDFSCluster.Builder(conf).format(false)
     cluster = new MiniDFSCluster.Builder(conf).format(false)
         .numDataNodes(NUM_DATANODES).build();
         .numDataNodes(NUM_DATANODES).build();
@@ -212,9 +212,9 @@ public class TestFSImageWithSnapshot {
     assertEquals(root, sdirs[0].getFullPath());
     assertEquals(root, sdirs[0].getFullPath());
     
     
     // save namespace and restart cluster
     // save namespace and restart cluster
-    hdfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+    hdfs.setSafeMode(SafeModeAction.ENTER);
     hdfs.saveNamespace();
     hdfs.saveNamespace();
-    hdfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+    hdfs.setSafeMode(SafeModeAction.LEAVE);
     cluster.shutdown();
     cluster.shutdown();
     cluster = new MiniDFSCluster.Builder(conf).format(false)
     cluster = new MiniDFSCluster.Builder(conf).format(false)
         .numDataNodes(NUM_DATANODES).build();
         .numDataNodes(NUM_DATANODES).build();
@@ -406,9 +406,9 @@ public class TestFSImageWithSnapshot {
     out.hsync(EnumSet.of(SyncFlag.UPDATE_LENGTH));      
     out.hsync(EnumSet.of(SyncFlag.UPDATE_LENGTH));      
     
     
     // save namespace and restart cluster
     // save namespace and restart cluster
-    hdfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+    hdfs.setSafeMode(SafeModeAction.ENTER);
     hdfs.saveNamespace();
     hdfs.saveNamespace();
-    hdfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+    hdfs.setSafeMode(SafeModeAction.LEAVE);
     
     
     cluster.shutdown();
     cluster.shutdown();
     cluster = new MiniDFSCluster.Builder(conf).format(false)
     cluster = new MiniDFSCluster.Builder(conf).format(false)
@@ -430,9 +430,9 @@ public class TestFSImageWithSnapshot {
     out.close();
     out.close();
     
     
     // save namespace
     // save namespace
-    hdfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+    hdfs.setSafeMode(SafeModeAction.ENTER);
     hdfs.saveNamespace();
     hdfs.saveNamespace();
-    hdfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+    hdfs.setSafeMode(SafeModeAction.LEAVE);
     
     
     // append to the empty file
     // append to the empty file
     out = hdfs.append(file);
     out = hdfs.append(file);
@@ -500,9 +500,9 @@ public class TestFSImageWithSnapshot {
     hdfs = cluster.getFileSystem();
     hdfs = cluster.getFileSystem();
     
     
     // save namespace to fsimage
     // save namespace to fsimage
-    hdfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+    hdfs.setSafeMode(SafeModeAction.ENTER);
     hdfs.saveNamespace();
     hdfs.saveNamespace();
-    hdfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+    hdfs.setSafeMode(SafeModeAction.LEAVE);
     
     
     cluster.shutdown();
     cluster.shutdown();
     cluster = new MiniDFSCluster.Builder(conf).format(false)
     cluster = new MiniDFSCluster.Builder(conf).format(false)
@@ -582,9 +582,9 @@ public class TestFSImageWithSnapshot {
   void restartCluster() throws Exception {
   void restartCluster() throws Exception {
     final File before = dumpTree2File("before.txt");
     final File before = dumpTree2File("before.txt");
 
 
-    hdfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+    hdfs.setSafeMode(SafeModeAction.ENTER);
     hdfs.saveNamespace();
     hdfs.saveNamespace();
-    hdfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+    hdfs.setSafeMode(SafeModeAction.LEAVE);
 
 
     cluster.shutdown();
     cluster.shutdown();
     cluster = new MiniDFSCluster.Builder(conf).format(false)
     cluster = new MiniDFSCluster.Builder(conf).format(false)
@@ -641,9 +641,9 @@ public class TestFSImageWithSnapshot {
     hdfs.delete(renamePath1, true);
     hdfs.delete(renamePath1, true);
     hdfs.deleteSnapshot(dir1, "s1");
     hdfs.deleteSnapshot(dir1, "s1");
     // save namespace and restart cluster
     // save namespace and restart cluster
-    hdfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+    hdfs.setSafeMode(SafeModeAction.ENTER);
     hdfs.saveNamespace();
     hdfs.saveNamespace();
-    hdfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+    hdfs.setSafeMode(SafeModeAction.LEAVE);
     cluster.shutdown();
     cluster.shutdown();
     cluster = new MiniDFSCluster.Builder(conf).format(false)
     cluster = new MiniDFSCluster.Builder(conf).format(false)
             .numDataNodes(NUM_DATANODES).build();
             .numDataNodes(NUM_DATANODES).build();
@@ -686,9 +686,9 @@ public class TestFSImageWithSnapshot {
         new Path("/dir1/.snapshot/s4/dira/dirb/diry/file1")));
         new Path("/dir1/.snapshot/s4/dira/dirb/diry/file1")));
 
 
     // save namespace and restart cluster
     // save namespace and restart cluster
-    hdfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+    hdfs.setSafeMode(SafeModeAction.ENTER);
     hdfs.saveNamespace();
     hdfs.saveNamespace();
-    hdfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+    hdfs.setSafeMode(SafeModeAction.LEAVE);
 
 
     cluster.shutdown();
     cluster.shutdown();
     cluster = new MiniDFSCluster.Builder(conf).format(false)
     cluster = new MiniDFSCluster.Builder(conf).format(false)
@@ -727,9 +727,9 @@ public class TestFSImageWithSnapshot {
     assertTrue(hdfs.exists(file1));
     assertTrue(hdfs.exists(file1));
 
 
     // save namespace and restart cluster
     // save namespace and restart cluster
-    hdfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+    hdfs.setSafeMode(SafeModeAction.ENTER);
     hdfs.saveNamespace();
     hdfs.saveNamespace();
-    hdfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+    hdfs.setSafeMode(SafeModeAction.LEAVE);
 
 
     cluster.shutdown();
     cluster.shutdown();
     cluster = new MiniDFSCluster.Builder(conf).format(false)
     cluster = new MiniDFSCluster.Builder(conf).format(false)
@@ -772,9 +772,9 @@ public class TestFSImageWithSnapshot {
         "/dir1/.snapshot/s4/dira/dirb/diry/file1")));
         "/dir1/.snapshot/s4/dira/dirb/diry/file1")));
 
 
     // save namespace and restart cluster
     // save namespace and restart cluster
-    hdfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+    hdfs.setSafeMode(SafeModeAction.ENTER);
     hdfs.saveNamespace();
     hdfs.saveNamespace();
-    hdfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+    hdfs.setSafeMode(SafeModeAction.LEAVE);
 
 
     cluster.shutdown();
     cluster.shutdown();
     cluster = new MiniDFSCluster.Builder(conf).format(false)
     cluster = new MiniDFSCluster.Builder(conf).format(false)
@@ -817,9 +817,9 @@ public class TestFSImageWithSnapshot {
         new Path("/dir1/.snapshot/s4/dira/dirb/diry/file1")));
         new Path("/dir1/.snapshot/s4/dira/dirb/diry/file1")));
 
 
     // save namespace and restart cluster
     // save namespace and restart cluster
-    hdfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+    hdfs.setSafeMode(SafeModeAction.ENTER);
     hdfs.saveNamespace();
     hdfs.saveNamespace();
-    hdfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+    hdfs.setSafeMode(SafeModeAction.LEAVE);
 
 
     cluster.shutdown();
     cluster.shutdown();
     cluster = new MiniDFSCluster.Builder(conf).format(false)
     cluster = new MiniDFSCluster.Builder(conf).format(false)

+ 4 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImageWithXAttr.java

@@ -24,11 +24,12 @@ import java.util.Map;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.SafeModeAction;
 import org.apache.hadoop.fs.XAttrSetFlag;
 import org.apache.hadoop.fs.XAttrSetFlag;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
+
 import org.junit.AfterClass;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.BeforeClass;
@@ -123,9 +124,9 @@ public class TestFSImageWithXAttr {
   private void restart(DistributedFileSystem fs, boolean persistNamespace)
   private void restart(DistributedFileSystem fs, boolean persistNamespace)
       throws IOException {
       throws IOException {
     if (persistNamespace) {
     if (persistNamespace) {
-      fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+      fs.setSafeMode(SafeModeAction.ENTER);
       fs.saveNamespace();
       fs.saveNamespace();
-      fs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+      fs.setSafeMode(SafeModeAction.LEAVE);
     }
     }
 
 
     cluster.restartNameNode();
     cluster.restartNameNode();

+ 6 - 6
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java

@@ -48,6 +48,7 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FsShell;
 import org.apache.hadoop.fs.FsShell;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.SafeModeAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.AppendTestUtil;
 import org.apache.hadoop.hdfs.AppendTestUtil;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
@@ -57,7 +58,6 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
@@ -954,9 +954,9 @@ public class TestFileTruncate {
   @Test
   @Test
   public void testTruncateEditLogLoad() throws IOException {
   public void testTruncateEditLogLoad() throws IOException {
     // purge previously accumulated edits
     // purge previously accumulated edits
-    fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+    fs.setSafeMode(SafeModeAction.ENTER);
     fs.saveNamespace();
     fs.saveNamespace();
-    fs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+    fs.setSafeMode(SafeModeAction.LEAVE);
 
 
     int startingFileSize = 2 * BLOCK_SIZE + BLOCK_SIZE / 2;
     int startingFileSize = 2 * BLOCK_SIZE + BLOCK_SIZE / 2;
     int toTruncate = 1;
     int toTruncate = 1;
@@ -1052,7 +1052,7 @@ public class TestFileTruncate {
     assertFileLength(snapshotFile, startingFileSize);
     assertFileLength(snapshotFile, startingFileSize);
 
 
     // empty edits and restart
     // empty edits and restart
-    fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+    fs.setSafeMode(SafeModeAction.ENTER);
     fs.saveNamespace();
     fs.saveNamespace();
     cluster.restartNameNode(true);
     cluster.restartNameNode(true);
     assertThat("Total block count should be unchanged from start-up",
     assertThat("Total block count should be unchanged from start-up",
@@ -1247,10 +1247,10 @@ public class TestFileTruncate {
     final DFSAdmin dfsadmin = new DFSAdmin(cluster.getConfiguration(0));
     final DFSAdmin dfsadmin = new DFSAdmin(cluster.getConfiguration(0));
     DistributedFileSystem dfs = cluster.getFileSystem();
     DistributedFileSystem dfs = cluster.getFileSystem();
     //start rolling upgrade
     //start rolling upgrade
-    dfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+    dfs.setSafeMode(SafeModeAction.ENTER);
     int status = dfsadmin.run(new String[]{"-rollingUpgrade", "prepare"});
     int status = dfsadmin.run(new String[]{"-rollingUpgrade", "prepare"});
     assertEquals("could not prepare for rolling upgrade", 0, status);
     assertEquals("could not prepare for rolling upgrade", 0, status);
-    dfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+    dfs.setSafeMode(SafeModeAction.LEAVE);
 
 
     Path dir = new Path("/testTruncateWithRollingUpgrade");
     Path dir = new Path("/testTruncateWithRollingUpgrade");
     fs.mkdirs(dir);
     fs.mkdirs(dir);

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestLeaseManager.java

@@ -24,6 +24,7 @@ import static org.junit.Assert.assertTrue;
 
 
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.SafeModeAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
@@ -31,7 +32,6 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.OpenFilesIterator;
 import org.apache.hadoop.hdfs.protocol.OpenFilesIterator;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
@@ -165,9 +165,9 @@ public class TestLeaseManager {
           file.getFileUnderConstructionFeature().getClientName(), file);
           file.getFileUnderConstructionFeature().getClientName(), file);
 
 
       // Save a fsimage.
       // Save a fsimage.
-      dfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+      dfs.setSafeMode(SafeModeAction.ENTER);
       cluster.getNameNodeRpc().saveNamespace(0,0);
       cluster.getNameNodeRpc().saveNamespace(0,0);
-      dfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+      dfs.setSafeMode(SafeModeAction.LEAVE);
 
 
       // Restart the namenode.
       // Restart the namenode.
       cluster.restartNameNode(true);
       cluster.restartNameNode(true);

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeRetryCacheMetrics.java

@@ -19,8 +19,8 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 
 import java.io.IOException;
 import java.io.IOException;
 
 
+import org.apache.hadoop.fs.SafeModeAction;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
 import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
 import org.apache.hadoop.ipc.metrics.RetryCacheMetrics;
 import org.apache.hadoop.ipc.metrics.RetryCacheMetrics;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
@@ -104,9 +104,9 @@ public class TestNameNodeRetryCacheMetrics {
   }
   }
 
 
   private void trySaveNamespace() throws IOException {
   private void trySaveNamespace() throws IOException {
-    filesystem.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER);
+    filesystem.setSafeMode(SafeModeAction.ENTER);
     filesystem.saveNamespace();
     filesystem.saveNamespace();
-    filesystem.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
+    filesystem.setSafeMode(SafeModeAction.LEAVE);
   }
   }
 
 
 }
 }

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNestedEncryptionZones.java

@@ -24,13 +24,13 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystemTestHelper;
 import org.apache.hadoop.fs.FileSystemTestHelper;
 import org.apache.hadoop.fs.FsShell;
 import org.apache.hadoop.fs.FsShell;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.SafeModeAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.hadoop.util.ToolRunner;
@@ -135,9 +135,9 @@ public class TestNestedEncryptionZones {
 
 
     // Checkpoint and restart NameNode, to test if nested EZs can be loaded
     // Checkpoint and restart NameNode, to test if nested EZs can be loaded
     // from fsimage
     // from fsimage
-    fs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER);
+    fs.setSafeMode(SafeModeAction.ENTER);
     fs.saveNamespace();
     fs.saveNamespace();
-    fs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
+    fs.setSafeMode(SafeModeAction.LEAVE);
     cluster.restartNameNodes();
     cluster.restartNameNodes();
     cluster.waitActive();
     cluster.waitActive();
     fs = cluster.getFileSystem();
     fs = cluster.getFileSystem();

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaByStorageType.java

@@ -22,6 +22,7 @@ import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.SafeModeAction;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DFSTestUtil;
@@ -723,9 +724,9 @@ public class TestQuotaByStorageType {
     assertEquals(file1Len, ssdConsumed);
     assertEquals(file1Len, ssdConsumed);
 
 
     // Restart the namenode with checkpoint to make sure fsImage is correct
     // Restart the namenode with checkpoint to make sure fsImage is correct
-    dfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER);
+    dfs.setSafeMode(SafeModeAction.ENTER);
     dfs.saveNamespace();
     dfs.saveNamespace();
-    dfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
+    dfs.setSafeMode(SafeModeAction.LEAVE);
     cluster.restartNameNode(true);
     cluster.restartNameNode(true);
     refreshClusterState();
     refreshClusterState();
 
 

+ 9 - 9
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestReencryption.java

@@ -41,6 +41,7 @@ import org.apache.hadoop.fs.FileEncryptionInfo;
 import org.apache.hadoop.fs.FileSystemTestHelper;
 import org.apache.hadoop.fs.FileSystemTestHelper;
 import org.apache.hadoop.fs.FileSystemTestWrapper;
 import org.apache.hadoop.fs.FileSystemTestWrapper;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.SafeModeAction;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
@@ -51,7 +52,6 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.client.CreateEncryptionZoneFlag;
 import org.apache.hadoop.hdfs.client.CreateEncryptionZoneFlag;
 import org.apache.hadoop.hdfs.client.HdfsAdmin;
 import org.apache.hadoop.hdfs.client.HdfsAdmin;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.ReencryptAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.ReencryptAction;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.ReencryptionStatus;
 import org.apache.hadoop.hdfs.protocol.ReencryptionStatus;
 import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
 import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
 import org.apache.hadoop.hdfs.protocol.ZoneReencryptionStatus;
 import org.apache.hadoop.hdfs.protocol.ZoneReencryptionStatus;
@@ -538,9 +538,9 @@ public class TestReencryption {
 
 
     final FileEncryptionInfo fei0new = getFileEncryptionInfo(encFile0);
     final FileEncryptionInfo fei0new = getFileEncryptionInfo(encFile0);
     final FileEncryptionInfo fei9new = getFileEncryptionInfo(encFile9);
     final FileEncryptionInfo fei9new = getFileEncryptionInfo(encFile9);
-    fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+    fs.setSafeMode(SafeModeAction.ENTER);
     fs.saveNamespace();
     fs.saveNamespace();
-    fs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+    fs.setSafeMode(SafeModeAction.LEAVE);
     restartClusterDisableReencrypt();
     restartClusterDisableReencrypt();
 
 
     assertKeyVersionEquals(encFile0, fei0new);
     assertKeyVersionEquals(encFile0, fei0new);
@@ -651,9 +651,9 @@ public class TestReencryption {
     dfsAdmin.reencryptEncryptionZone(zone, ReencryptAction.START);
     dfsAdmin.reencryptEncryptionZone(zone, ReencryptAction.START);
     waitForQueuedZones(1);
     waitForQueuedZones(1);
 
 
-    fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+    fs.setSafeMode(SafeModeAction.ENTER);
     fs.saveNamespace();
     fs.saveNamespace();
-    fs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+    fs.setSafeMode(SafeModeAction.LEAVE);
 
 
     // verify after loading from fsimage the command is loaded
     // verify after loading from fsimage the command is loaded
     restartClusterDisableReencrypt();
     restartClusterDisableReencrypt();
@@ -716,9 +716,9 @@ public class TestReencryption {
     }
     }
 
 
     // Verify the same is true after loading from FSImage
     // Verify the same is true after loading from FSImage
-    fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+    fs.setSafeMode(SafeModeAction.ENTER);
     fs.saveNamespace();
     fs.saveNamespace();
-    fs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+    fs.setSafeMode(SafeModeAction.LEAVE);
 
 
     restartClusterDisableReencrypt();
     restartClusterDisableReencrypt();
     waitForQueuedZones(numZones - cancelled.size());
     waitForQueuedZones(numZones - cancelled.size());
@@ -1714,7 +1714,7 @@ public class TestReencryption {
     dfsAdmin.reencryptEncryptionZone(zone, ReencryptAction.START);
     dfsAdmin.reencryptEncryptionZone(zone, ReencryptAction.START);
     waitForReencryptedFiles(zone.toString(), 5);
     waitForReencryptedFiles(zone.toString(), 5);
 
 
-    fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+    fs.setSafeMode(SafeModeAction.ENTER);
     getEzManager().resumeReencryptForTesting();
     getEzManager().resumeReencryptForTesting();
     for (int i = 0; i < 3; ++i) {
     for (int i = 0; i < 3; ++i) {
       Thread.sleep(1000);
       Thread.sleep(1000);
@@ -1727,7 +1727,7 @@ public class TestReencryption {
       assertEquals(5, zs.getFilesReencrypted());
       assertEquals(5, zs.getFilesReencrypted());
     }
     }
 
 
-    fs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+    fs.setSafeMode(SafeModeAction.LEAVE);
     // trigger the background thread to run, without having to
     // trigger the background thread to run, without having to
     // wait for DFS_NAMENODE_REENCRYPT_SLEEP_INTERVAL_KEY
     // wait for DFS_NAMENODE_REENCRYPT_SLEEP_INTERVAL_KEY
     getHandler().notifyNewSubmission();
     getHandler().notifyNewSubmission();

+ 18 - 17
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSaveNamespace.java

@@ -47,6 +47,7 @@ import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.SafeModeAction;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.fs.permission.PermissionStatus;
@@ -55,7 +56,7 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
@@ -212,7 +213,7 @@ public class TestSaveNamespace {
       doAnEdit(fsn, 1);
       doAnEdit(fsn, 1);
 
 
       // Save namespace - this may fail, depending on fault injected
       // Save namespace - this may fail, depending on fault injected
-      fsn.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+      fsn.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER);
       try {
       try {
         fsn.saveNamespace(0, 0);
         fsn.saveNamespace(0, 0);
         if (shouldFail) {
         if (shouldFail) {
@@ -226,7 +227,7 @@ public class TestSaveNamespace {
         }
         }
       }
       }
       
       
-      fsn.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+      fsn.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
       // Should still be able to perform edits
       // Should still be able to perform edits
       doAnEdit(fsn, 2);
       doAnEdit(fsn, 2);
 
 
@@ -281,7 +282,7 @@ public class TestSaveNamespace {
 
 
     try {
     try {
       doAnEdit(fsn, 1);
       doAnEdit(fsn, 1);
-      fsn.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+      fsn.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER);
 
 
       // Save namespace - should mark the first storage dir as faulty
       // Save namespace - should mark the first storage dir as faulty
       // since it's not traversable.
       // since it's not traversable.
@@ -420,7 +421,7 @@ public class TestSaveNamespace {
       doAnEdit(fsn, 1);
       doAnEdit(fsn, 1);
 
 
       // Save namespace
       // Save namespace
-      fsn.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+      fsn.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER);
       try {
       try {
         fsn.saveNamespace(0, 0);
         fsn.saveNamespace(0, 0);
         fail("saveNamespace did not fail even when all directories failed!");
         fail("saveNamespace did not fail even when all directories failed!");
@@ -469,7 +470,7 @@ public class TestSaveNamespace {
       doAnEdit(fsn, 2);
       doAnEdit(fsn, 2);
 
 
       // Save namespace
       // Save namespace
-      fsn.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+      fsn.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER);
       fsn.saveNamespace(0, 0);
       fsn.saveNamespace(0, 0);
 
 
       // Now shut down and restart the NN
       // Now shut down and restart the NN
@@ -503,7 +504,7 @@ public class TestSaveNamespace {
       doAnEdit(fsn, 1);
       doAnEdit(fsn, 1);
       assertEquals(2, fsn.getEditLog().getLastWrittenTxId());
       assertEquals(2, fsn.getEditLog().getLastWrittenTxId());
       
       
-      fsn.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+      fsn.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER);
       fsn.saveNamespace(0, 0);
       fsn.saveNamespace(0, 0);
 
 
       // 2 more txns: END the first segment, BEGIN a new one
       // 2 more txns: END the first segment, BEGIN a new one
@@ -560,7 +561,7 @@ public class TestSaveNamespace {
       final Canceler canceler = new Canceler();
       final Canceler canceler = new Canceler();
       
       
       // Save namespace
       // Save namespace
-      fsn.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+      fsn.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER);
       try {
       try {
         Future<Void> saverFuture = pool.submit(new Callable<Void>() {
         Future<Void> saverFuture = pool.submit(new Callable<Void>() {
           @Override
           @Override
@@ -628,9 +629,9 @@ public class TestSaveNamespace {
       out = fs.create(new Path("/test-source/foo")); // don't close
       out = fs.create(new Path("/test-source/foo")); // don't close
       fs.rename(new Path("/test-source/"), new Path("/test-target/"));
       fs.rename(new Path("/test-source/"), new Path("/test-target/"));
 
 
-      fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+      fs.setSafeMode(SafeModeAction.ENTER);
       cluster.getNameNodeRpc().saveNamespace(0, 0);
       cluster.getNameNodeRpc().saveNamespace(0, 0);
-      fs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+      fs.setSafeMode(SafeModeAction.LEAVE);
     } finally {
     } finally {
       IOUtils.cleanupWithLogger(LOG, out, fs);
       IOUtils.cleanupWithLogger(LOG, out, fs);
       cluster.shutdown();
       cluster.shutdown();
@@ -646,9 +647,9 @@ public class TestSaveNamespace {
     try {
     try {
       cluster.getNamesystem().leaseManager.addLease("me",
       cluster.getNamesystem().leaseManager.addLease("me",
               INodeId.ROOT_INODE_ID + 1);
               INodeId.ROOT_INODE_ID + 1);
-      fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+      fs.setSafeMode(SafeModeAction.ENTER);
       cluster.getNameNodeRpc().saveNamespace(0, 0);
       cluster.getNameNodeRpc().saveNamespace(0, 0);
-      fs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+      fs.setSafeMode(SafeModeAction.LEAVE);
     } finally {
     } finally {
       cluster.shutdown();
       cluster.shutdown();
     }
     }
@@ -678,9 +679,9 @@ public class TestSaveNamespace {
           file.getFileWithSnapshotFeature().getDiffs() != null);
           file.getFileWithSnapshotFeature().getDiffs() != null);
 
 
       // saveNamespace
       // saveNamespace
-      fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+      fs.setSafeMode(SafeModeAction.ENTER);
       cluster.getNameNodeRpc().saveNamespace(0, 0);
       cluster.getNameNodeRpc().saveNamespace(0, 0);
-      fs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+      fs.setSafeMode(SafeModeAction.LEAVE);
 
 
       // restart namenode
       // restart namenode
       cluster.restartNameNode(true);
       cluster.restartNameNode(true);
@@ -708,7 +709,7 @@ public class TestSaveNamespace {
       final FSImage fsimage = cluster.getNameNode().getFSImage();
       final FSImage fsimage = cluster.getNameNode().getFSImage();
       final long before = fsimage.getStorage().getMostRecentCheckpointTxId();
       final long before = fsimage.getStorage().getMostRecentCheckpointTxId();
 
 
-      fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+      fs.setSafeMode(SafeModeAction.ENTER);
       // set the timewindow to 1 hour and tx gap to 1000, which means that if
       // set the timewindow to 1 hour and tx gap to 1000, which means that if
       // there is a checkpoint during the past 1 hour or the tx number happening
       // there is a checkpoint during the past 1 hour or the tx number happening
       // after the latest checkpoint is <= 1000, this saveNamespace request
       // after the latest checkpoint is <= 1000, this saveNamespace request
@@ -723,14 +724,14 @@ public class TestSaveNamespace {
       // do another checkpoint. this time set the timewindow to 1s
       // do another checkpoint. this time set the timewindow to 1s
       // we should see a new checkpoint
       // we should see a new checkpoint
       cluster.getNameNodeRpc().saveNamespace(1, 1000);
       cluster.getNameNodeRpc().saveNamespace(1, 1000);
-      fs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+      fs.setSafeMode(SafeModeAction.LEAVE);
 
 
       after = fsimage.getStorage().getMostRecentCheckpointTxId();
       after = fsimage.getStorage().getMostRecentCheckpointTxId();
       Assert.assertTrue(after > before);
       Assert.assertTrue(after > before);
 
 
       fs.mkdirs(new Path("/foo/bar/baz")); // 3 new tx
       fs.mkdirs(new Path("/foo/bar/baz")); // 3 new tx
 
 
-      fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+      fs.setSafeMode(SafeModeAction.ENTER);
       cluster.getNameNodeRpc().saveNamespace(3600, 5); // 3 + end/start segment
       cluster.getNameNodeRpc().saveNamespace(3600, 5); // 3 + end/start segment
       long after2 = fsimage.getStorage().getMostRecentCheckpointTxId();
       long after2 = fsimage.getStorage().getMostRecentCheckpointTxId();
       // no checkpoint should be made
       // no checkpoint should be made

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHAMetrics.java

@@ -22,12 +22,12 @@ import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.SafeModeAction;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.IOUtils;
 import org.junit.Test;
 import org.junit.Test;
@@ -157,7 +157,7 @@ public class TestHAMetrics {
 
 
       // Save fsimage so that nn does not build up namesystem by replaying
       // Save fsimage so that nn does not build up namesystem by replaying
       // edits, but load from the image.
       // edits, but load from the image.
-      ((DistributedFileSystem)fs).setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+      ((DistributedFileSystem)fs).setSafeMode(SafeModeAction.ENTER);
       ((DistributedFileSystem)fs).saveNamespace();
       ((DistributedFileSystem)fs).saveNamespace();
 
 
       // Flip the two namenodes and restart the standby, which will load
       // Flip the two namenodes and restart the standby, which will load

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestNNHealthCheck.java

@@ -29,12 +29,12 @@ import static org.junit.Assert.fail;
 import java.io.IOException;
 import java.io.IOException;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.SafeModeAction;
 import org.apache.hadoop.ha.HAServiceProtocol;
 import org.apache.hadoop.ha.HAServiceProtocol;
 import org.apache.hadoop.ha.HealthCheckFailedException;
 import org.apache.hadoop.ha.HealthCheckFailedException;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.namenode.MockNameNodeResourceChecker;
 import org.apache.hadoop.hdfs.server.namenode.MockNameNodeResourceChecker;
 import org.apache.hadoop.hdfs.tools.NNHAServiceTarget;
 import org.apache.hadoop.hdfs.tools.NNHAServiceTarget;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.ipc.RemoteException;
@@ -105,7 +105,7 @@ public class TestNNHealthCheck {
 
 
     // manually set safemode.
     // manually set safemode.
     cluster.getFileSystem(0)
     cluster.getFileSystem(0)
-        .setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER);
+        .setSafeMode(SafeModeAction.ENTER);
 
 
     NNHAServiceTarget haTarget = new NNHAServiceTarget(conf,
     NNHAServiceTarget haTarget = new NNHAServiceTarget(conf,
         DFSUtil.getNamenodeNameServiceId(conf), "nn1");
         DFSUtil.getNamenodeNameServiceId(conf), "nn1");

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestObserverNode.java

@@ -48,6 +48,7 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.SafeModeAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.ha.ServiceFailedException;
 import org.apache.hadoop.ha.ServiceFailedException;
@@ -57,7 +58,6 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.qjournal.MiniQJMHACluster;
 import org.apache.hadoop.hdfs.qjournal.MiniQJMHACluster;
@@ -357,7 +357,7 @@ public class TestObserverNode {
     assertSentTo(2);
     assertSentTo(2);
 
 
     // Set observer to safe mode.
     // Set observer to safe mode.
-    dfsCluster.getFileSystem(2).setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+    dfsCluster.getFileSystem(2).setSafeMode(SafeModeAction.ENTER);
 
 
     // Mock block manager for observer to generate some fake blocks which
     // Mock block manager for observer to generate some fake blocks which
     // will trigger the (retriable) safe mode exception.
     // will trigger the (retriable) safe mode exception.
@@ -380,7 +380,7 @@ public class TestObserverNode {
     Mockito.reset(bmSpy);
     Mockito.reset(bmSpy);
 
 
     // Remove safe mode on observer, request should still go to it.
     // Remove safe mode on observer, request should still go to it.
-    dfsCluster.getFileSystem(2).setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+    dfsCluster.getFileSystem(2).setSafeMode(SafeModeAction.LEAVE);
     dfs.open(testPath).close();
     dfs.open(testPath).close();
     assertSentTo(2);
     assertSentTo(2);
   }
   }

+ 17 - 17
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestFSImageWithOrderedSnapshotDeletion.java

@@ -18,9 +18,9 @@
 package org.apache.hadoop.hdfs.server.namenode.snapshot;
 package org.apache.hadoop.hdfs.server.namenode.snapshot;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.SafeModeAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.*;
 import org.apache.hadoop.hdfs.*;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.visitor.NamespacePrintVisitor;
 import org.apache.hadoop.hdfs.server.namenode.visitor.NamespacePrintVisitor;
@@ -163,9 +163,9 @@ public class TestFSImageWithOrderedSnapshotDeletion {
   void restartCluster() throws Exception {
   void restartCluster() throws Exception {
     final File before = dumpTree2File("before.txt");
     final File before = dumpTree2File("before.txt");
 
 
-    hdfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER);
+    hdfs.setSafeMode(SafeModeAction.ENTER);
     hdfs.saveNamespace();
     hdfs.saveNamespace();
-    hdfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
+    hdfs.setSafeMode(SafeModeAction.LEAVE);
 
 
     cluster.shutdown();
     cluster.shutdown();
     cluster = new MiniDFSCluster.Builder(conf).format(false)
     cluster = new MiniDFSCluster.Builder(conf).format(false)
@@ -222,9 +222,9 @@ public class TestFSImageWithOrderedSnapshotDeletion {
     hdfs.delete(renamePath1, true);
     hdfs.delete(renamePath1, true);
     hdfs.deleteSnapshot(dir1, "s1");
     hdfs.deleteSnapshot(dir1, "s1");
     // save namespace and restart cluster
     // save namespace and restart cluster
-    hdfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER);
+    hdfs.setSafeMode(SafeModeAction.ENTER);
     hdfs.saveNamespace();
     hdfs.saveNamespace();
-    hdfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
+    hdfs.setSafeMode(SafeModeAction.LEAVE);
     cluster.shutdown();
     cluster.shutdown();
     cluster = new MiniDFSCluster.Builder(conf).format(false)
     cluster = new MiniDFSCluster.Builder(conf).format(false)
         .numDataNodes(NUM_DATANODES).build();
         .numDataNodes(NUM_DATANODES).build();
@@ -267,9 +267,9 @@ public class TestFSImageWithOrderedSnapshotDeletion {
         new Path("/dir1/.snapshot/s4/dira/dirb/diry/file1")));
         new Path("/dir1/.snapshot/s4/dira/dirb/diry/file1")));
 
 
     // save namespace and restart cluster
     // save namespace and restart cluster
-    hdfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER);
+    hdfs.setSafeMode(SafeModeAction.ENTER);
     hdfs.saveNamespace();
     hdfs.saveNamespace();
-    hdfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
+    hdfs.setSafeMode(SafeModeAction.LEAVE);
 
 
     cluster.shutdown();
     cluster.shutdown();
     cluster = new MiniDFSCluster.Builder(conf).format(false)
     cluster = new MiniDFSCluster.Builder(conf).format(false)
@@ -308,9 +308,9 @@ public class TestFSImageWithOrderedSnapshotDeletion {
     assertTrue(hdfs.exists(file1));
     assertTrue(hdfs.exists(file1));
 
 
     // save namespace and restart cluster
     // save namespace and restart cluster
-    hdfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER);
+    hdfs.setSafeMode(SafeModeAction.ENTER);
     hdfs.saveNamespace();
     hdfs.saveNamespace();
-    hdfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
+    hdfs.setSafeMode(SafeModeAction.LEAVE);
 
 
     cluster.shutdown();
     cluster.shutdown();
     cluster = new MiniDFSCluster.Builder(conf).format(false)
     cluster = new MiniDFSCluster.Builder(conf).format(false)
@@ -353,9 +353,9 @@ public class TestFSImageWithOrderedSnapshotDeletion {
         "/dir1/.snapshot/s4/dira/dirb/diry/file1")));
         "/dir1/.snapshot/s4/dira/dirb/diry/file1")));
 
 
     // save namespace and restart cluster
     // save namespace and restart cluster
-    hdfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER);
+    hdfs.setSafeMode(SafeModeAction.ENTER);
     hdfs.saveNamespace();
     hdfs.saveNamespace();
-    hdfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
+    hdfs.setSafeMode(SafeModeAction.LEAVE);
 
 
     cluster.shutdown();
     cluster.shutdown();
     cluster = new MiniDFSCluster.Builder(conf).format(false)
     cluster = new MiniDFSCluster.Builder(conf).format(false)
@@ -398,9 +398,9 @@ public class TestFSImageWithOrderedSnapshotDeletion {
         new Path("/dir1/.snapshot/s4/dira/dirb/diry/file1")));
         new Path("/dir1/.snapshot/s4/dira/dirb/diry/file1")));
 
 
     // save namespace and restart cluster
     // save namespace and restart cluster
-    hdfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER);
+    hdfs.setSafeMode(SafeModeAction.ENTER);
     hdfs.saveNamespace();
     hdfs.saveNamespace();
-    hdfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
+    hdfs.setSafeMode(SafeModeAction.LEAVE);
 
 
     cluster.shutdown();
     cluster.shutdown();
     cluster = new MiniDFSCluster.Builder(conf).format(false)
     cluster = new MiniDFSCluster.Builder(conf).format(false)
@@ -443,9 +443,9 @@ public class TestFSImageWithOrderedSnapshotDeletion {
     hdfs.delete(renamePath1, true);
     hdfs.delete(renamePath1, true);
     hdfs.deleteSnapshot(dir1, "s1");
     hdfs.deleteSnapshot(dir1, "s1");
     // save namespace and restart cluster
     // save namespace and restart cluster
-    hdfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER);
+    hdfs.setSafeMode(SafeModeAction.ENTER);
     hdfs.saveNamespace();
     hdfs.saveNamespace();
-    hdfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
+    hdfs.setSafeMode(SafeModeAction.LEAVE);
     cluster.shutdown();
     cluster.shutdown();
     cluster = new MiniDFSCluster.Builder(conf).format(false)
     cluster = new MiniDFSCluster.Builder(conf).format(false)
         .numDataNodes(NUM_DATANODES).build();
         .numDataNodes(NUM_DATANODES).build();
@@ -487,9 +487,9 @@ public class TestFSImageWithOrderedSnapshotDeletion {
     deleteSnapshot(sub1, snap6);
     deleteSnapshot(sub1, snap6);
     deleteSnapshot(sub1, snap3);
     deleteSnapshot(sub1, snap3);
     // save namespace and restart Namenode
     // save namespace and restart Namenode
-    hdfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER);
+    hdfs.setSafeMode(SafeModeAction.ENTER);
     hdfs.saveNamespace();
     hdfs.saveNamespace();
-    hdfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
+    hdfs.setSafeMode(SafeModeAction.LEAVE);
     cluster.restartNameNode(true);
     cluster.restartNameNode(true);
   }
   }
 }
 }

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestOrderedSnapshotDeletion.java

@@ -19,13 +19,13 @@ package org.apache.hadoop.hdfs.server.namenode.snapshot;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.SafeModeAction;
 import org.apache.hadoop.fs.XAttr;
 import org.apache.hadoop.fs.XAttr;
 import org.apache.hadoop.fs.XAttrSetFlag;
 import org.apache.hadoop.fs.XAttrSetFlag;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.XAttrHelper;
 import org.apache.hadoop.hdfs.XAttrHelper;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.XAttrFeature;
 import org.apache.hadoop.hdfs.server.namenode.XAttrFeature;
 import org.junit.After;
 import org.junit.After;
@@ -199,9 +199,9 @@ public class TestOrderedSnapshotDeletion {
     hdfs.mkdirs(sub1);
     hdfs.mkdirs(sub1);
     hdfs.createSnapshot(snapshottableDir, "s1");
     hdfs.createSnapshot(snapshottableDir, "s1");
     assertXAttrSet("s1", hdfs, null);
     assertXAttrSet("s1", hdfs, null);
-    hdfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER);
+    hdfs.setSafeMode(SafeModeAction.ENTER);
     hdfs.saveNamespace();
     hdfs.saveNamespace();
-    hdfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
+    hdfs.setSafeMode(SafeModeAction.LEAVE);
     cluster.restartNameNodes();
     cluster.restartNameNodes();
     assertXAttrSet("s1", hdfs, null);
     assertXAttrSet("s1", hdfs, null);
   }
   }

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRandomOpsWithSnapshots.java

@@ -23,11 +23,11 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.SafeModeAction;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.SnapshotException;
 import org.apache.hadoop.hdfs.protocol.SnapshotException;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.GenericTestUtils;
@@ -582,9 +582,9 @@ public class TestRandomOpsWithSnapshots {
     // Randomly decide whether we want to do a check point
     // Randomly decide whether we want to do a check point
     if (generator.nextBoolean()) {
     if (generator.nextBoolean()) {
       LOG.info("checkClusterHealth, doing a checkpoint on NN.");
       LOG.info("checkClusterHealth, doing a checkpoint on NN.");
-      hdfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER);
+      hdfs.setSafeMode(SafeModeAction.ENTER);
       hdfs.saveNamespace();
       hdfs.saveNamespace();
-      hdfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
+      hdfs.setSafeMode(SafeModeAction.LEAVE);
     }
     }
 
 
     /** Restart name node making sure loading from image successfully */
     /** Restart name node making sure loading from image successfully */

+ 7 - 7
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java

@@ -24,11 +24,11 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Options.Rename;
 import org.apache.hadoop.fs.Options.Rename;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.SafeModeAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.*;
 import org.apache.hadoop.hdfs.*;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
@@ -565,9 +565,9 @@ public class TestRenameWithSnapshots {
     SnapshotTestHelper.dumpTree2File(fsdir, fsnMiddle);
     SnapshotTestHelper.dumpTree2File(fsdir, fsnMiddle);
    
    
     // save namespace and restart cluster
     // save namespace and restart cluster
-    hdfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+    hdfs.setSafeMode(SafeModeAction.ENTER);
     hdfs.saveNamespace();
     hdfs.saveNamespace();
-    hdfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+    hdfs.setSafeMode(SafeModeAction.LEAVE);
     cluster.shutdown();
     cluster.shutdown();
     cluster = new MiniDFSCluster.Builder(conf).format(false)
     cluster = new MiniDFSCluster.Builder(conf).format(false)
         .numDataNodes(REPL).build();
         .numDataNodes(REPL).build();
@@ -1811,9 +1811,9 @@ public class TestRenameWithSnapshots {
     // correct. Note that when loading fsimage, foo and bar will be converted 
     // correct. Note that when loading fsimage, foo and bar will be converted 
     // back to normal INodeDirectory and INodeFile since they do not store any 
     // back to normal INodeDirectory and INodeFile since they do not store any 
     // snapshot data
     // snapshot data
-    hdfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+    hdfs.setSafeMode(SafeModeAction.ENTER);
     hdfs.saveNamespace();
     hdfs.saveNamespace();
-    hdfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+    hdfs.setSafeMode(SafeModeAction.LEAVE);
     cluster.shutdown();
     cluster.shutdown();
     cluster = new MiniDFSCluster.Builder(conf).format(false)
     cluster = new MiniDFSCluster.Builder(conf).format(false)
         .numDataNodes(REPL).build();
         .numDataNodes(REPL).build();
@@ -2485,9 +2485,9 @@ public class TestRenameWithSnapshots {
     deleteSnapshot(sub1, snap6);
     deleteSnapshot(sub1, snap6);
     deleteSnapshot(sub1, snap3);
     deleteSnapshot(sub1, snap3);
     // save namespace and restart Namenode
     // save namespace and restart Namenode
-    hdfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+    hdfs.setSafeMode(SafeModeAction.ENTER);
     hdfs.saveNamespace();
     hdfs.saveNamespace();
-    hdfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+    hdfs.setSafeMode(SafeModeAction.LEAVE);
     cluster.restartNameNode(true);
     cluster.restartNameNode(true);
   }
   }
 
 

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshot.java

@@ -40,6 +40,7 @@ import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.SafeModeAction;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
@@ -48,7 +49,6 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
 import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
 import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
@@ -208,9 +208,9 @@ public class TestSnapshot {
     SnapshotTestHelper.dumpTree2File(fsdir, fsnMiddle);
     SnapshotTestHelper.dumpTree2File(fsdir, fsnMiddle);
    
    
     // save namespace and restart cluster
     // save namespace and restart cluster
-    hdfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+    hdfs.setSafeMode(SafeModeAction.ENTER);
     hdfs.saveNamespace();
     hdfs.saveNamespace();
-    hdfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+    hdfs.setSafeMode(SafeModeAction.LEAVE);
     cluster.shutdown();
     cluster.shutdown();
     cluster = new MiniDFSCluster.Builder(conf).format(false)
     cluster = new MiniDFSCluster.Builder(conf).format(false)
         .numDataNodes(REPLICATION).build();
         .numDataNodes(REPLICATION).build();

+ 13 - 13
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java

@@ -32,6 +32,7 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FsShell;
 import org.apache.hadoop.fs.FsShell;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.SafeModeAction;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
@@ -40,7 +41,6 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
@@ -167,9 +167,9 @@ public class TestSnapshotDeletion {
         .getNumSnapshottableDirs());
         .getNumSnapshottableDirs());
     assertEquals(0, cluster.getNamesystem().getSnapshotManager()
     assertEquals(0, cluster.getNamesystem().getSnapshotManager()
         .getSnapshottableDirs().size());
         .getSnapshottableDirs().size());
-    hdfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+    hdfs.setSafeMode(SafeModeAction.ENTER);
     hdfs.saveNamespace();
     hdfs.saveNamespace();
-    hdfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+    hdfs.setSafeMode(SafeModeAction.LEAVE);
     cluster.restartNameNode(0);
     cluster.restartNameNode(0);
   }
   }
 
 
@@ -1143,10 +1143,10 @@ public class TestSnapshotDeletion {
     hdfs.allowSnapshot(foo);
     hdfs.allowSnapshot(foo);
 
 
     hdfs.createSnapshot(foo, snapshotName);
     hdfs.createSnapshot(foo, snapshotName);
-    hdfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+    hdfs.setSafeMode(SafeModeAction.ENTER);
     hdfs.saveNamespace();
     hdfs.saveNamespace();
 
 
-    hdfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+    hdfs.setSafeMode(SafeModeAction.LEAVE);
     hdfs.deleteSnapshot(foo, snapshotName);
     hdfs.deleteSnapshot(foo, snapshotName);
     hdfs.delete(bar, true);
     hdfs.delete(bar, true);
     hdfs.delete(foo, true);
     hdfs.delete(foo, true);
@@ -1188,9 +1188,9 @@ public class TestSnapshotDeletion {
     hdfs.deleteSnapshot(st, "s1");
     hdfs.deleteSnapshot(st, "s1");
     hdfs.deleteSnapshot(st, "s2");
     hdfs.deleteSnapshot(st, "s2");
 
 
-    hdfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+    hdfs.setSafeMode(SafeModeAction.ENTER);
     hdfs.saveNamespace();
     hdfs.saveNamespace();
-    hdfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+    hdfs.setSafeMode(SafeModeAction.LEAVE);
 
 
     cluster.restartNameNodes();
     cluster.restartNameNodes();
   }
   }
@@ -1273,9 +1273,9 @@ public class TestSnapshotDeletion {
     exception.expectMessage(error);
     exception.expectMessage(error);
     hdfs.concat(dest, files);
     hdfs.concat(dest, files);
 
 
-    hdfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+    hdfs.setSafeMode(SafeModeAction.ENTER);
     hdfs.saveNamespace();
     hdfs.saveNamespace();
-    hdfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+    hdfs.setSafeMode(SafeModeAction.LEAVE);
 
 
     cluster.restartNameNodes();
     cluster.restartNameNodes();
   }
   }
@@ -1310,9 +1310,9 @@ public class TestSnapshotDeletion {
 
 
     hdfs.deleteSnapshot(st, "s1");
     hdfs.deleteSnapshot(st, "s1");
 
 
-    hdfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+    hdfs.setSafeMode(SafeModeAction.ENTER);
     hdfs.saveNamespace();
     hdfs.saveNamespace();
-    hdfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+    hdfs.setSafeMode(SafeModeAction.LEAVE);
 
 
     cluster.restartNameNodes();
     cluster.restartNameNodes();
   }
   }
@@ -1354,9 +1354,9 @@ public class TestSnapshotDeletion {
 
 
     hdfs.deleteSnapshot(st, "s1");
     hdfs.deleteSnapshot(st, "s1");
 
 
-    hdfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+    hdfs.setSafeMode(SafeModeAction.ENTER);
     hdfs.saveNamespace();
     hdfs.saveNamespace();
-    hdfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+    hdfs.setSafeMode(SafeModeAction.LEAVE);
 
 
     cluster.restartNameNodes();
     cluster.restartNameNodes();
   }
   }

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestXAttrWithSnapshot.java

@@ -28,12 +28,12 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FsShell;
 import org.apache.hadoop.fs.FsShell;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.SafeModeAction;
 import org.apache.hadoop.fs.XAttrSetFlag;
 import org.apache.hadoop.fs.XAttrSetFlag;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
 import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
@@ -155,9 +155,9 @@ public class TestXAttrWithSnapshot {
         hdfs.getSnapshotDiffReport(path, snapshotName, "");
         hdfs.getSnapshotDiffReport(path, snapshotName, "");
     System.out.println(report);
     System.out.println(report);
     Assert.assertEquals(0, report.getDiffList().size());
     Assert.assertEquals(0, report.getDiffList().size());
-    hdfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER);
+    hdfs.setSafeMode(SafeModeAction.ENTER);
     hdfs.saveNamespace();
     hdfs.saveNamespace();
-    hdfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
+    hdfs.setSafeMode(SafeModeAction.LEAVE);
     cluster.restartNameNode(true);
     cluster.restartNameNode(true);
     report =
     report =
         hdfs.getSnapshotDiffReport(path, snapshotName, "");
         hdfs.getSnapshotDiffReport(path, snapshotName, "");

+ 4 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java

@@ -65,6 +65,7 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystemTestHelper;
 import org.apache.hadoop.fs.FileSystemTestHelper;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.SafeModeAction;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.fs.permission.PermissionStatus;
@@ -76,7 +77,6 @@ import org.apache.hadoop.hdfs.protocol.AddErasureCodingPolicyResponse;
 import org.apache.hadoop.hdfs.protocol.BlockType;
 import org.apache.hadoop.hdfs.protocol.BlockType;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicyState;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicyState;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
 import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
 import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
 import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
@@ -358,9 +358,9 @@ public class TestOfflineImageViewer {
       filesECCount++;
       filesECCount++;
 
 
       // Write results to the fsimage file
       // Write results to the fsimage file
-      hdfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER, false);
+      hdfs.setSafeMode(SafeModeAction.ENTER, false);
       hdfs.saveNamespace();
       hdfs.saveNamespace();
-      hdfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE, false);
+      hdfs.setSafeMode(SafeModeAction.LEAVE, false);
 
 
       // Determine location of fsimage file
       // Determine location of fsimage file
       originalFsimage = FSImageTestUtil.findLatestImageFile(FSImageTestUtil
       originalFsimage = FSImageTestUtil.findLatestImageFile(FSImageTestUtil
@@ -1285,7 +1285,7 @@ public class TestOfflineImageViewer {
       }
       }
 
 
       // Write results to the fsimage file
       // Write results to the fsimage file
-      hdfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER, false);
+      hdfs.setSafeMode(SafeModeAction.ENTER, false);
       hdfs.saveNamespace();
       hdfs.saveNamespace();
       // Determine location of fsimage file
       // Determine location of fsimage file
       fsimageFile =
       fsimageFile =

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerForAcl.java

@@ -38,11 +38,11 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.SafeModeAction;
 import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.fs.permission.AclStatus;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
 import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
 import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
 import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetUtils;
@@ -147,7 +147,7 @@ public class TestOfflineImageViewerForAcl {
       writtenAcls.put(file.toString(), hdfs.getAclStatus(file));
       writtenAcls.put(file.toString(), hdfs.getAclStatus(file));
 
 
       // Write results to the fsimage file
       // Write results to the fsimage file
-      hdfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER, false);
+      hdfs.setSafeMode(SafeModeAction.ENTER, false);
       hdfs.saveNamespace();
       hdfs.saveNamespace();
 
 
       // Determine the location of the fsimage file
       // Determine the location of the fsimage file

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerForContentSummary.java

@@ -30,9 +30,9 @@ import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.SafeModeAction;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
 import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
 import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
 import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetUtils;
@@ -102,7 +102,7 @@ public class TestOfflineImageViewerForContentSummary {
       symLinkSummaryForDirContainsFromDFS = hdfs.getContentSummary(new Path(
       symLinkSummaryForDirContainsFromDFS = hdfs.getContentSummary(new Path(
           "/dirForLinks"));
           "/dirForLinks"));
       // Write results to the fsimage file
       // Write results to the fsimage file
-      hdfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER, false);
+      hdfs.setSafeMode(SafeModeAction.ENTER, false);
       hdfs.saveNamespace();
       hdfs.saveNamespace();
       // Determine the location of the fsimage file
       // Determine the location of the fsimage file
       originalFsimage = FSImageTestUtil.findLatestImageFile(FSImageTestUtil
       originalFsimage = FSImageTestUtil.findLatestImageFile(FSImageTestUtil

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerForErasureCodingPolicy.java

@@ -20,11 +20,11 @@ package org.apache.hadoop.hdfs.tools.offlineImageViewer;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.SafeModeAction;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
 import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
 import org.junit.AfterClass;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.BeforeClass;
@@ -120,7 +120,7 @@ public class TestOfflineImageViewerForErasureCodingPolicy {
       }
       }
 
 
       // Write results to the fsimage file
       // Write results to the fsimage file
-      hdfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER, false);
+      hdfs.setSafeMode(SafeModeAction.ENTER, false);
       hdfs.saveNamespace();
       hdfs.saveNamespace();
 
 
       // Determine the location of the fsimage file
       // Determine the location of the fsimage file

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerForStoragePolicy.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.tools.offlineImageViewer;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.SafeModeAction;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
@@ -116,7 +117,7 @@ public class TestOfflineImageViewerForStoragePolicy {
       hdfs.setStoragePolicy(dir, HdfsConstants.HOT_STORAGE_POLICY_NAME);
       hdfs.setStoragePolicy(dir, HdfsConstants.HOT_STORAGE_POLICY_NAME);
 
 
       // Write results to the fsimage file
       // Write results to the fsimage file
-      hdfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER, false);
+      hdfs.setSafeMode(SafeModeAction.ENTER, false);
       hdfs.saveNamespace();
       hdfs.saveNamespace();
 
 
       // Determine the location of the fsimage file
       // Determine the location of the fsimage file

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerForXAttr.java

@@ -36,11 +36,11 @@ import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.SafeModeAction;
 import org.apache.hadoop.fs.XAttr;
 import org.apache.hadoop.fs.XAttr;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.XAttrHelper;
 import org.apache.hadoop.hdfs.XAttrHelper;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
 import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
 import org.apache.hadoop.hdfs.web.JsonUtil;
 import org.apache.hadoop.hdfs.web.JsonUtil;
 import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
 import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
@@ -81,7 +81,7 @@ public class TestOfflineImageViewerForXAttr {
       hdfs.setXAttr(dir, "user.attr1", "value1".getBytes());
       hdfs.setXAttr(dir, "user.attr1", "value1".getBytes());
       hdfs.setXAttr(dir, "user.attr2", "value2".getBytes());
       hdfs.setXAttr(dir, "user.attr2", "value2".getBytes());
       // Write results to the fsimage file
       // Write results to the fsimage file
-      hdfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER, false);
+      hdfs.setSafeMode(SafeModeAction.ENTER, false);
       hdfs.saveNamespace();
       hdfs.saveNamespace();
 
 
       List<XAttr> attributes = new ArrayList<XAttr>();
       List<XAttr> attributes = new ArrayList<XAttr>();

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerWithStripedBlocks.java

@@ -26,6 +26,7 @@ import java.io.IOException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.SafeModeAction;
 import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DFSTestUtil;
@@ -33,7 +34,6 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.StripedFileTestUtil;
 import org.apache.hadoop.hdfs.StripedFileTestUtil;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
 import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
@@ -121,7 +121,7 @@ public class TestOfflineImageViewerWithStripedBlocks {
 
 
   private void testFileSize(int numBytes) throws IOException,
   private void testFileSize(int numBytes) throws IOException,
       UnresolvedLinkException, SnapshotAccessControlException {
       UnresolvedLinkException, SnapshotAccessControlException {
-    fs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+    fs.setSafeMode(SafeModeAction.LEAVE);
     File orgFsimage = null;
     File orgFsimage = null;
     Path file = new Path("/eczone/striped");
     Path file = new Path("/eczone/striped");
     FSDataOutputStream out = fs.create(file, true);
     FSDataOutputStream out = fs.create(file, true);
@@ -130,7 +130,7 @@ public class TestOfflineImageViewerWithStripedBlocks {
     out.close();
     out.close();
 
 
     // Write results to the fsimage file
     // Write results to the fsimage file
-    fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER, false);
+    fs.setSafeMode(SafeModeAction.ENTER, false);
     fs.saveNamespace();
     fs.saveNamespace();
 
 
     // Determine location of fsimage file
     // Determine location of fsimage file

+ 10 - 10
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/TestHistoryFileManager.java

@@ -32,11 +32,11 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.SafeModeAction;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.mapreduce.JobID;
 import org.apache.hadoop.mapreduce.JobID;
 import org.apache.hadoop.mapreduce.TypeConverter;
 import org.apache.hadoop.mapreduce.TypeConverter;
 import org.apache.hadoop.mapreduce.v2.hs.HistoryFileManager.HistoryFileInfo;
 import org.apache.hadoop.mapreduce.v2.hs.HistoryFileManager.HistoryFileInfo;
@@ -87,9 +87,9 @@ public class TestHistoryFileManager {
   public void cleanTest() throws Exception {
   public void cleanTest() throws Exception {
     new File(coreSitePath).delete();
     new File(coreSitePath).delete();
     dfsCluster.getFileSystem().setSafeMode(
     dfsCluster.getFileSystem().setSafeMode(
-        HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
+        SafeModeAction.LEAVE);
     dfsCluster2.getFileSystem().setSafeMode(
     dfsCluster2.getFileSystem().setSafeMode(
-        HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
+        SafeModeAction.LEAVE);
   }
   }
 
 
   private String getDoneDirNameForTest() {
   private String getDoneDirNameForTest() {
@@ -119,7 +119,7 @@ public class TestHistoryFileManager {
   @Test
   @Test
   public void testCreateDirsWithFileSystem() throws Exception {
   public void testCreateDirsWithFileSystem() throws Exception {
     dfsCluster.getFileSystem().setSafeMode(
     dfsCluster.getFileSystem().setSafeMode(
-        HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
+        SafeModeAction.LEAVE);
     Assert.assertFalse(dfsCluster.getFileSystem().isInSafeMode());
     Assert.assertFalse(dfsCluster.getFileSystem().isInSafeMode());
     testTryCreateHistoryDirs(dfsCluster.getConfiguration(0), true);
     testTryCreateHistoryDirs(dfsCluster.getConfiguration(0), true);
   }
   }
@@ -127,9 +127,9 @@ public class TestHistoryFileManager {
   @Test
   @Test
   public void testCreateDirsWithAdditionalFileSystem() throws Exception {
   public void testCreateDirsWithAdditionalFileSystem() throws Exception {
     dfsCluster.getFileSystem().setSafeMode(
     dfsCluster.getFileSystem().setSafeMode(
-        HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
+        SafeModeAction.LEAVE);
     dfsCluster2.getFileSystem().setSafeMode(
     dfsCluster2.getFileSystem().setSafeMode(
-        HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
+        SafeModeAction.LEAVE);
     Assert.assertFalse(dfsCluster.getFileSystem().isInSafeMode());
     Assert.assertFalse(dfsCluster.getFileSystem().isInSafeMode());
     Assert.assertFalse(dfsCluster2.getFileSystem().isInSafeMode());
     Assert.assertFalse(dfsCluster2.getFileSystem().isInSafeMode());
 
 
@@ -157,7 +157,7 @@ public class TestHistoryFileManager {
   @Test
   @Test
   public void testCreateDirsWithFileSystemInSafeMode() throws Exception {
   public void testCreateDirsWithFileSystemInSafeMode() throws Exception {
     dfsCluster.getFileSystem().setSafeMode(
     dfsCluster.getFileSystem().setSafeMode(
-        HdfsConstants.SafeModeAction.SAFEMODE_ENTER);
+        SafeModeAction.ENTER);
     Assert.assertTrue(dfsCluster.getFileSystem().isInSafeMode());
     Assert.assertTrue(dfsCluster.getFileSystem().isInSafeMode());
     testTryCreateHistoryDirs(dfsCluster.getConfiguration(0), false);
     testTryCreateHistoryDirs(dfsCluster.getConfiguration(0), false);
   }
   }
@@ -175,7 +175,7 @@ public class TestHistoryFileManager {
   public void testCreateDirsWithFileSystemBecomingAvailBeforeTimeout()
   public void testCreateDirsWithFileSystemBecomingAvailBeforeTimeout()
       throws Exception {
       throws Exception {
     dfsCluster.getFileSystem().setSafeMode(
     dfsCluster.getFileSystem().setSafeMode(
-        HdfsConstants.SafeModeAction.SAFEMODE_ENTER);
+        SafeModeAction.ENTER);
     Assert.assertTrue(dfsCluster.getFileSystem().isInSafeMode());
     Assert.assertTrue(dfsCluster.getFileSystem().isInSafeMode());
     new Thread() {
     new Thread() {
       @Override
       @Override
@@ -183,7 +183,7 @@ public class TestHistoryFileManager {
         try {
         try {
           Thread.sleep(500);
           Thread.sleep(500);
           dfsCluster.getFileSystem().setSafeMode(
           dfsCluster.getFileSystem().setSafeMode(
-              HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
+              SafeModeAction.LEAVE);
           Assert.assertTrue(dfsCluster.getFileSystem().isInSafeMode());
           Assert.assertTrue(dfsCluster.getFileSystem().isInSafeMode());
         } catch (Exception ex) {
         } catch (Exception ex) {
           Assert.fail(ex.toString());
           Assert.fail(ex.toString());
@@ -198,7 +198,7 @@ public class TestHistoryFileManager {
   public void testCreateDirsWithFileSystemNotBecomingAvailBeforeTimeout()
   public void testCreateDirsWithFileSystemNotBecomingAvailBeforeTimeout()
       throws Exception {
       throws Exception {
     dfsCluster.getFileSystem().setSafeMode(
     dfsCluster.getFileSystem().setSafeMode(
-        HdfsConstants.SafeModeAction.SAFEMODE_ENTER);
+        SafeModeAction.ENTER);
     Assert.assertTrue(dfsCluster.getFileSystem().isInSafeMode());
     Assert.assertTrue(dfsCluster.getFileSystem().isInSafeMode());
     final ControlledClock clock = new ControlledClock();
     final ControlledClock clock = new ControlledClock();
     clock.setTime(1);
     clock.setTime(1);

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/TestHistoryFileManagerInitWithNonRunningDFS.java

@@ -19,10 +19,10 @@
 package org.apache.hadoop.mapreduce.v2.hs;
 package org.apache.hadoop.mapreduce.v2.hs;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.SafeModeAction;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 
 
 import org.junit.Assert;
 import org.junit.Assert;
 import org.junit.Test;
 import org.junit.Test;
@@ -57,7 +57,7 @@ public class TestHistoryFileManagerInitWithNonRunningDFS {
     try {
     try {
       // set up a cluster with its name node in safe mode
       // set up a cluster with its name node in safe mode
       dfsCluster.getFileSystem().setSafeMode(
       dfsCluster.getFileSystem().setSafeMode(
-          HdfsConstants.SafeModeAction.SAFEMODE_ENTER);
+          SafeModeAction.ENTER);
       Assert.assertTrue(dfsCluster.getFileSystem().isInSafeMode());
       Assert.assertTrue(dfsCluster.getFileSystem().isInSafeMode());
 
 
       HistoryFileManager hfm = new HistoryFileManager();
       HistoryFileManager hfm = new HistoryFileManager();