PVFS
This documentation is for an unreleased version of Apache Paimon. We recommend you use the latest stable version.

Paimon Virtual Storage #

The REST Catalog provides built-in storage, including Paimon Table, Format Table, and Object Table (also known as Fileset or Volume), both of which require direct access to the file system. And our REST Catalog generates UUID paths, which makes it difficult to directly access the file system.

So there is PVFS, which can allow users to access it through similar methods pvfs://catalog_name/database_name/table_name/, use the path to access all internal tables in the REST Catalog, including Paimon Table, Format Table, and Object Table. Another advantage is that all user access to this file system is through the permission system of Paimon REST Catalog, without the need to maintain another file system permission system.

API Behavior #

For example, if you have a catalog named ‘my_catalog’, the list behavior should be:

  • listStatus(Path('pvfs://my_catalog/')): return all databases, only virtual paths in FileStatus.
  • listStatus(Path('pvfs://my_catalog/my_database')): return all tables, only virtual paths in FileStatus.

All paths return virtual paths, reading and writing files will actually read and write data according to the true path of the table.

  • newInputStream(Path('pvfs://my_catalog/my_database/my_table')): get the real path from rest server, and use real filesystem to read data.

Java SDK #

Provide a Java SDK to implement Hadoop FileSystem. In this way, compute engines can integrate ‘PVFS’ very easy.

For example, Java code can do:

Configuration conf = new Configuration();
conf.set("fs.AbstractFileSystem.pvfs.impl", "org.apache.paimon.vfs.hadoop.Pvfs");
conf.set("fs.pvfs.impl", "org.apache.paimon.vfs.hadoop.PaimonVirtualFileSystem");
conf.set("fs.pvfs.uri", "http://localhost:10000");
conf.set("fs.pvfs.token.provider", "bear");
conf.set("fs.pvfs.token", "token");
Path path = new Path("pvfs://catalog_name/database_name/table_name/a.csv");
FileSystem fs = path.getFileSystem(conf);
FileStatus fileStatus = fs.getFileStatus(path);

For example, Spark SQL can do:

val spark = SparkSession.builder()
.appName("PVFS CSV Analysis")
.config("spark.hadoop.fs.pvfs.impl", "org.apache.paimon.vfs.hadoop.PaimonVirtualFileSystem")
.config("spark.hadoop.fs.pvfs.uri", "http://localhost:10000")
.config("spark.hadoop.fs.pvfs.token.provider", "bear")
.config("spark.hadoop.fs.pvfs.token", "token")
.getOrCreate()
spark.sql(
s"""
|CREATE TEMPORARY VIEW csv_table
|USING csv
|OPTIONS (
|  path 'pvfs://catalog_name/database_name/my_format_table_name/a.csv',
|  header 'true',
|  inferSchema 'true'
|)
""".stripMargin
)

spark.sql("SELECT * FROM csv_table LIMIT 5").show()

For example, use Hadoop shell command:

<!-- Configure following configuration in hadoop `core-site.xml` -->
<property>
  <name>fs.AbstractFileSystem.pvfs.impl</name>
  <value>org.apache.paimon.vfs.hadoop.Pvfs</value>
</property>

<property>
  <name>fs.pvfs.impl</name>
  <value>org.apache.paimon.vfs.hadoop.PaimonVirtualFileSystem</value>
</property>

<property>
  <name>fs.pvfs.uri</name>
  <value>http://localhost:10000</value>
</property>

<property>
  <name>fs.pvfs.token.provider</name>
  <value>bear</value>
</property>

<property>
  <name>fs.pvfs.token</name>
  <value>token</value>
</property>

Example: execute hadoop shell to list the virtual path

./${HADOOP_HOME}/bin/hadoop dfs -ls pvfs://catalog_name/database_name/table_name

Python SDK #

Python SDK provide fsspec style API, can be easily integrated to Python ecesystem.

For example, Python code can do:

import pypaimon

options = {
"uri": 'key',
'token.provider' = 'bear'
'token' = '<token>'
}
fs = pypaimon.PaimonVirtualFileSystem(options)
fs.ls("pvfs://catalog_name/database_name/table_name")

For example, Pyarrow can do:

import pypaimon
import pyarrow.parquet as pq

options = {
"uri": 'key',
'token.provider' = 'bear'
'token' = '<token>'
}
fs = pypaimon.PaimonVirtualFileSystem(options)
path = 'pvfs://catalog_name/database_name/table_name/a.parquet'
dataset = pq.ParquetDataset(path, filesystem=fs)
table = dataset.read()
df = table.to_pandas()

For example, Ray can do:

import pypaimon
import ray

options = {
"uri": 'key',
'token.provider' = 'bear'
'token' = '<token>'
}
fs = pypaimon.PaimonVirtualFileSystem(options)

ds = ray.data.read_parquet(filesystem=fs,paths="pvfs://....parquet")
Edit This Page
Copyright © 2025 The Apache Software Foundation. Apache Paimon, Paimon, and its feather logo are trademarks of The Apache Software Foundation.