You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by sa...@apache.org on 2023/07/23 15:26:13 UTC

[pinot] branch master updated: Sanitise API inputs used as file path variables (#11132)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 4dd2bb5bb4 Sanitise API inputs used as file path variables  (#11132)
4dd2bb5bb4 is described below

commit 4dd2bb5bb4df97c529dba2c259e23a12837077dd
Author: Saurabh Dubey <sa...@gmail.com>
AuthorDate: Sun Jul 23 20:56:06 2023 +0530

    Sanitise API inputs used as file path variables  (#11132)
    
    * Fix path traversal vulnerabilities
    
    ---------
    
    Co-authored-by: Saurabh Dubey <sa...@Saurabhs-MacBook-Pro.local>
---
 .../org/apache/pinot/common/utils/FileUtils.java   | 20 ++++++++++++++++++
 .../resources/LLCSegmentCompletionHandlers.java    | 24 ++++++++++++++--------
 .../PinotSegmentUploadDownloadRestletResource.java | 17 +++++++++++----
 .../pinot/controller/util/FileIngestionHelper.java |  4 ++--
 .../pinot/server/api/resources/TablesResource.java | 12 +++++++----
 5 files changed, 59 insertions(+), 18 deletions(-)

diff --git a/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUtils.java b/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUtils.java
index 8667d4c444..2c1cb10c43 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUtils.java
+++ b/pinot-common/src/main/java/org/apache/pinot/common/utils/FileUtils.java
@@ -114,4 +114,24 @@ public class FileUtils {
       throws IOException {
     close(Arrays.asList(closeables));
   }
+
+  /**
+   * Concatenates the folderDir and filename and validates that the resulting file path is still within the folderDir.
+   * @param folderDir the parent directory
+   * @param filename the filename to concatenate to the parent directory
+   * @param msg the error message if the resulting file path is not within the parent directory
+   * @param args the error message arguments
+   * @return File object representing the concatenated file path
+   * @throws IllegalArgumentException if the resulting file path is not within the parent directory
+   * @throws IOException if the resulting file path is invalid
+   */
+  public static File concatAndValidateFile(File folderDir, String filename, String msg, Object... args)
+      throws IllegalArgumentException, IOException {
+    File filePath = new File(folderDir, filename);
+    if (!filePath.getCanonicalPath().startsWith(folderDir.getCanonicalPath() + File.separator)) {
+      throw new IllegalArgumentException(String.format(msg, args));
+    }
+
+    return filePath;
+  }
 }
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/LLCSegmentCompletionHandlers.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/LLCSegmentCompletionHandlers.java
index c11bed4008..b287f07dfc 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/LLCSegmentCompletionHandlers.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/LLCSegmentCompletionHandlers.java
@@ -448,8 +448,10 @@ public class LLCSegmentCompletionHandlers {
           "Invalid multi-part for segment: %s", segmentName);
       FormDataBodyPart bodyPart = map.values().iterator().next().get(0);
 
-      File localTempFile = new File(ControllerFilePathProvider.getInstance().getFileUploadTempDir(),
-          getTempSegmentFileName(segmentName));
+      File localTempFile = org.apache.pinot.common.utils.FileUtils.concatAndValidateFile(
+          ControllerFilePathProvider.getInstance().getFileUploadTempDir(), getTempSegmentFileName(segmentName),
+          "Invalid segment name: %s", segmentName);
+
       try (InputStream inputStream = bodyPart.getValueAs(InputStream.class)) {
         Files.copy(inputStream, localTempFile.toPath());
       } catch (Exception e) {
@@ -468,8 +470,10 @@ public class LLCSegmentCompletionHandlers {
    */
   private static SegmentMetadataImpl extractMetadataFromLocalSegmentFile(File segmentFile)
       throws Exception {
-    File tempIndexDir =
-        new File(ControllerFilePathProvider.getInstance().getUntarredFileTempDir(), segmentFile.getName());
+    File tempIndexDir = org.apache.pinot.common.utils.FileUtils.concatAndValidateFile(
+        ControllerFilePathProvider.getInstance().getUntarredFileTempDir(), segmentFile.getName(),
+        "Invalid segment file: %s", segmentFile);
+
     try {
       FileUtils.forceMkdir(tempIndexDir);
 
@@ -494,8 +498,10 @@ public class LLCSegmentCompletionHandlers {
    */
   private static SegmentMetadataImpl extractSegmentMetadataFromForm(FormDataMultiPart form, String segmentName)
       throws IOException {
-    File tempIndexDir = new File(ControllerFilePathProvider.getInstance().getUntarredFileTempDir(),
-        getTempSegmentFileName(segmentName));
+    File tempIndexDir = org.apache.pinot.common.utils.FileUtils.concatAndValidateFile(
+        ControllerFilePathProvider.getInstance().getUntarredFileTempDir(), getTempSegmentFileName(segmentName),
+        "Invalid segment name: %s", segmentName);
+
     try {
       FileUtils.forceMkdir(tempIndexDir);
 
@@ -532,8 +538,10 @@ public class LLCSegmentCompletionHandlers {
    */
   private static SegmentMetadataImpl extractMetadataFromSegmentFileURI(URI segmentFileURI, String segmentName)
       throws Exception {
-    File localTempFile =
-        new File(ControllerFilePathProvider.getInstance().getFileUploadTempDir(), getTempSegmentFileName(segmentName));
+    File localTempFile = org.apache.pinot.common.utils.FileUtils.concatAndValidateFile(
+        ControllerFilePathProvider.getInstance().getFileUploadTempDir(), getTempSegmentFileName(segmentName),
+        "Invalid segment name: %s", segmentName);
+
     try {
       SegmentFetcherFactory.fetchSegmentToLocal(segmentFileURI, localTempFile);
       return extractMetadataFromLocalSegmentFile(localTempFile);
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentUploadDownloadRestletResource.java b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentUploadDownloadRestletResource.java
index 31ca292f6b..90f6a663a1 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentUploadDownloadRestletResource.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotSegmentUploadDownloadRestletResource.java
@@ -96,7 +96,6 @@ import org.apache.pinot.spi.filesystem.PinotFS;
 import org.apache.pinot.spi.filesystem.PinotFSFactory;
 import org.apache.pinot.spi.utils.CommonConstants;
 import org.apache.pinot.spi.utils.JsonUtils;
-import org.apache.pinot.spi.utils.StringUtil;
 import org.apache.pinot.spi.utils.builder.TableNameBuilder;
 import org.glassfish.grizzly.http.server.Request;
 import org.glassfish.jersey.media.multipart.FormDataBodyPart;
@@ -169,7 +168,12 @@ public class PinotSegmentUploadDownloadRestletResource {
     File segmentFile;
     // If the segment file is local, just use it as the return entity; otherwise copy it from remote to local first.
     if (CommonConstants.Segment.LOCAL_SEGMENT_SCHEME.equals(dataDirURI.getScheme())) {
-      segmentFile = new File(new File(dataDirURI), StringUtil.join(File.separator, tableName, segmentName));
+      File dataDir = new File(dataDirURI);
+      File tableDir = org.apache.pinot.common.utils.FileUtils.concatAndValidateFile(dataDir, tableName,
+          "Invalid table name: %s", tableName);
+      segmentFile = org.apache.pinot.common.utils.FileUtils.concatAndValidateFile(tableDir, segmentName,
+          "Invalid segment name: %s", segmentName);
+
       if (!segmentFile.exists()) {
         throw new ControllerApplicationException(LOGGER,
             "Segment " + segmentName + " or table " + tableName + " not found in " + segmentFile.getAbsolutePath(),
@@ -184,8 +188,13 @@ public class PinotSegmentUploadDownloadRestletResource {
             "Segment: " + segmentName + " of table: " + tableName + " not found at: " + remoteSegmentFileURI,
             Response.Status.NOT_FOUND);
       }
-      segmentFile = new File(new File(ControllerFilePathProvider.getInstance().getFileDownloadTempDir(), tableName),
-          segmentName + "-" + UUID.randomUUID());
+      File downloadTempDir = ControllerFilePathProvider.getInstance().getFileDownloadTempDir();
+      File tableDir = org.apache.pinot.common.utils.FileUtils.concatAndValidateFile(downloadTempDir, tableName,
+          "Invalid table name: %s", tableName);
+      segmentFile =
+          org.apache.pinot.common.utils.FileUtils.concatAndValidateFile(tableDir, segmentName + "-" + UUID.randomUUID(),
+              "Invalid segment name: %s", segmentName);
+
       pinotFS.copyToLocalFile(remoteSegmentFileURI, segmentFile);
       // Streaming in the tmp file and delete it afterward.
       builder.entity((StreamingOutput) output -> {
diff --git a/pinot-controller/src/main/java/org/apache/pinot/controller/util/FileIngestionHelper.java b/pinot-controller/src/main/java/org/apache/pinot/controller/util/FileIngestionHelper.java
index 3c3a335bd7..e9ee2f902a 100644
--- a/pinot-controller/src/main/java/org/apache/pinot/controller/util/FileIngestionHelper.java
+++ b/pinot-controller/src/main/java/org/apache/pinot/controller/util/FileIngestionHelper.java
@@ -92,9 +92,9 @@ public class FileIngestionHelper {
     String tableNameWithType = _tableConfig.getTableName();
     // 1. append a timestamp for easy debugging
     // 2. append a random string to avoid using the same working directory when multiple tasks are running in parallel
-    File workingDir = new File(_ingestionDir,
+    File workingDir = org.apache.pinot.common.utils.FileUtils.concatAndValidateFile(_ingestionDir,
         String.format("%s_%s_%d_%s", WORKING_DIR_PREFIX, tableNameWithType, System.currentTimeMillis(),
-            RandomStringUtils.random(10, true, false)));
+            RandomStringUtils.random(10, true, false)), "Invalid table name: %S", tableNameWithType);
     LOGGER.info("Starting ingestion of {} payload to table: {} using working dir: {}", payload._payloadType,
         tableNameWithType, workingDir.getAbsolutePath());
 
diff --git a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java
index e518d2d6b1..8148956ae6 100644
--- a/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java
+++ b/pinot-server/src/main/java/org/apache/pinot/server/api/resources/TablesResource.java
@@ -394,8 +394,10 @@ public class TablesResource {
           new File(_serverInstance.getInstanceDataManager().getSegmentFileDirectory(), PEER_SEGMENT_DOWNLOAD_DIR);
       tmpSegmentTarDir.mkdir();
 
-      File segmentTarFile = new File(tmpSegmentTarDir, tableNameWithType + "_" + segmentName + "_" + UUID.randomUUID()
-          + TarGzCompressionUtils.TAR_GZ_FILE_EXTENSION);
+      File segmentTarFile = org.apache.pinot.common.utils.FileUtils.concatAndValidateFile(tmpSegmentTarDir,
+          tableNameWithType + "_" + segmentName + "_" + UUID.randomUUID() + TarGzCompressionUtils.TAR_GZ_FILE_EXTENSION,
+          "Invalid table / segment name: %s , %s", tableNameWithType, segmentName);
+
       TarGzCompressionUtils.createTarGzFile(new File(tableDataManager.getTableDataDir(), segmentName), segmentTarFile);
       Response.ResponseBuilder builder = Response.ok();
       builder.entity((StreamingOutput) output -> {
@@ -529,8 +531,10 @@ public class TablesResource {
           new File(_serverInstance.getInstanceDataManager().getSegmentFileDirectory(), SEGMENT_UPLOAD_DIR);
       segmentTarUploadDir.mkdir();
 
-      segmentTarFile = new File(segmentTarUploadDir, tableNameWithType + "_" + segmentName + "_" + UUID.randomUUID()
-          + TarGzCompressionUtils.TAR_GZ_FILE_EXTENSION);
+      segmentTarFile = org.apache.pinot.common.utils.FileUtils.concatAndValidateFile(segmentTarUploadDir,
+          tableNameWithType + "_" + segmentName + "_" + UUID.randomUUID() + TarGzCompressionUtils.TAR_GZ_FILE_EXTENSION,
+          "Invalid table / segment name: %s, %s", tableNameWithType, segmentName);
+
       TarGzCompressionUtils.createTarGzFile(new File(tableDataManager.getTableDataDir(), segmentName), segmentTarFile);
 
       // Use segment uploader to upload the segment tar file to segment store and return the segment download url.


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