This is an automated email from the ASF dual-hosted git repository.
JingsongLi pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/paimon.git
The following commit(s) were added to refs/heads/master by this push:
new eb0dc176e2 [fix] fix pvfs bug in python (#8012)
eb0dc176e2 is described below
commit eb0dc176e2f3a5407cb540f4443440f31d38d239
Author: jerry <[email protected]>
AuthorDate: Thu May 28 21:20:20 2026 +0800
[fix] fix pvfs bug in python (#8012)
---
paimon-python/pypaimon/filesystem/pvfs.py | 13 ++++++++-
paimon-python/pypaimon/tests/pvfs_test.py | 44 +++++++++++++++++++++++++++++++
2 files changed, 56 insertions(+), 1 deletion(-)
diff --git a/paimon-python/pypaimon/filesystem/pvfs.py
b/paimon-python/pypaimon/filesystem/pvfs.py
index 236c00481c..a56128ca40 100644
--- a/paimon-python/pypaimon/filesystem/pvfs.py
+++ b/paimon-python/pypaimon/filesystem/pvfs.py
@@ -18,6 +18,7 @@
import datetime
import importlib
import logging
+import posixpath
import time
from abc import ABC
from dataclasses import dataclass
@@ -100,7 +101,12 @@ class PVFSTableIdentifier(PVFSIdentifier):
def get_actual_path(self, storage_location: str):
if self.sub_path:
- return '{}/{}'.format(storage_location.rstrip("/"),
self.sub_path.lstrip("/"))
+ normalized_sub = posixpath.normpath(self.sub_path)
+ if normalized_sub == ".." or normalized_sub.startswith("../") or
normalized_sub.startswith("/"):
+ raise ValueError(
+ "Path traversal detected: resolved path escapes table
storage boundary"
+ )
+ return '{}/{}'.format(storage_location.rstrip("/"), normalized_sub)
return storage_location
def get_virtual_location(self):
@@ -733,6 +739,11 @@ class PaimonVirtualFileSystem(fsspec.AbstractFileSystem):
return None
components = [component for component in
path_without_protocol.rstrip('/').split('/') if component]
+ for component in components:
+ if component == '..' or '\x00' in component:
+ raise ValueError(
+ "Invalid path: path traversal components are not allowed"
+ )
catalog: str = None
endpoint: str = self.options.get(CatalogOptions.URI)
if len(components) > 0:
diff --git a/paimon-python/pypaimon/tests/pvfs_test.py
b/paimon-python/pypaimon/tests/pvfs_test.py
index fd3c178535..813368c3fd 100644
--- a/paimon-python/pypaimon/tests/pvfs_test.py
+++ b/paimon-python/pypaimon/tests/pvfs_test.py
@@ -199,3 +199,47 @@ class PVFSTest(unittest.TestCase):
self.assertTrue(self.pvfs.created(table_data_new_virtual_path) is not
None)
self.assertTrue(self.pvfs.modified(table_data_new_virtual_path) is not
None)
self.assertEqual('Hello World',
self.pvfs.cat_file(date_file_new_virtual_path).decode('utf-8'))
+
+ def test_path_traversal_rejected_in_extract(self):
+ """Paths containing '..' components must be rejected at parse time."""
+ traversal_paths = [
+
f'pvfs://{self.catalog}/{self.database}/{self.table}/../other_table/secret.parquet',
+
f'pvfs://{self.catalog}/{self.database}/{self.table}/../../other_db/t/data',
+
f'pvfs://{self.catalog}/{self.database}/{self.table}/../../../etc/passwd',
+ f'pvfs://{self.catalog}/../{self.database}/{self.table}',
+ ]
+ for path in traversal_paths:
+ with self.assertRaises(ValueError, msg=f"Should reject: {path}"):
+ self.pvfs._extract_pvfs_identifier(path)
+
+ def test_path_traversal_rejected_in_get_actual_path(self):
+ """Even if '..' reaches get_actual_path, boundary check must block
it."""
+ from pypaimon.filesystem.pvfs import PVFSTableIdentifier
+ identifier = PVFSTableIdentifier(
+ endpoint="http://localhost",
+ catalog="cat",
+ database="db",
+ table="tbl",
+ sub_path="../../other_table/secret.parquet"
+ )
+ with self.assertRaises(ValueError):
+ identifier.get_actual_path("/warehouse/cat/db/tbl")
+
+ def test_null_byte_rejected(self):
+ """Null bytes in path components must be rejected."""
+ path =
f'pvfs://{self.catalog}/{self.database}/{self.table}/file\x00.parquet'
+ with self.assertRaises(ValueError):
+ self.pvfs._extract_pvfs_identifier(path)
+
+ def test_legitimate_subpaths_allowed(self):
+ """Normal sub-paths without traversal must still work."""
+ from pypaimon.filesystem.pvfs import PVFSTableIdentifier
+ identifier = PVFSTableIdentifier(
+ endpoint="http://localhost",
+ catalog="cat",
+ database="db",
+ table="tbl",
+ sub_path="partition=1/bucket-0/data.parquet"
+ )
+ result = identifier.get_actual_path("/warehouse/cat/db/tbl")
+ self.assertEqual(result,
"/warehouse/cat/db/tbl/partition=1/bucket-0/data.parquet")