You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hdfs-commits@hadoop.apache.org by ar...@apache.org on 2019/10/25 17:22:06 UTC

[hadoop-ozone] branch master updated: HDDS-2341. Validate tar entry path during extraction (#72)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new d1f57f8  HDDS-2341. Validate tar entry path during extraction (#72)
d1f57f8 is described below

commit d1f57f84cf2cb02aab3c6a7d6c298917b95c3d30
Author: Doroszlai, Attila <64...@users.noreply.github.com>
AuthorDate: Fri Oct 25 19:21:56 2019 +0200

    HDDS-2341. Validate tar entry path during extraction (#72)
---
 .../java/org/apache/hadoop/hdds/HddsUtils.java     |  23 +++
 .../java/org/apache/hadoop/hdds/TestHddsUtils.java |  21 +++
 .../container/keyvalue/TarContainerPacker.java     |  31 ++--
 .../container/keyvalue/TestTarContainerPacker.java | 204 +++++++++++++++++----
 .../org/apache/hadoop/ozone/recon/ReconUtils.java  |  11 +-
 5 files changed, 236 insertions(+), 54 deletions(-)

diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsUtils.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsUtils.java
index d7b20fd..211c6aa 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsUtils.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsUtils.java
@@ -24,6 +24,7 @@ import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
 import java.net.InetSocketAddress;
 import java.net.UnknownHostException;
+import java.nio.file.Path;
 import java.util.Calendar;
 import java.util.Collection;
 import java.util.HashSet;
@@ -32,6 +33,7 @@ import java.util.Optional;
 import java.util.TimeZone;
 import java.util.concurrent.TimeUnit;
 
+import com.google.common.base.Preconditions;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
@@ -502,4 +504,25 @@ public final class HddsUtils {
         DefaultMetricsSystem.instance());
     return metricsSystem;
   }
+
+  /**
+   * Basic validation for {@code path}: checks that it is a descendant of
+   * (or the same as) the given {@code ancestor}.
+   * @param path the path to be validated
+   * @param ancestor a trusted path that is supposed to be the ancestor of
+   *     {@code path}
+   * @throws NullPointerException if either {@code path} or {@code ancestor} is
+   *     null
+   * @throws IllegalArgumentException if {@code ancestor} is not really the
+   *     ancestor of {@code path}
+   */
+  public static void validatePath(Path path, Path ancestor) {
+    Preconditions.checkNotNull(path,
+        "Path should not be null");
+    Preconditions.checkNotNull(ancestor,
+        "Ancestor should not be null");
+    Preconditions.checkArgument(
+        path.normalize().startsWith(ancestor.normalize()),
+        "Path should be a descendant of " + ancestor);
+  }
 }
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/TestHddsUtils.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/TestHddsUtils.java
index 7563610..491bdc9 100644
--- a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/TestHddsUtils.java
+++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/TestHddsUtils.java
@@ -17,8 +17,10 @@
  */
 package org.apache.hadoop.hdds;
 
+import java.nio.file.Paths;
 import java.util.Optional;
 
+import org.apache.hadoop.test.LambdaTestUtils;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -39,4 +41,23 @@ public class TestHddsUtils {
         HddsUtils.getHostName(":1234"));
   }
 
+  @Test
+  public void validatePath() throws Exception {
+    HddsUtils.validatePath(Paths.get("/"), Paths.get("/"));
+    HddsUtils.validatePath(Paths.get("/a"), Paths.get("/"));
+    HddsUtils.validatePath(Paths.get("/a"), Paths.get("/a"));
+    HddsUtils.validatePath(Paths.get("/a/b"), Paths.get("/a"));
+    HddsUtils.validatePath(Paths.get("/a/b/c"), Paths.get("/a"));
+    HddsUtils.validatePath(Paths.get("/a/../a/b"), Paths.get("/a"));
+
+    LambdaTestUtils.intercept(IllegalArgumentException.class,
+        () -> HddsUtils.validatePath(Paths.get("/b/c"), Paths.get("/a")));
+    LambdaTestUtils.intercept(IllegalArgumentException.class,
+        () -> HddsUtils.validatePath(Paths.get("/"), Paths.get("/a")));
+    LambdaTestUtils.intercept(IllegalArgumentException.class,
+        () -> HddsUtils.validatePath(Paths.get("/a/.."), Paths.get("/a")));
+    LambdaTestUtils.intercept(IllegalArgumentException.class,
+        () -> HddsUtils.validatePath(Paths.get("/a/../b"), Paths.get("/a")));
+  }
+
 }
\ No newline at end of file
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/TarContainerPacker.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/TarContainerPacker.java
index 13689a7..b039f6e 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/TarContainerPacker.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/TarContainerPacker.java
@@ -30,11 +30,11 @@ import java.nio.file.Path;
 import java.nio.file.Paths;
 import java.util.stream.Collectors;
 
+import org.apache.hadoop.hdds.HddsUtils;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ozone.container.common.interfaces.Container;
 import org.apache.hadoop.ozone.container.common.interfaces.ContainerPacker;
 
-import com.google.common.base.Preconditions;
 import org.apache.commons.compress.archivers.ArchiveEntry;
 import org.apache.commons.compress.archivers.ArchiveOutputStream;
 import org.apache.commons.compress.archivers.tar.TarArchiveEntry;
@@ -52,9 +52,9 @@ import org.apache.commons.io.IOUtils;
 public class TarContainerPacker
     implements ContainerPacker<KeyValueContainerData> {
 
-  private static final String CHUNKS_DIR_NAME = OzoneConsts.STORAGE_DIR_CHUNKS;
+  static final String CHUNKS_DIR_NAME = OzoneConsts.STORAGE_DIR_CHUNKS;
 
-  private static final String DB_DIR_NAME = "db";
+  static final String DB_DIR_NAME = "db";
 
   private static final String CONTAINER_FILE_NAME = "container.yaml";
 
@@ -66,7 +66,6 @@ public class TarContainerPacker
    *
    * @param container container which defines the destination structure.
    * @param inputStream the input stream.
-   * @throws IOException
    */
   @Override
   public byte[] unpackContainerData(Container<KeyValueContainerData> container,
@@ -87,13 +86,15 @@ public class TarContainerPacker
       while (entry != null) {
         String name = entry.getName();
         if (name.startsWith(DB_DIR_NAME + "/")) {
-          Path destinationPath = containerData.getDbFile().toPath()
+          Path dbRoot = containerData.getDbFile().toPath();
+          Path destinationPath = dbRoot
               .resolve(name.substring(DB_DIR_NAME.length() + 1));
-          extractEntry(tarInput, entry.getSize(), destinationPath);
+          extractEntry(tarInput, entry.getSize(), dbRoot, destinationPath);
         } else if (name.startsWith(CHUNKS_DIR_NAME + "/")) {
-          Path destinationPath = Paths.get(containerData.getChunksPath())
+          Path chunksRoot = Paths.get(containerData.getChunksPath());
+          Path destinationPath = chunksRoot
               .resolve(name.substring(CHUNKS_DIR_NAME.length() + 1));
-          extractEntry(tarInput, entry.getSize(), destinationPath);
+          extractEntry(tarInput, entry.getSize(), chunksRoot, destinationPath);
         } else if (name.equals(CONTAINER_FILE_NAME)) {
           //Don't do anything. Container file should be unpacked in a
           //separated step by unpackContainerDescriptor call.
@@ -115,11 +116,12 @@ public class TarContainerPacker
   }
 
   private void extractEntry(TarArchiveInputStream tarInput, long size,
-      Path path) throws IOException {
-    Preconditions.checkNotNull(path, "Path element should not be null");
-    Path parent = Preconditions.checkNotNull(path.getParent(),
-        "Path element should have a parent directory");
-    Files.createDirectories(parent);
+                            Path ancestor, Path path) throws IOException {
+    HddsUtils.validatePath(path, ancestor);
+    Path parent = path.getParent();
+    if (parent != null) {
+      Files.createDirectories(parent);
+    }
     try (BufferedOutputStream bos = new BufferedOutputStream(
         new FileOutputStream(path.toAbsolutePath().toString()))) {
       int bufferSize = 1024;
@@ -145,7 +147,6 @@ public class TarContainerPacker
    *
    * @param container Container to archive (data + metadata).
    * @param destination   Destination tar file/stream.
-   * @throws IOException
    */
   @Override
   public void pack(Container<KeyValueContainerData> container,
@@ -235,7 +236,7 @@ public class TarContainerPacker
     }
   }
 
-  private void includeFile(File file, String entryName,
+  static void includeFile(File file, String entryName,
       ArchiveOutputStream archiveOutputStream) throws IOException {
     ArchiveEntry archiveEntry =
         archiveOutputStream.createArchiveEntry(file, entryName);
diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestTarContainerPacker.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestTarContainerPacker.java
index 9e6f653..b3a0edf 100644
--- a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestTarContainerPacker.java
+++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestTarContainerPacker.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.ozone.container.keyvalue;
 
+import java.io.File;
 import java.io.FileInputStream;
 import java.io.FileOutputStream;
 import java.io.FileWriter;
@@ -30,7 +31,11 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.UUID;
+import java.util.concurrent.atomic.AtomicInteger;
 
+import org.apache.commons.compress.archivers.ArchiveOutputStream;
+import org.apache.commons.compress.archivers.tar.TarArchiveOutputStream;
+import org.apache.commons.compress.compressors.CompressorOutputStream;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.ozone.container.common.interfaces.ContainerPacker;
 
@@ -41,11 +46,14 @@ import org.apache.commons.compress.compressors.CompressorInputStream;
 import org.apache.commons.compress.compressors.CompressorStreamFactory;
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.test.LambdaTestUtils;
+import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.commons.compress.compressors.CompressorStreamFactory.GZIP;
 
 /**
  * Test the tar/untar for a given container.
@@ -62,7 +70,8 @@ public class TestTarContainerPacker {
 
   private static final String TEST_DESCRIPTOR_FILE_CONTENT = "descriptor";
 
-  private ContainerPacker packer = new TarContainerPacker();
+  private final ContainerPacker<KeyValueContainerData> packer
+      = new TarContainerPacker();
 
   private static final Path SOURCE_CONTAINER_ROOT =
       Paths.get("target/test/data/packer-source-dir");
@@ -70,21 +79,34 @@ public class TestTarContainerPacker {
   private static final Path DEST_CONTAINER_ROOT =
       Paths.get("target/test/data/packer-dest-dir");
 
+  private static final Path TEMP_DIR =
+      Paths.get("target/test/data/packer-tmp-dir");
+
+  private static final AtomicInteger CONTAINER_ID = new AtomicInteger(1);
+
   @BeforeClass
   public static void init() throws IOException {
     initDir(SOURCE_CONTAINER_ROOT);
     initDir(DEST_CONTAINER_ROOT);
+    initDir(TEMP_DIR);
+  }
+
+  @AfterClass
+  public static void cleanup() throws IOException {
+    FileUtils.deleteDirectory(SOURCE_CONTAINER_ROOT.toFile());
+    FileUtils.deleteDirectory(DEST_CONTAINER_ROOT.toFile());
+    FileUtils.deleteDirectory(TEMP_DIR.toFile());
   }
 
   private static void initDir(Path path) throws IOException {
     if (path.toFile().exists()) {
       FileUtils.deleteDirectory(path.toFile());
     }
-    path.toFile().mkdirs();
+    Files.createDirectories(path);
   }
 
-  private KeyValueContainerData createContainer(long id, Path dir,
-      OzoneConfiguration conf) throws IOException {
+  private KeyValueContainerData createContainer(Path dir) throws IOException {
+    long id = CONTAINER_ID.getAndIncrement();
 
     Path containerDir = dir.resolve("container" + id);
     Path dbDir = containerDir.resolve("db");
@@ -98,7 +120,6 @@ public class TestTarContainerPacker {
     containerData.setMetadataPath(dbDir.getParent().toString());
     containerData.setDbFile(dbDir.toFile());
 
-
     return containerData;
   }
 
@@ -109,35 +130,21 @@ public class TestTarContainerPacker {
     OzoneConfiguration conf = new OzoneConfiguration();
 
     KeyValueContainerData sourceContainerData =
-        createContainer(1L, SOURCE_CONTAINER_ROOT, conf);
+        createContainer(SOURCE_CONTAINER_ROOT);
 
     KeyValueContainer sourceContainer =
         new KeyValueContainer(sourceContainerData, conf);
 
     //sample db file in the metadata directory
-    try (FileWriter writer = new FileWriter(
-        sourceContainerData.getDbFile().toPath()
-            .resolve(TEST_DB_FILE_NAME)
-            .toFile())) {
-      IOUtils.write(TEST_DB_FILE_CONTENT, writer);
-    }
+    writeDbFile(sourceContainerData, TEST_DB_FILE_NAME);
 
     //sample chunk file in the chunk directory
-    try (FileWriter writer = new FileWriter(
-        Paths.get(sourceContainerData.getChunksPath())
-            .resolve(TEST_CHUNK_FILE_NAME)
-            .toFile())) {
-      IOUtils.write(TEST_CHUNK_FILE_CONTENT, writer);
-    }
+    writeChunkFile(sourceContainerData, TEST_CHUNK_FILE_NAME);
 
     //sample container descriptor file
-    try (FileWriter writer = new FileWriter(
-        sourceContainer.getContainerFile())) {
-      IOUtils.write(TEST_DESCRIPTOR_FILE_CONTENT, writer);
-    }
+    writeDescriptor(sourceContainer);
 
-    Path targetFile =
-        SOURCE_CONTAINER_ROOT.getParent().resolve("container.tar.gz");
+    Path targetFile = TEMP_DIR.resolve("container.tar.gz");
 
     //WHEN: pack it
     try (FileOutputStream output = new FileOutputStream(targetFile.toFile())) {
@@ -147,7 +154,7 @@ public class TestTarContainerPacker {
     //THEN: check the result
     try (FileInputStream input = new FileInputStream(targetFile.toFile())) {
       CompressorInputStream uncompressed = new CompressorStreamFactory()
-          .createCompressorInputStream(CompressorStreamFactory.GZIP, input);
+          .createCompressorInputStream(GZIP, input);
       TarArchiveInputStream tarStream = new TarArchiveInputStream(uncompressed);
 
       TarArchiveEntry entry;
@@ -169,12 +176,12 @@ public class TestTarContainerPacker {
     }
 
     KeyValueContainerData destinationContainerData =
-        createContainer(2L, DEST_CONTAINER_ROOT, conf);
+        createContainer(DEST_CONTAINER_ROOT);
 
     KeyValueContainer destinationContainer =
         new KeyValueContainer(destinationContainerData, conf);
 
-    String descriptor = "";
+    String descriptor;
 
     //unpackContainerData
     try (FileInputStream input = new FileInputStream(targetFile.toFile())) {
@@ -184,22 +191,151 @@ public class TestTarContainerPacker {
     }
 
     assertExampleMetadataDbIsGood(
-        destinationContainerData.getDbFile().toPath());
+        destinationContainerData.getDbFile().toPath(),
+        TEST_DB_FILE_NAME);
     assertExampleChunkFileIsGood(
-        Paths.get(destinationContainerData.getChunksPath()));
+        Paths.get(destinationContainerData.getChunksPath()),
+        TEST_CHUNK_FILE_NAME);
     Assert.assertFalse(
-        "Descriptor file should not been exctarcted by the "
+        "Descriptor file should not have been extracted by the "
             + "unpackContainerData Call",
         destinationContainer.getContainerFile().exists());
     Assert.assertEquals(TEST_DESCRIPTOR_FILE_CONTENT, descriptor);
+  }
+
+  @Test
+  public void unpackContainerDataWithValidRelativeDbFilePath()
+      throws Exception {
+    //GIVEN
+    KeyValueContainerData sourceContainerData =
+        createContainer(SOURCE_CONTAINER_ROOT);
+
+    String fileName = "sub/dir/" + TEST_DB_FILE_NAME;
+    File file = writeDbFile(sourceContainerData, fileName);
+    String entryName = TarContainerPacker.DB_DIR_NAME + "/" + fileName;
+
+    File containerFile = packContainerWithSingleFile(file, entryName);
+
+    // WHEN
+    KeyValueContainerData dest = unpackContainerData(containerFile);
+
+    // THEN
+    assertExampleMetadataDbIsGood(dest.getDbFile().toPath(), fileName);
+  }
+
+  @Test
+  public void unpackContainerDataWithValidRelativeChunkFilePath()
+      throws Exception {
+    //GIVEN
+    KeyValueContainerData sourceContainerData =
+        createContainer(SOURCE_CONTAINER_ROOT);
+
+    String fileName = "sub/dir/" + TEST_CHUNK_FILE_NAME;
+    File file = writeChunkFile(sourceContainerData, fileName);
+    String entryName = TarContainerPacker.CHUNKS_DIR_NAME + "/" + fileName;
+
+    File containerFile = packContainerWithSingleFile(file, entryName);
+
+    // WHEN
+    KeyValueContainerData dest = unpackContainerData(containerFile);
+
+    // THEN
+    assertExampleChunkFileIsGood(Paths.get(dest.getChunksPath()), fileName);
+  }
+
+  @Test
+  public void unpackContainerDataWithInvalidRelativeDbFilePath()
+      throws Exception {
+    //GIVEN
+    KeyValueContainerData sourceContainerData =
+        createContainer(SOURCE_CONTAINER_ROOT);
+
+    String fileName = "../db_file";
+    File file = writeDbFile(sourceContainerData, fileName);
+    String entryName = TarContainerPacker.DB_DIR_NAME + "/" + fileName;
+
+    File containerFile = packContainerWithSingleFile(file, entryName);
+
+    LambdaTestUtils.intercept(IllegalArgumentException.class,
+        () -> unpackContainerData(containerFile));
+  }
+
+  @Test
+  public void unpackContainerDataWithInvalidRelativeChunkFilePath()
+      throws Exception {
+    //GIVEN
+    KeyValueContainerData sourceContainerData =
+        createContainer(SOURCE_CONTAINER_ROOT);
+
+    String fileName = "../chunk_file";
+    File file = writeChunkFile(sourceContainerData, fileName);
+    String entryName = TarContainerPacker.CHUNKS_DIR_NAME + "/" + fileName;
+
+    File containerFile = packContainerWithSingleFile(file, entryName);
+
+    LambdaTestUtils.intercept(IllegalArgumentException.class,
+        () -> unpackContainerData(containerFile));
+  }
+
+  private KeyValueContainerData unpackContainerData(File containerFile)
+      throws IOException {
+    try (FileInputStream input = new FileInputStream(containerFile)) {
+      OzoneConfiguration conf = new OzoneConfiguration();
+      KeyValueContainerData data = createContainer(DEST_CONTAINER_ROOT);
+      KeyValueContainer container = new KeyValueContainer(data, conf);
+      packer.unpackContainerData(container, input);
+      return data;
+    }
+  }
+
+  private void writeDescriptor(KeyValueContainer container) throws IOException {
+    try (FileWriter writer = new FileWriter(container.getContainerFile())) {
+      IOUtils.write(TEST_DESCRIPTOR_FILE_CONTENT, writer);
+    }
+  }
+
+  private File writeChunkFile(
+      KeyValueContainerData containerData, String chunkFileName)
+      throws IOException {
+    Path path = Paths.get(containerData.getChunksPath())
+        .resolve(chunkFileName);
+    Files.createDirectories(path.getParent());
+    File file = path.toFile();
+    try (FileWriter writer = new FileWriter(file)) {
+      IOUtils.write(TEST_CHUNK_FILE_CONTENT, writer);
+    }
+    return file;
+  }
 
+  private File writeDbFile(
+      KeyValueContainerData containerData, String dbFileName)
+      throws IOException {
+    Path path = containerData.getDbFile().toPath()
+        .resolve(dbFileName);
+    Files.createDirectories(path.getParent());
+    File file = path.toFile();
+    try (FileWriter writer = new FileWriter(file)) {
+      IOUtils.write(TEST_DB_FILE_CONTENT, writer);
+    }
+    return file;
   }
 
+  private File packContainerWithSingleFile(File file, String entryName)
+      throws Exception {
+    File targetFile = TEMP_DIR.resolve("container.tar.gz").toFile();
+    try (FileOutputStream output = new FileOutputStream(targetFile);
+         CompressorOutputStream gzipped = new CompressorStreamFactory()
+             .createCompressorOutputStream(GZIP, output);
+         ArchiveOutputStream archive = new TarArchiveOutputStream(gzipped)) {
+      TarContainerPacker.includeFile(file, entryName, archive);
+    }
+    return targetFile;
+  }
 
-  private void assertExampleMetadataDbIsGood(Path dbPath)
+  private void assertExampleMetadataDbIsGood(Path dbPath, String filename)
       throws IOException {
 
-    Path dbFile = dbPath.resolve(TEST_DB_FILE_NAME);
+    Path dbFile = dbPath.resolve(filename);
 
     Assert.assertTrue(
         "example DB file is missing after pack/unpackContainerData: " + dbFile,
@@ -213,10 +349,10 @@ public class TestTarContainerPacker {
     }
   }
 
-  private void assertExampleChunkFileIsGood(Path chunkDirPath)
+  private void assertExampleChunkFileIsGood(Path chunkPath, String filename)
       throws IOException {
 
-    Path chunkFile = chunkDirPath.resolve(TEST_CHUNK_FILE_NAME);
+    Path chunkFile = chunkPath.resolve(filename);
 
     Assert.assertTrue(
         "example chunk file is missing after pack/unpackContainerData: "
diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/ReconUtils.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/ReconUtils.java
index 2d29d3f..74f601c 100644
--- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/ReconUtils.java
+++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/ReconUtils.java
@@ -40,6 +40,7 @@ import org.apache.commons.compress.archivers.tar.TarArchiveOutputStream;
 import org.apache.commons.compress.compressors.gzip.GzipCompressorInputStream;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdds.HddsConfigKeys;
+import org.apache.hadoop.hdds.HddsUtils;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.http.HttpEntity;
 import org.apache.http.HttpResponse;
@@ -172,13 +173,14 @@ public class ReconUtils {
 
       try (TarArchiveInputStream tarInStream =
                new TarArchiveInputStream(gzIn)) {
-        TarArchiveEntry entry = null;
+        TarArchiveEntry entry;
 
         while ((entry = (TarArchiveEntry) tarInStream.getNextEntry()) != null) {
+          Path path = Paths.get(destPath.toString(), entry.getName());
+          HddsUtils.validatePath(path, destPath);
+          File f = path.toFile();
           //If directory, create a directory.
           if (entry.isDirectory()) {
-            File f = new File(Paths.get(destPath.toString(),
-                entry.getName()).toString());
             boolean success = f.mkdirs();
             if (!success) {
               LOG.error("Unable to create directory found in tar.");
@@ -188,8 +190,7 @@ public class ReconUtils {
             int count;
             byte[] data = new byte[WRITE_BUFFER];
 
-            FileOutputStream fos = new FileOutputStream(
-                Paths.get(destPath.toString(), entry.getName()).toString());
+            FileOutputStream fos = new FileOutputStream(f);
             try (BufferedOutputStream dest =
                      new BufferedOutputStream(fos, WRITE_BUFFER)) {
               while ((count =


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