You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by we...@apache.org on 2021/04/02 06:07:01 UTC

[hadoop] branch branch-3.3 updated (90bbaca -> 57d2fff)

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

weichiu pushed a change to branch branch-3.3
in repository https://gitbox.apache.org/repos/asf/hadoop.git.


    from 90bbaca  HADOOP-17587. Kinit with keytab should not display the keytab file's full path in any logs. Contributed by  Ravuri Sushma sree.
     new 2b207ea  HDFS-15667. Audit log record the unexpected allowed result when delete (#2437)
     new 5187bd3  HDFS-15266. Add missing DFSOps Statistics in WebHDFS. Contributed by Ayush Saxena.
     new 41fad50  HDFS-15265. HttpFS: validate content-type in HttpFSUtils. Contributed by hemanthboyina.
     new 1148d38  HDFS-15332. Quota Space consumed was wrong in truncate with Snapshots. Contributed by hemanthboyina.
     new 57d2fff  HDFS-15246. ArrayIndexOfboundsException in BlockManager CreateLocatedBlock. Contributed by Hemanth Boyina.

The 5 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../apache/hadoop/hdfs/DistributedFileSystem.java  | 13 +++-
 .../apache/hadoop/hdfs/web/WebHdfsFileSystem.java  |  8 +++
 .../apache/hadoop/fs/http/client/HttpFSUtils.java  | 13 +++-
 .../hadoop/fs/http/server/TestHttpFSServer.java    | 31 +++++++++
 .../hadoop/hdfs/server/namenode/FSDirRenameOp.java |  7 +-
 .../hadoop/hdfs/server/namenode/FSNamesystem.java  |  2 +-
 .../namenode/snapshot/FileWithSnapshotFeature.java |  6 ++
 .../hadoop/hdfs/TestDistributedFileSystem.java     | 21 +++++-
 .../namenode/TestAuditLoggerWithCommands.java      | 13 ++++
 .../hdfs/server/namenode/TestFileTruncate.java     | 75 ++++++++++++++++++++++
 .../org/apache/hadoop/hdfs/web/TestWebHDFS.java    | 60 +++++++++++++++++
 11 files changed, 240 insertions(+), 9 deletions(-)

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


[hadoop] 05/05: HDFS-15246. ArrayIndexOfboundsException in BlockManager CreateLocatedBlock. Contributed by Hemanth Boyina.

Posted by we...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

weichiu pushed a commit to branch branch-3.3
in repository https://gitbox.apache.org/repos/asf/hadoop.git

commit 57d2fff84bdf52db29ce44b1cb989b363fcaa8c3
Author: Inigo Goiri <in...@apache.org>
AuthorDate: Tue Jul 21 09:00:07 2020 -0700

    HDFS-15246. ArrayIndexOfboundsException in BlockManager CreateLocatedBlock. Contributed by Hemanth Boyina.
    
    (cherry picked from commit 8b7695bb2628574b4450bac19c12b29db9ee0628)
---
 .../hadoop/hdfs/server/namenode/FSDirRenameOp.java |  7 ++++-
 .../hdfs/server/namenode/TestFileTruncate.java     | 35 ++++++++++++++++++++++
 2 files changed, 41 insertions(+), 1 deletion(-)

diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java
index 2fd2523..c60acaa 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java
@@ -655,7 +655,12 @@ class FSDirRenameOp {
       // snapshot is taken on the dst tree, changes will be recorded in the
       // latest snapshot of the src tree.
       if (isSrcInSnapshot) {
-        srcChild.recordModification(srcLatestSnapshotId);
+        if (srcChild.isFile()) {
+          INodeFile file = srcChild.asFile();
+          file.recordModification(srcLatestSnapshotId, true);
+        } else {
+          srcChild.recordModification(srcLatestSnapshotId);
+        }
       }
 
       // check srcChild for reference
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
index 335772b..5c75abb 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
@@ -40,6 +40,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
@@ -70,6 +71,7 @@ import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.ToolRunner;
 import org.slf4j.event.Level;
 import org.junit.After;
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -1460,4 +1462,37 @@ public class TestFileTruncate {
         fs.getQuotaUsage(root).getSpaceConsumed());
 
   }
+
+  /**
+   * Test truncate on a snapshotted file.
+   */
+  @Test
+  public void testTruncatewithRenameandSnapshot() throws Exception {
+    final Path dir = new Path("/dir");
+    fs.mkdirs(dir, new FsPermission((short) 0777));
+    final Path file = new Path(dir, "file");
+    final Path movedFile = new Path("/file");
+
+    // 1. create a file and snapshot for dir which is having a file
+    DFSTestUtil.createFile(fs, file, 10, (short) 3, 0);
+    fs.allowSnapshot(dir);
+    Path snapshotPath = fs.createSnapshot(dir, "s0");
+    assertTrue(fs.exists(snapshotPath));
+
+    // 2. move the file
+    fs.rename(file, new Path("/"));
+
+    // 3.truncate the moved file
+    final boolean isReady = fs.truncate(movedFile, 5);
+    if (!isReady) {
+      checkBlockRecovery(movedFile);
+    }
+    FileStatus fileStatus = fs.getFileStatus(movedFile);
+    assertEquals(5, fileStatus.getLen());
+
+    // 4. get block locations of file which is in snapshot
+    LocatedBlocks locations = fs.getClient().getNamenode()
+        .getBlockLocations("/dir/.snapshot/s0/file", 0, 10);
+    assertEquals(10, locations.get(0).getBlockSize());
+  }
 }

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


[hadoop] 02/05: HDFS-15266. Add missing DFSOps Statistics in WebHDFS. Contributed by Ayush Saxena.

Posted by we...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

weichiu pushed a commit to branch branch-3.3
in repository https://gitbox.apache.org/repos/asf/hadoop.git

commit 5187bd37ae9c38dc55bb1e0451064a8f191cfca0
Author: Ayush Saxena <ay...@apache.org>
AuthorDate: Fri Apr 17 00:11:02 2020 +0530

    HDFS-15266. Add missing DFSOps Statistics in WebHDFS. Contributed by Ayush Saxena.
    
    (cherry picked from commit 37d65822235fe8285d10232589aba39ededd3be1)
---
 .../apache/hadoop/hdfs/DistributedFileSystem.java  | 13 +++--
 .../apache/hadoop/hdfs/web/WebHdfsFileSystem.java  |  8 +++
 .../hadoop/hdfs/TestDistributedFileSystem.java     | 21 ++++++--
 .../org/apache/hadoop/hdfs/web/TestWebHDFS.java    | 60 ++++++++++++++++++++++
 4 files changed, 96 insertions(+), 6 deletions(-)

diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
index 53655d1..d585bbc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
@@ -819,6 +819,8 @@ public class DistributedFileSystem extends FileSystem
   @Override
   public Collection<BlockStoragePolicy> getAllStoragePolicies()
       throws IOException {
+    statistics.incrementReadOps(1);
+    storageStatistics.incrementOpCounter(OpType.GET_STORAGE_POLICIES);
     return Arrays.asList(dfs.getStoragePolicies());
   }
 
@@ -840,9 +842,7 @@ public class DistributedFileSystem extends FileSystem
    */
   @Deprecated
   public BlockStoragePolicy[] getStoragePolicies() throws IOException {
-    statistics.incrementReadOps(1);
-    storageStatistics.incrementOpCounter(OpType.GET_STORAGE_POLICIES);
-    return dfs.getStoragePolicies();
+    return getAllStoragePolicies().toArray(new BlockStoragePolicy[0]);
   }
 
   /**
@@ -2162,6 +2162,9 @@ public class DistributedFileSystem extends FileSystem
    */
   public SnapshottableDirectoryStatus[] getSnapshottableDirListing()
       throws IOException {
+    statistics.incrementReadOps(1);
+    storageStatistics
+        .incrementOpCounter(OpType.GET_SNAPSHOTTABLE_DIRECTORY_LIST);
     return dfs.getSnapshottableDirListing();
   }
 
@@ -2334,6 +2337,8 @@ public class DistributedFileSystem extends FileSystem
    */
   public SnapshotDiffReport getSnapshotDiffReport(final Path snapshotDir,
       final String fromSnapshot, final String toSnapshot) throws IOException {
+    statistics.incrementReadOps(1);
+    storageStatistics.incrementOpCounter(OpType.GET_SNAPSHOT_DIFF);
     Path absF = fixRelativePart(snapshotDir);
     return new FileSystemLinkResolver<SnapshotDiffReport>() {
       @Override
@@ -3282,6 +3287,8 @@ public class DistributedFileSystem extends FileSystem
    */
   @Override
   public Path getTrashRoot(Path path) {
+    statistics.incrementReadOps(1);
+    storageStatistics.incrementOpCounter(OpType.GET_TRASH_ROOT);
     try {
       if ((path == null) || !dfs.isHDFSEncryptionEnabled()) {
         return super.getTrashRoot(path);
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
index ca59cf3..7a0857e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
@@ -1341,6 +1341,8 @@ public class WebHdfsFileSystem extends FileSystem
 
   @Override
   public void satisfyStoragePolicy(final Path p) throws IOException {
+    statistics.incrementWriteOps(1);
+    storageStatistics.incrementOpCounter(OpType.SATISFY_STORAGE_POLICY);
     final HttpOpParam.Op op = PutOpParam.Op.SATISFYSTORAGEPOLICY;
     new FsPathRunner(op, p).run();
   }
@@ -1430,6 +1432,7 @@ public class WebHdfsFileSystem extends FileSystem
 
   public SnapshotDiffReport getSnapshotDiffReport(final Path snapshotDir,
       final String fromSnapshot, final String toSnapshot) throws IOException {
+    statistics.incrementReadOps(1);
     storageStatistics.incrementOpCounter(OpType.GET_SNAPSHOT_DIFF);
     final HttpOpParam.Op op = GetOpParam.Op.GETSNAPSHOTDIFF;
     return new FsPathResponseRunner<SnapshotDiffReport>(op, snapshotDir,
@@ -1444,6 +1447,7 @@ public class WebHdfsFileSystem extends FileSystem
 
   public SnapshottableDirectoryStatus[] getSnapshottableDirectoryList()
       throws IOException {
+    statistics.incrementReadOps(1);
     storageStatistics
         .incrementOpCounter(OpType.GET_SNAPSHOTTABLE_DIRECTORY_LIST);
     final HttpOpParam.Op op = GetOpParam.Op.GETSNAPSHOTTABLEDIRECTORYLIST;
@@ -2005,6 +2009,8 @@ public class WebHdfsFileSystem extends FileSystem
   @Override
   public Collection<BlockStoragePolicy> getAllStoragePolicies()
       throws IOException {
+    statistics.incrementReadOps(1);
+    storageStatistics.incrementOpCounter(OpType.GET_STORAGE_POLICIES);
     final HttpOpParam.Op op = GetOpParam.Op.GETALLSTORAGEPOLICY;
     return new FsPathResponseRunner<Collection<BlockStoragePolicy>>(op, null) {
       @Override
@@ -2017,6 +2023,8 @@ public class WebHdfsFileSystem extends FileSystem
 
   @Override
   public BlockStoragePolicy getStoragePolicy(Path src) throws IOException {
+    statistics.incrementReadOps(1);
+    storageStatistics.incrementOpCounter(OpType.GET_STORAGE_POLICY);
     final HttpOpParam.Op op = GetOpParam.Op.GETSTORAGEPOLICY;
     return new FsPathResponseRunner<BlockStoragePolicy>(op, src) {
       @Override
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
index 3c90356..e96bd62 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
@@ -920,6 +920,21 @@ public class TestDistributedFileSystem {
       dfs.getEZForPath(dir);
       checkStatistics(dfs, ++readOps, writeOps, 0);
       checkOpStatistics(OpType.GET_ENCRYPTION_ZONE, opCount + 1);
+
+      opCount = getOpStatistics(OpType.GET_SNAPSHOTTABLE_DIRECTORY_LIST);
+      dfs.getSnapshottableDirListing();
+      checkStatistics(dfs, ++readOps, writeOps, 0);
+      checkOpStatistics(OpType.GET_SNAPSHOTTABLE_DIRECTORY_LIST, opCount + 1);
+
+      opCount = getOpStatistics(OpType.GET_STORAGE_POLICIES);
+      dfs.getAllStoragePolicies();
+      checkStatistics(dfs, ++readOps, writeOps, 0);
+      checkOpStatistics(OpType.GET_STORAGE_POLICIES, opCount + 1);
+
+      opCount = getOpStatistics(OpType.GET_TRASH_ROOT);
+      dfs.getTrashRoot(dir);
+      checkStatistics(dfs, ++readOps, writeOps, 0);
+      checkOpStatistics(OpType.GET_TRASH_ROOT, opCount + 1);
     }
   }
 
@@ -1060,7 +1075,7 @@ public class TestDistributedFileSystem {
   }
 
   /** Checks statistics. -1 indicates do not check for the operations */
-  private void checkStatistics(FileSystem fs, int readOps, int writeOps,
+  public static void checkStatistics(FileSystem fs, int readOps, int writeOps,
       int largeReadOps) {
     assertEquals(readOps, DFSTestUtil.getStatistics(fs).getReadOps());
     assertEquals(writeOps, DFSTestUtil.getStatistics(fs).getWriteOps());
@@ -1166,12 +1181,12 @@ public class TestDistributedFileSystem {
     }
   }
 
-  private static void checkOpStatistics(OpType op, long count) {
+  public static void checkOpStatistics(OpType op, long count) {
     assertEquals("Op " + op.getSymbol() + " has unexpected count!",
         count, getOpStatistics(op));
   }
 
-  private static long getOpStatistics(OpType op) {
+  public static long getOpStatistics(OpType op) {
     return GlobalStorageStatistics.INSTANCE.get(
         DFSOpsCountStatistics.NAME)
         .getLong(op.getSymbol());
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java
index d02a5fe..27a6985 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java
@@ -24,6 +24,9 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CHECKSUM_TYPE_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ENCRYPT_DATA_TRANSFER_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_KEY;
+import static org.apache.hadoop.hdfs.TestDistributedFileSystem.checkOpStatistics;
+import static org.apache.hadoop.hdfs.TestDistributedFileSystem.checkStatistics;
+import static org.apache.hadoop.hdfs.TestDistributedFileSystem.getOpStatistics;
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY;
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_KEY;
 import static org.junit.Assert.assertEquals;
@@ -58,6 +61,7 @@ import java.util.Random;
 import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableList;
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.fs.QuotaUsage;
+import org.apache.hadoop.hdfs.DFSOpsCountStatistics;
 import org.apache.hadoop.test.LambdaTestUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -2013,6 +2017,62 @@ public class TestWebHDFS {
         ecpolicyForECfile, ecPolicyName);
   }
 
+  @Test
+  public void testStatistics() throws Exception {
+    final Configuration conf = new HdfsConfiguration();
+    conf.set(DFSConfigKeys.DFS_STORAGE_POLICY_SATISFIER_MODE_KEY,
+        StoragePolicySatisfierMode.EXTERNAL.toString());
+    StoragePolicySatisfier sps = new StoragePolicySatisfier(conf);
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).storageTypes(
+          new StorageType[][] {{StorageType.DISK, StorageType.ARCHIVE}})
+          .storagesPerDatanode(2).numDataNodes(1).build();
+      cluster.waitActive();
+      sps.init(new ExternalSPSContext(sps, DFSTestUtil
+          .getNameNodeConnector(conf, HdfsServerConstants.MOVER_ID_PATH, 1,
+              false)));
+      sps.start(StoragePolicySatisfierMode.EXTERNAL);
+      sps.start(StoragePolicySatisfierMode.EXTERNAL);
+      final WebHdfsFileSystem webHdfs = WebHdfsTestUtil
+          .getWebHdfsFileSystem(conf, WebHdfsConstants.WEBHDFS_SCHEME);
+      Path dir = new Path("/test");
+      webHdfs.mkdirs(dir);
+      int readOps = 0;
+      int writeOps = 0;
+      FileSystem.clearStatistics();
+
+      long opCount =
+          getOpStatistics(DFSOpsCountStatistics.OpType.GET_STORAGE_POLICY);
+      webHdfs.getStoragePolicy(dir);
+      checkStatistics(webHdfs, ++readOps, writeOps, 0);
+      checkOpStatistics(DFSOpsCountStatistics.OpType.GET_STORAGE_POLICY,
+          opCount + 1);
+
+      opCount =
+          getOpStatistics(DFSOpsCountStatistics.OpType.GET_STORAGE_POLICIES);
+      webHdfs.getAllStoragePolicies();
+      checkStatistics(webHdfs, ++readOps, writeOps, 0);
+      checkOpStatistics(DFSOpsCountStatistics.OpType.GET_STORAGE_POLICIES,
+          opCount + 1);
+
+      opCount =
+          getOpStatistics(DFSOpsCountStatistics.OpType.SATISFY_STORAGE_POLICY);
+      webHdfs.satisfyStoragePolicy(dir);
+      checkStatistics(webHdfs, readOps, ++writeOps, 0);
+      checkOpStatistics(DFSOpsCountStatistics.OpType.SATISFY_STORAGE_POLICY,
+          opCount + 1);
+
+      opCount = getOpStatistics(
+          DFSOpsCountStatistics.OpType.GET_SNAPSHOTTABLE_DIRECTORY_LIST);
+      webHdfs.getSnapshottableDirectoryList();
+      checkStatistics(webHdfs, ++readOps, writeOps, 0);
+      checkOpStatistics(
+          DFSOpsCountStatistics.OpType.GET_SNAPSHOTTABLE_DIRECTORY_LIST,
+          opCount + 1);
+    } finally {
+      cluster.shutdown();
+    }
+  }
   /**
    * Get FileStatus JSONObject from ListStatus response.
    */

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


[hadoop] 03/05: HDFS-15265. HttpFS: validate content-type in HttpFSUtils. Contributed by hemanthboyina.

Posted by we...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

weichiu pushed a commit to branch branch-3.3
in repository https://gitbox.apache.org/repos/asf/hadoop.git

commit 41fad5056c009591dcae0a619c5f0894f13f4e7e
Author: Inigo Goiri <in...@apache.org>
AuthorDate: Wed Apr 29 13:41:36 2020 -0700

    HDFS-15265. HttpFS: validate content-type in HttpFSUtils. Contributed by hemanthboyina.
    
    (cherry picked from commit 31b2f687eff5a77ea83c2279cc94c4c7158160dc)
---
 .../apache/hadoop/fs/http/client/HttpFSUtils.java  | 13 ++++++++-
 .../hadoop/fs/http/server/TestHttpFSServer.java    | 31 ++++++++++++++++++++++
 2 files changed, 43 insertions(+), 1 deletion(-)

diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSUtils.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSUtils.java
index fcc7bab..bd9baaa 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSUtils.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSUtils.java
@@ -33,6 +33,8 @@ import java.text.MessageFormat;
 import java.util.List;
 import java.util.Map;
 
+import javax.ws.rs.core.MediaType;
+
 /**
  * Utility methods used by HttpFS classes.
  */
@@ -127,8 +129,17 @@ public class HttpFSUtils {
    * @throws IOException thrown if the <code>InputStream</code> could not be
    * JSON parsed.
    */
-  static Object jsonParse(HttpURLConnection conn) throws IOException {
+  public static Object jsonParse(HttpURLConnection conn) throws IOException {
     try {
+      String contentType = conn.getContentType();
+      if (contentType != null) {
+        final MediaType parsed = MediaType.valueOf(contentType);
+        if (!MediaType.APPLICATION_JSON_TYPE.isCompatible(parsed)) {
+          throw new IOException("Content-Type \"" + contentType
+              + "\" is incompatible with \"" + MediaType.APPLICATION_JSON
+              + "\" (parsed=\"" + parsed + "\")");
+        }
+      }
       JSONParser parser = new JSONParser();
       return parser.parse(
           new InputStreamReader(conn.getInputStream(), StandardCharsets.UTF_8));
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServer.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServer.java
index ebb118e..2f0ef9a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServer.java
@@ -71,6 +71,8 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.XAttrCodec;
+import org.apache.hadoop.fs.http.client.HttpFSUtils;
+import org.apache.hadoop.fs.http.client.HttpFSFileSystem.Operation;
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.DataParam;
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.NoRedirectParam;
 import org.apache.hadoop.fs.permission.AclEntry;
@@ -89,6 +91,7 @@ import org.apache.hadoop.security.authentication.util.Signer;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.test.HFSTestCase;
 import org.apache.hadoop.test.HadoopUsersConfTestHelper;
+import org.apache.hadoop.test.LambdaTestUtils;
 import org.apache.hadoop.test.TestDir;
 import org.apache.hadoop.test.TestDirHelper;
 import org.apache.hadoop.test.TestHdfs;
@@ -1917,4 +1920,32 @@ public class TestHttpFSServer extends HFSTestCase {
     Assert.assertEquals(TestJettyHelper.getJettyURL() + "/webhdfs/v1" + path,
         location);
   }
+
+  @Test
+  @TestDir
+  @TestJetty
+  @TestHdfs
+  public void testContentType() throws Exception {
+    createHttpFSServer(false, false);
+    FileSystem fs = FileSystem.get(TestHdfsHelper.getHdfsConf());
+    Path dir = new Path("/tmp");
+    Path file = new Path(dir, "foo");
+    fs.mkdirs(dir);
+    fs.create(file);
+
+    String user = HadoopUsersConfTestHelper.getHadoopUsers()[0];
+    URL url = new URL(TestJettyHelper.getJettyURL(), MessageFormat.format(
+        "/webhdfs/v1/tmp/foo?user.name={0}&op=open&offset=1&length=2", user));
+
+    // test jsonParse with non-json type.
+    final HttpURLConnection conn = (HttpURLConnection) url.openConnection();
+    conn.setRequestMethod(Operation.OPEN.getMethod());
+    conn.connect();
+
+    LambdaTestUtils.intercept(IOException.class,
+        "Content-Type \"text/html;charset=iso-8859-1\" "
+            + "is incompatible with \"application/json\"",
+        () -> HttpFSUtils.jsonParse(conn));
+    conn.disconnect();
+  }
 }

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


[hadoop] 01/05: HDFS-15667. Audit log record the unexpected allowed result when delete (#2437)

Posted by we...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

weichiu pushed a commit to branch branch-3.3
in repository https://gitbox.apache.org/repos/asf/hadoop.git

commit 2b207ea4029acbfd88bf69292aeb7b606bf50249
Author: maobaolong <30...@qq.com>
AuthorDate: Tue Nov 10 13:01:10 2020 +0800

    HDFS-15667. Audit log record the unexpected allowed result when delete (#2437)
    
    (cherry picked from commit 95c96605b30cc31839a04bd5d4061a2c89e4c09c)
---
 .../apache/hadoop/hdfs/server/namenode/FSNamesystem.java    |  2 +-
 .../hdfs/server/namenode/TestAuditLoggerWithCommands.java   | 13 +++++++++++++
 2 files changed, 14 insertions(+), 1 deletion(-)

diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index 194f1af..ba52441 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -3256,7 +3256,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       throw e;
     }
     getEditLog().logSync();
-    logAuditEvent(true, operationName, src);
+    logAuditEvent(ret, operationName, src);
     if (toRemovedBlocks != null) {
       removeBlocks(toRemovedBlocks); // Incremental deletion of blocks
     }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLoggerWithCommands.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLoggerWithCommands.java
index 0814d4a..4d379b1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLoggerWithCommands.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLoggerWithCommands.java
@@ -51,6 +51,7 @@ import org.junit.Before;
 import org.junit.Test;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.fail;
 import org.mockito.Mockito;
 
@@ -1205,6 +1206,18 @@ public class TestAuditLoggerWithCommands {
     }
   }
 
+  @Test
+  public void testDeleteRoot() throws Exception {
+    Path srcDir = new Path("/");
+    fileSys = DFSTestUtil.getFileSystemAs(user1, conf);
+    boolean result = fileSys.delete(srcDir, true);
+    fileSys.close();
+    assertFalse(result);
+    String aceDeletePattern =
+        ".*allowed=false.*ugi=theDoctor.*cmd=delete.*";
+    verifyAuditLogs(aceDeletePattern);
+  }
+
   private void verifyAuditRestoreFailedStorageACE(
       FSNamesystem fsNamesystem, String arg) throws IOException {
     String operationName = fsNamesystem.getFailedStorageCommand(arg);

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


[hadoop] 04/05: HDFS-15332. Quota Space consumed was wrong in truncate with Snapshots. Contributed by hemanthboyina.

Posted by we...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

weichiu pushed a commit to branch branch-3.3
in repository https://gitbox.apache.org/repos/asf/hadoop.git

commit 1148d3895f140465d199025d886dc87dcc411282
Author: Inigo Goiri <in...@apache.org>
AuthorDate: Wed May 6 13:18:32 2020 -0700

    HDFS-15332. Quota Space consumed was wrong in truncate with Snapshots. Contributed by hemanthboyina.
    
    (cherry picked from commit 130f89e068ca346a44fa6619ae0726c1e5cc5d06)
---
 .../namenode/snapshot/FileWithSnapshotFeature.java |  6 ++++
 .../hdfs/server/namenode/TestFileTruncate.java     | 40 ++++++++++++++++++++++
 2 files changed, 46 insertions(+)

diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshotFeature.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshotFeature.java
index 7fa7ce7..44c258c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshotFeature.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshotFeature.java
@@ -163,6 +163,12 @@ public class FileWithSnapshotFeature implements INode.Feature {
       if (removed.getBlocks() != null) {
         allBlocks.addAll(Arrays.asList(removed.getBlocks()));
       }
+      for (FileDiff diff : diffs) {
+        BlockInfo[] diffBlocks = diff.getBlocks();
+        if (diffBlocks != null) {
+          allBlocks.addAll(Arrays.asList(diffBlocks));
+        }
+      }
       for (BlockInfo b: allBlocks) {
         short replication = b.getReplication();
         long blockSize = b.isComplete() ? b.getNumBytes() : file
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
index 0964b3e..335772b 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
@@ -1420,4 +1420,44 @@ public class TestFileTruncate {
     fs.concat(trg, srcs);
     assertEquals(1, fs.getContentSummary(new Path(dir)).getFileCount());
   }
+
+  /**
+   * Test Quota space consumed with multiple snapshots.
+   */
+  @Test
+  public void testQuotaSpaceConsumedWithSnapshots() throws IOException {
+    Path root = new Path("/");
+    Path dir = new Path(root, "dir");
+    fs.mkdirs(dir);
+    fs.allowSnapshot(dir);
+
+    // create a file
+    Path file2 = new Path(dir, "file2");
+    DFSTestUtil.createFile(fs, file2, 30, (short) 1, 0);
+
+    // create a snapshot and truncate the file
+    fs.createSnapshot(dir, "s1");
+    boolean isReady = fs.truncate(file2, 20);
+    if (!isReady) {
+      checkBlockRecovery(file2);
+    }
+
+    // create one more snapshot and truncate the file which exists in previous
+    // snapshot
+    fs.createSnapshot(dir, "s2");
+    isReady = fs.truncate(file2, 10);
+    if (!isReady) {
+      checkBlockRecovery(file2);
+    }
+
+    // delete the snapshots and check quota space consumed usage
+    fs.deleteSnapshot(dir, "s1");
+    fs.deleteSnapshot(dir, "s2");
+    assertEquals(fs.getContentSummary(root).getSpaceConsumed(),
+        fs.getQuotaUsage(root).getSpaceConsumed());
+    fs.delete(dir, true);
+    assertEquals(fs.getContentSummary(root).getSpaceConsumed(),
+        fs.getQuotaUsage(root).getSpaceConsumed());
+
+  }
 }

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